From 81c4d19c612208b932f8579427cb895385336c6e Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 19 Feb 2013 12:43:13 -0800 Subject: [PATCH 001/531] Maven and sbt build changes for SparkGraph. --- graph/pom.xml | 106 ++++++++++++++++++ .../src/main/scala/spark/graph/package.scala | 7 ++ pom.xml | 1 + project/SparkBuild.scala | 6 +- run | 2 + run2.cmd | 2 + 6 files changed, 123 insertions(+), 1 deletion(-) create mode 100644 graph/pom.xml create mode 100644 graph/src/main/scala/spark/graph/package.scala diff --git a/graph/pom.xml b/graph/pom.xml new file mode 100644 index 0000000000000..1cd9cda98bc4b --- /dev/null +++ b/graph/pom.xml @@ -0,0 +1,106 @@ + + + 4.0.0 + + org.spark-project + parent + 0.7.0-SNAPSHOT + ../pom.xml + + + org.spark-project + spark-graph + jar + Spark Graph + http://spark-project.org/ + + + + org.eclipse.jetty + jetty-server + + + + org.scalatest + scalatest_${scala.version} + test + + + org.scalacheck + scalacheck_${scala.version} + test + + + + target/scala-${scala.version}/classes + target/scala-${scala.version}/test-classes + + + org.scalatest + scalatest-maven-plugin + + + + + + + hadoop1 + + + org.spark-project + spark-core + ${project.version} + hadoop1 + + + org.apache.hadoop + hadoop-core + provided + + + + + + org.apache.maven.plugins + maven-jar-plugin + + hadoop1 + + + + + + + hadoop2 + + + org.spark-project + spark-core + ${project.version} + hadoop2 + + + org.apache.hadoop + hadoop-core + provided + + + org.apache.hadoop + hadoop-client + provided + + + + + + org.apache.maven.plugins + maven-jar-plugin + + hadoop2 + + + + + + + diff --git a/graph/src/main/scala/spark/graph/package.scala b/graph/src/main/scala/spark/graph/package.scala new file mode 100644 index 0000000000000..35b9f24461330 --- /dev/null +++ b/graph/src/main/scala/spark/graph/package.scala @@ -0,0 +1,7 @@ +package spark + +package object graph { + type Vid = Int + type Pid = Int + type Status = Boolean +} diff --git a/pom.xml b/pom.xml index 3ea989a082b2a..11a1f87ed3715 100644 --- a/pom.xml +++ b/pom.xml @@ -41,6 +41,7 @@ core bagel examples + graph streaming repl repl-bin diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 03b8094f7dce5..3e383b1300b37 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -17,7 +17,7 @@ object SparkBuild extends Build { //val HADOOP_VERSION = "2.0.0-mr1-cdh4.1.1" //val HADOOP_MAJOR_VERSION = "2" - lazy val root = Project("root", file("."), settings = rootSettings) aggregate(core, repl, examples, bagel, streaming) + lazy val root = Project("root", file("."), settings = rootSettings) aggregate(core, repl, examples, bagel, graph, streaming) lazy val core = Project("core", file("core"), settings = coreSettings) @@ -27,6 +27,8 @@ object SparkBuild extends Build { lazy val bagel = Project("bagel", file("bagel"), settings = bagelSettings) dependsOn (core) + lazy val graph = Project("graph", file("graph"), settings = graphSettings) dependsOn (core) + lazy val streaming = Project("streaming", file("streaming"), settings = streamingSettings) dependsOn (core) // A configuration to set an alternative publishLocalConfiguration @@ -161,6 +163,8 @@ object SparkBuild extends Build { def bagelSettings = sharedSettings ++ Seq(name := "spark-bagel") + def graphSettings = sharedSettings ++ Seq(name := "spark-graph") + def streamingSettings = sharedSettings ++ Seq( name := "spark-streaming", libraryDependencies ++= Seq( diff --git a/run b/run index a0946294497a8..4ecaa70511729 100755 --- a/run +++ b/run @@ -63,6 +63,7 @@ CORE_DIR="$FWDIR/core" REPL_DIR="$FWDIR/repl" EXAMPLES_DIR="$FWDIR/examples" BAGEL_DIR="$FWDIR/bagel" +GRAPH_DIR="$FWDIR/graph" STREAMING_DIR="$FWDIR/streaming" PYSPARK_DIR="$FWDIR/python" @@ -95,6 +96,7 @@ if [ -e repl-bin/target ]; then done fi CLASSPATH+=":$BAGEL_DIR/target/scala-$SCALA_VERSION/classes" +CLASSPATH+=":$GRAPH_DIR/target/scala-$SCALA_VERSION/classes" for jar in `find $PYSPARK_DIR/lib -name '*jar'`; do CLASSPATH+=":$jar" done diff --git a/run2.cmd b/run2.cmd index 67f1e465e47b3..eb5e283e9f4ca 100644 --- a/run2.cmd +++ b/run2.cmd @@ -34,6 +34,7 @@ set CORE_DIR=%FWDIR%core set REPL_DIR=%FWDIR%repl set EXAMPLES_DIR=%FWDIR%examples set BAGEL_DIR=%FWDIR%bagel +set GRAPH_DIR=%FWDIR%graph set PYSPARK_DIR=%FWDIR%python rem Build up classpath @@ -45,6 +46,7 @@ for /R "%FWDIR%\lib_managed\bundles" %%j in (*.jar) do set CLASSPATH=!CLASSPATH! for /R "%REPL_DIR%\lib" %%j in (*.jar) do set CLASSPATH=!CLASSPATH!;%%j for /R "%PYSPARK_DIR%\lib" %%j in (*.jar) do set CLASSPATH=!CLASSPATH!;%%j set CLASSPATH=%CLASSPATH%;%BAGEL_DIR%\target\scala-%SCALA_VERSION%\classes +set CLASSPATH=%CLASSPATH%;%GRAPH_DIR%\target\scala-%SCALA_VERSION%\classes rem Figure out whether to run our class with java or with the scala launcher. rem In most cases, we'd prefer to execute our process with java because scala From 28ebe04496fb0163409d471672e21822b9d24d77 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 2 Apr 2013 01:16:08 +0800 Subject: [PATCH 002/531] Added a Graph class that supports joining vertices with edges. --- graph/src/main/scala/spark/graph/package.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/graph/src/main/scala/spark/graph/package.scala b/graph/src/main/scala/spark/graph/package.scala index 35b9f24461330..90ccba431c4a0 100644 --- a/graph/src/main/scala/spark/graph/package.scala +++ b/graph/src/main/scala/spark/graph/package.scala @@ -1,6 +1,7 @@ package spark package object graph { + type Vid = Int type Pid = Int type Status = Boolean From d7011b0f7868552a7a019fc655c69cd11a5bf0da Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 2 Apr 2013 01:17:44 +0800 Subject: [PATCH 003/531] Added a Graph class that supports joining vertices with edges. --- .../scala/spark/graph/EdgeDirection.scala | 9 + .../spark/graph/EdgeWithVerticesRDD.scala | 83 +++++++++ graph/src/main/scala/spark/graph/Graph.scala | 173 ++++++++++++++++++ .../spark/graph/GraphKryoRegistrator.scala | 28 +++ graph/src/main/scala/spark/graph/Timer.scala | 14 ++ 5 files changed, 307 insertions(+) create mode 100644 graph/src/main/scala/spark/graph/EdgeDirection.scala create mode 100644 graph/src/main/scala/spark/graph/EdgeWithVerticesRDD.scala create mode 100644 graph/src/main/scala/spark/graph/Graph.scala create mode 100644 graph/src/main/scala/spark/graph/GraphKryoRegistrator.scala create mode 100644 graph/src/main/scala/spark/graph/Timer.scala diff --git a/graph/src/main/scala/spark/graph/EdgeDirection.scala b/graph/src/main/scala/spark/graph/EdgeDirection.scala new file mode 100644 index 0000000000000..a5bd9749bfbfe --- /dev/null +++ b/graph/src/main/scala/spark/graph/EdgeDirection.scala @@ -0,0 +1,9 @@ +package spark.graph + + +object EdgeDirection extends Enumeration { + + type EdgeDirection = Value + + val None, In, Out, Both = Value +} diff --git a/graph/src/main/scala/spark/graph/EdgeWithVerticesRDD.scala b/graph/src/main/scala/spark/graph/EdgeWithVerticesRDD.scala new file mode 100644 index 0000000000000..3bc73e1946c36 --- /dev/null +++ b/graph/src/main/scala/spark/graph/EdgeWithVerticesRDD.scala @@ -0,0 +1,83 @@ +package spark.graph + +import spark.{Aggregator, HashPartitioner, Partition, RDD, SparkEnv, TaskContext} +import spark.{Dependency, OneToOneDependency, ShuffleDependency} +import spark.SparkContext._ +import spark.graph.Graph.EdgePartition + + +private[graph] +class EdgeWithVerticesPartition(idx: Int, val eTablePartition: Partition) extends Partition { + override val index: Int = idx + override def hashCode(): Int = idx +} + + +/** + * A RDD that brings together edge data with its associated vertex data. + */ +private[graph] +class EdgeWithVerticesRDD[VD: Manifest, ED: Manifest]( + @transient vTable: RDD[(Vid, (VD, Array[Pid]))], + eTable: RDD[(Pid, EdgePartition[ED])]) + extends RDD[EdgeWithVertices[VD, ED]](eTable.context, Nil) { + + @transient + private val shuffleDependency = { + // Join vid2pid and vTable, generate a shuffle dependency on the joined result, and get + // the shuffle id so we can use it on the slave. + val vTableReplicated = vTable.flatMap { case (vid, (vdata, pids)) => + pids.iterator.map { pid => (pid, (vid, vdata)) } + } + new ShuffleDependency(vTableReplicated, eTable.partitioner.get) + } + + private val shuffleId = shuffleDependency.shuffleId + + override def getDependencies: List[Dependency[_]] = { + List(new OneToOneDependency(eTable), shuffleDependency) + } + + override def getPartitions = Array.tabulate[Partition](eTable.partitions.size) { + i => new EdgeWithVerticesPartition(i, eTable.partitions(i)): Partition + } + + override val partitioner = eTable.partitioner + + override def getPreferredLocations(s: Partition) = + eTable.preferredLocations(s.asInstanceOf[EdgeWithVerticesPartition].eTablePartition) + + override def compute(s: Partition, context: TaskContext): Iterator[EdgeWithVertices[VD, ED]] = { + val split = s.asInstanceOf[EdgeWithVerticesPartition] + + // Fetch the vertices and put them in a hashmap. + // TODO: use primitive hashmaps for primitive VD types. + val vmap = new it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap[VD]//(1000000) + val fetcher = SparkEnv.get.shuffleFetcher + fetcher.fetch[Pid, (Vid, VD)](shuffleId, split.index, context.taskMetrics).foreach { + case (pid, (vid, vdata)) => vmap.put(vid, vdata) + } + + val (pid, edgePartition) = eTable.iterator(split.eTablePartition, context).next() + .asInstanceOf[(Pid, EdgePartition[ED])] + + // Return an iterator that looks up the hash map to find matching vertices for each edge. + new Iterator[EdgeWithVertices[VD, ED]] { + private var pos = 0 + private val e = new EdgeWithVertices[VD, ED] + e.src = new Vertex[VD] + e.dst = new Vertex[VD] + + override def hasNext: Boolean = pos < edgePartition.size + override def next() = { + e.src.id = edgePartition.srcIds.getInt(pos) + e.src.data = vmap.get(e.src.id) + e.dst.id = edgePartition.dstIds.getInt(pos) + e.dst.data = vmap.get(e.dst.id) + e.data = edgePartition.data(pos) + pos += 1 + e + } + } + } +} diff --git a/graph/src/main/scala/spark/graph/Graph.scala b/graph/src/main/scala/spark/graph/Graph.scala new file mode 100644 index 0000000000000..2719ed01931e6 --- /dev/null +++ b/graph/src/main/scala/spark/graph/Graph.scala @@ -0,0 +1,173 @@ +package spark.graph + +import scala.collection.JavaConversions._ +import scala.collection.mutable.ArrayBuffer + +import com.esotericsoftware.kryo._ + +import it.unimi.dsi.fastutil.ints.IntArrayList + +import spark.{ClosureCleaner, HashPartitioner, KryoRegistrator, SparkContext, RDD} +import spark.SparkContext._ +import spark.storage.StorageLevel + + + +class Vertex[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) VD] { + var id: Vid = _ + var data: VD = _ + + def this(id: Int, data: VD) { this(); this.id = id; this.data = data; } +} + + +class Edge[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED] { + var src: Vid = _ + var dst: Vid = _ + var data: ED = _ + + def this(src: Vid, dst: Vid, data: ED) { + this(); this.src = src; this.dst = dst; this.data = data; + } +} + + +class EdgeWithVertices[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) VD, + @specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED] { + + var src: Vertex[VD] = _ + var dst: Vertex[VD] = _ + var data: ED = _ + + def otherVertex(vid: Vid): Vertex[VD] = if (src.id == vid) dst else src + + def vertex(vid: Vid): Vertex[VD] = if (src.id == vid) src else dst +} + + +/** + * A Graph RDD that supports computation on graphs. + */ +class Graph[VD: Manifest, ED: Manifest]( + private val _vertices: RDD[Vertex[VD]], + private val _edges: RDD[Edge[ED]]) { + + import Graph.EdgePartition + + var numEdgePartitions = 5 + var numVertexPartitions = 5 + + private val eTable: RDD[(Pid, EdgePartition[ED])] = Graph.createETable( + _edges, numEdgePartitions) + + private val vTable: RDD[(Vid, (VD, Array[Pid]))] = Graph.createVTable( + _vertices, eTable, numVertexPartitions) + + def edges: RDD[Edge[ED]] = eTable.mapPartitions { iter => iter.next._2.iterator } + + def edgesWithVertices: RDD[EdgeWithVertices[VD, ED]] = new EdgeWithVerticesRDD(vTable, eTable) + +} + + +object Graph { + + /** + * A partition of edges. This is created so we can store edge data in columnar format so it is + * more efficient to store the data in memory. + */ + private[graph] + class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED: Manifest] { + val srcIds: IntArrayList = new IntArrayList + val dstIds: IntArrayList = new IntArrayList + // TODO: Specialize data. + val data: ArrayBuffer[ED] = new ArrayBuffer[ED] + + /** Add a new edge to the partition. */ + def add(src: Vid, dst: Vid, d: ED) { + srcIds.add(src) + dstIds.add(dst) + data += d + } + + def trim() { + srcIds.trim() + dstIds.trim() + } + + def size: Int = srcIds.size + + def iterator = new Iterator[Edge[ED]] { + private var edge = new Edge[ED] + private var pos = 0 + + override def hasNext: Boolean = pos < size + + override def next(): Edge[ED] = { + edge.src = srcIds.get(pos) + edge.dst = dstIds.get(pos) + edge.data = data(pos) + pos += 1 + edge + } + } + } + + private[graph] + def createVTable[VD: Manifest, ED: Manifest]( + vertices: RDD[Vertex[VD]], + eTable: RDD[(Pid, EdgePartition[ED])], + numPartitions: Int) = { + val partitioner = new HashPartitioner(numPartitions) + + // A key-value RDD. The key is a vertex id, and the value is a list of + // partitions that contains edges referencing the vertex. + val vid2pid : RDD[(Int, Seq[Pid])] = eTable.mapPartitions { iter => + val (pid, edgePartition) = iter.next() + val vSet = new it.unimi.dsi.fastutil.ints.IntOpenHashSet + var i = 0 + while (i < edgePartition.srcIds.size) { + vSet.add(edgePartition.srcIds.getInt(i)) + vSet.add(edgePartition.dstIds.getInt(i)) + i += 1 + } + vSet.iterator.map { vid => (vid.intValue, pid) } + }.groupByKey(partitioner) + + vertices + .map { v => (v.id, v.data) } + .partitionBy(partitioner) + .leftOuterJoin(vid2pid) + .mapValues { + case (vdata, None) => (vdata, Array.empty[Pid]) + case (vdata, Some(pids)) => (vdata, pids.toArray) + } + .cache() + } + + /** + * Create the edge table RDD, which is much more efficient for Java heap storage than the + * normal edges data structure (RDD[(Vid, Vid, ED)]). + * + * The edge table contains multiple partitions, and each partition contains only one RDD + * key-value pair: the key is the partition id, and the value is an EdgePartition object + * containing all the edges in a partition. + */ + private[graph] + def createETable[ED: Manifest](edges: RDD[Edge[ED]], numPartitions: Int) + : RDD[(Pid, EdgePartition[ED])] = { + + edges.map { e => + // Random partitioning based on the source vertex id. + (math.abs(e.src) % numPartitions, (e.src, e.dst, e.data)) + } + .partitionBy(new HashPartitioner(numPartitions)) + .mapPartitionsWithIndex({ (pid, iter) => + val edgePartition = new Graph.EdgePartition[ED] + iter.foreach { case (_, (src, dst, data)) => edgePartition.add(src, dst, data) } + Iterator((pid, edgePartition)) + }, preservesPartitioning = true) + .cache() + } +} + diff --git a/graph/src/main/scala/spark/graph/GraphKryoRegistrator.scala b/graph/src/main/scala/spark/graph/GraphKryoRegistrator.scala new file mode 100644 index 0000000000000..e72e500fb8fc0 --- /dev/null +++ b/graph/src/main/scala/spark/graph/GraphKryoRegistrator.scala @@ -0,0 +1,28 @@ +package spark.graph + +import com.esotericsoftware.kryo.Kryo + +import spark.KryoRegistrator + + +class GraphKryoRegistrator extends KryoRegistrator { + + def registerClasses(kryo: Kryo) { + kryo.register(classOf[(Int, Float, Float)]) + registerClass[(Int, Float, Float), Float](kryo) + registerClass[(Int, Float), Float](kryo) + registerClass[Int, Float](kryo) + registerClass[Float, Float](kryo) + + // This avoids a large number of hash table lookups. + kryo.setReferences(false) + } + + private def registerClass[VD: Manifest, ED: Manifest](kryo: Kryo) { + //kryo.register(classManifest[VD].erasure) + // kryo.register(classManifest[ED].erasure) + kryo.register(classOf[(Vid, Vid, ED)]) + kryo.register(classOf[(Vid, ED)]) + //kryo.register(classOf[EdgeBlockRecord[ED]]) + } +} diff --git a/graph/src/main/scala/spark/graph/Timer.scala b/graph/src/main/scala/spark/graph/Timer.scala new file mode 100644 index 0000000000000..5ed4d70e1b967 --- /dev/null +++ b/graph/src/main/scala/spark/graph/Timer.scala @@ -0,0 +1,14 @@ +package spark.graph + + +class Timer { + + var lastTime = System.currentTimeMillis + + def tic = { + val currentTime = System.currentTimeMillis + val elapsedTime = (currentTime - lastTime)/1000.0 + lastTime = currentTime + elapsedTime + } +} From 25c71b185de330074674fbc390dab11de108e7ff Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 2 Apr 2013 01:26:20 +0800 Subject: [PATCH 004/531] Added a vertices method to Graph. --- graph/src/main/scala/spark/graph/Graph.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/graph/src/main/scala/spark/graph/Graph.scala b/graph/src/main/scala/spark/graph/Graph.scala index 2719ed01931e6..1bf36c2e0dfe2 100644 --- a/graph/src/main/scala/spark/graph/Graph.scala +++ b/graph/src/main/scala/spark/graph/Graph.scala @@ -63,6 +63,8 @@ class Graph[VD: Manifest, ED: Manifest]( private val vTable: RDD[(Vid, (VD, Array[Pid]))] = Graph.createVTable( _vertices, eTable, numVertexPartitions) + def vertices: RDD[Vertex[VD]] = vTable.map { case(vid, (data, pids)) => new Vertex(vid, data) } + def edges: RDD[Edge[ED]] = eTable.mapPartitions { iter => iter.next._2.iterator } def edgesWithVertices: RDD[EdgeWithVertices[VD, ED]] = new EdgeWithVerticesRDD(vTable, eTable) From d63c895945cc41b0bad7e9be96051489d0098c60 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 3 Apr 2013 00:42:33 +0800 Subject: [PATCH 005/531] Partial checkin of graphlab module. --- .../spark/graph/EdgeWithVerticesRDD.scala | 11 +-- graph/src/main/scala/spark/graph/Graph.scala | 67 ++++++++++--------- .../src/main/scala/spark/graph/GraphLab.scala | 45 +++++++++++++ .../src/main/scala/spark/graph/package.scala | 7 ++ 4 files changed, 93 insertions(+), 37 deletions(-) create mode 100644 graph/src/main/scala/spark/graph/GraphLab.scala diff --git a/graph/src/main/scala/spark/graph/EdgeWithVerticesRDD.scala b/graph/src/main/scala/spark/graph/EdgeWithVerticesRDD.scala index 3bc73e1946c36..e787fe4e2c172 100644 --- a/graph/src/main/scala/spark/graph/EdgeWithVerticesRDD.scala +++ b/graph/src/main/scala/spark/graph/EdgeWithVerticesRDD.scala @@ -20,7 +20,7 @@ private[graph] class EdgeWithVerticesRDD[VD: Manifest, ED: Manifest]( @transient vTable: RDD[(Vid, (VD, Array[Pid]))], eTable: RDD[(Pid, EdgePartition[ED])]) - extends RDD[EdgeWithVertices[VD, ED]](eTable.context, Nil) { + extends RDD[VertexHashMap, Iterator[EdgeWithVertices[VD, ED]]](eTable.context, Nil) { @transient private val shuffleDependency = { @@ -47,12 +47,14 @@ class EdgeWithVerticesRDD[VD: Manifest, ED: Manifest]( override def getPreferredLocations(s: Partition) = eTable.preferredLocations(s.asInstanceOf[EdgeWithVerticesPartition].eTablePartition) - override def compute(s: Partition, context: TaskContext): Iterator[EdgeWithVertices[VD, ED]] = { + override def compute(s: Partition, context: TaskContext) + : Iterator[VertexHashMap, Iterator[EdgeWithVertices[VD, ED]]] = { + val split = s.asInstanceOf[EdgeWithVerticesPartition] // Fetch the vertices and put them in a hashmap. // TODO: use primitive hashmaps for primitive VD types. - val vmap = new it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap[VD]//(1000000) + val vmap = new VertexHashMap[VD]//(1000000) val fetcher = SparkEnv.get.shuffleFetcher fetcher.fetch[Pid, (Vid, VD)](shuffleId, split.index, context.taskMetrics).foreach { case (pid, (vid, vdata)) => vmap.put(vid, vdata) @@ -62,7 +64,7 @@ class EdgeWithVerticesRDD[VD: Manifest, ED: Manifest]( .asInstanceOf[(Pid, EdgePartition[ED])] // Return an iterator that looks up the hash map to find matching vertices for each edge. - new Iterator[EdgeWithVertices[VD, ED]] { + val iter = new Iterator[EdgeWithVertices[VD, ED]] { private var pos = 0 private val e = new EdgeWithVertices[VD, ED] e.src = new Vertex[VD] @@ -79,5 +81,6 @@ class EdgeWithVerticesRDD[VD: Manifest, ED: Manifest]( e } } + (vmap, iter) } } diff --git a/graph/src/main/scala/spark/graph/Graph.scala b/graph/src/main/scala/spark/graph/Graph.scala index 1bf36c2e0dfe2..4f1b3b44d525a 100644 --- a/graph/src/main/scala/spark/graph/Graph.scala +++ b/graph/src/main/scala/spark/graph/Graph.scala @@ -3,38 +3,27 @@ package spark.graph import scala.collection.JavaConversions._ import scala.collection.mutable.ArrayBuffer -import com.esotericsoftware.kryo._ - import it.unimi.dsi.fastutil.ints.IntArrayList -import spark.{ClosureCleaner, HashPartitioner, KryoRegistrator, SparkContext, RDD} +import spark.{ClosureCleaner, HashPartitioner, SparkContext, RDD} import spark.SparkContext._ +import spark.graph.Graph.EdgePartition import spark.storage.StorageLevel +case class Vertex[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) VD] ( + var id: Vid = 0, + var data: VD = nullValue[VD]) -class Vertex[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) VD] { - var id: Vid = _ - var data: VD = _ - - def this(id: Int, data: VD) { this(); this.id = id; this.data = data; } -} - - -class Edge[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED] { - var src: Vid = _ - var dst: Vid = _ - var data: ED = _ - def this(src: Vid, dst: Vid, data: ED) { - this(); this.src = src; this.dst = dst; this.data = data; - } -} +case class Edge[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED] ( + var src: Vid = 0, + var dst: Vid = 0, + var data: ED = nullValue[ED]) class EdgeWithVertices[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) VD, @specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED] { - var src: Vertex[VD] = _ var dst: Vertex[VD] = _ var data: ED = _ @@ -49,25 +38,38 @@ class EdgeWithVertices[@specialized(Char, Int, Boolean, Byte, Long, Float, Doubl * A Graph RDD that supports computation on graphs. */ class Graph[VD: Manifest, ED: Manifest]( - private val _vertices: RDD[Vertex[VD]], - private val _edges: RDD[Edge[ED]]) { - - import Graph.EdgePartition + val rawVertices: RDD[Vertex[VD]], + val rawEdges: RDD[Edge[ED]]) { var numEdgePartitions = 5 var numVertexPartitions = 5 - private val eTable: RDD[(Pid, EdgePartition[ED])] = Graph.createETable( - _edges, numEdgePartitions) + val vertexPartitioner = new HashPartitioner(numVertexPartitions) + + val edgePartitioner = new HashPartitioner(numEdgePartitions) + + lazy val eTable: RDD[(Pid, EdgePartition[ED])] = Graph.createETable( + rawEdges, numEdgePartitions) - private val vTable: RDD[(Vid, (VD, Array[Pid]))] = Graph.createVTable( - _vertices, eTable, numVertexPartitions) + lazy val vTable: RDD[(Vid, (VD, Array[Pid]))] = Graph.createVTable( + rawVertices, eTable, numVertexPartitions) - def vertices: RDD[Vertex[VD]] = vTable.map { case(vid, (data, pids)) => new Vertex(vid, data) } + def vertices(): RDD[Vertex[VD]] = vTable.map { case(vid, (data, pids)) => new Vertex(vid, data) } - def edges: RDD[Edge[ED]] = eTable.mapPartitions { iter => iter.next._2.iterator } + def edges(): RDD[Edge[ED]] = eTable.mapPartitions { iter => iter.next._2.iterator } - def edgesWithVertices: RDD[EdgeWithVertices[VD, ED]] = new EdgeWithVerticesRDD(vTable, eTable) + def edgesWithVertices(): RDD[EdgeWithVertices[VD, ED]] = { + (new EdgeWithVerticesRDD(vTable, eTable)).mapPartitions { case(vmap, iter) => iter } + } + + def mapPartitions[U: ClassManifest]( + f: (VertexHashMap, Iterator[EdgeWithVertices[VD, ED]]) => Iterator[U], + preservesPartitioning: Boolean = false): RDD[U] = { + (new EdgeWithVerticesRDD(vTable, eTable)).mapPartitions({ part => + val (vmap, iter) = part.next() + iter.mapPartitions(f) + }, preservesPartitioning) + } } @@ -75,8 +77,7 @@ class Graph[VD: Manifest, ED: Manifest]( object Graph { /** - * A partition of edges. This is created so we can store edge data in columnar format so it is - * more efficient to store the data in memory. + * A partition of edges in 3 large columnar arrays. */ private[graph] class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED: Manifest] { diff --git a/graph/src/main/scala/spark/graph/GraphLab.scala b/graph/src/main/scala/spark/graph/GraphLab.scala new file mode 100644 index 0000000000000..ebf001937b312 --- /dev/null +++ b/graph/src/main/scala/spark/graph/GraphLab.scala @@ -0,0 +1,45 @@ +package spark.graph + + +class GraphLab { + + def iterateGAS[A: Manifest, VD: Manifest, ED: Manifest]( + graph: Graph[VD, ED], + gather: (Vid, EdgeWithVertices[VD, ED]) => A, + merge: (A, A) => A, + default: A, + apply: (Vertex[VD], A) => VD, + numIter: Int, + gatherEdges: EdgeDirection = EdgeDirection.In) = { + + val g = new Graph[(VD, A), ED](graph.rawVertices.map(v => (v, default)), graph.rawEdges) + + var i = 0 + while (i < numIter) { + + val gatherTable = g.mapPartitions { case(vmap, iter) => + val edgeSansAcc = new EdgeWithVertices[VD, ED]() + iter.map { edge: EdgeWithVertices[(VD, A), ED] => + edgeSansAcc.data = edge.data + edgeSansAcc.src.data = edge.src.data._1 + edgeSansAcc.dst.data = edge.dst.data._1 + edgeSansAcc.src.id = edge.src.id + edgeSansAcc.dst.id = edge.dst.id + if (gatherEdges == EdgeDirection.In || gatherEdges == EdgeDirection.Both) { + edge.dst.data._2 = merge(edge.dst.data._2, gather(edgeSansAcc.dst.id, edgeSansAcc)) + } + if (gatherEdges == EdgeDirection.Out || gatherEdges == EdgeDirection.Both) { + edge.src.data._2 = merge(edge.src.data._2, gather(edgeSansAcc.src.id, edgeSansAcc)) + } + } + + vmap.int2ObjectEntrySet().fastIterator().map{ case (vid, (vdata, acc)) => (vid, acc) } + }.reduceByKey(graph.vertexPartitioner, false) + + gatherTable + + i += 1 + } + } + +} diff --git a/graph/src/main/scala/spark/graph/package.scala b/graph/src/main/scala/spark/graph/package.scala index 90ccba431c4a0..e900a27b27a38 100644 --- a/graph/src/main/scala/spark/graph/package.scala +++ b/graph/src/main/scala/spark/graph/package.scala @@ -5,4 +5,11 @@ package object graph { type Vid = Int type Pid = Int type Status = Boolean + type VertexHashMap = it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap + + /** + * Return the default null-like value for a data type T. + */ + def nullValue[T] = null.asInstanceOf[T] + } From cb0efe92d1e706472aa219617ec02df3b77dc9d3 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 3 Apr 2013 23:12:01 +0800 Subject: [PATCH 006/531] Oh wow it finally compiles! --- .../spark/graph/EdgeWithVerticesRDD.scala | 8 +- graph/src/main/scala/spark/graph/Graph.scala | 106 ++++++++++++++---- .../src/main/scala/spark/graph/GraphLab.scala | 37 ++++-- .../src/main/scala/spark/graph/package.scala | 2 +- 4 files changed, 115 insertions(+), 38 deletions(-) diff --git a/graph/src/main/scala/spark/graph/EdgeWithVerticesRDD.scala b/graph/src/main/scala/spark/graph/EdgeWithVerticesRDD.scala index e787fe4e2c172..f200f7cbf668c 100644 --- a/graph/src/main/scala/spark/graph/EdgeWithVerticesRDD.scala +++ b/graph/src/main/scala/spark/graph/EdgeWithVerticesRDD.scala @@ -17,10 +17,10 @@ class EdgeWithVerticesPartition(idx: Int, val eTablePartition: Partition) extend * A RDD that brings together edge data with its associated vertex data. */ private[graph] -class EdgeWithVerticesRDD[VD: Manifest, ED: Manifest]( +class EdgeWithVerticesRDD[VD: ClassManifest, ED: ClassManifest]( @transient vTable: RDD[(Vid, (VD, Array[Pid]))], eTable: RDD[(Pid, EdgePartition[ED])]) - extends RDD[VertexHashMap, Iterator[EdgeWithVertices[VD, ED]]](eTable.context, Nil) { + extends RDD[(VertexHashMap[VD], Iterator[EdgeWithVertices[VD, ED]])](eTable.context, Nil) { @transient private val shuffleDependency = { @@ -48,7 +48,7 @@ class EdgeWithVerticesRDD[VD: Manifest, ED: Manifest]( eTable.preferredLocations(s.asInstanceOf[EdgeWithVerticesPartition].eTablePartition) override def compute(s: Partition, context: TaskContext) - : Iterator[VertexHashMap, Iterator[EdgeWithVertices[VD, ED]]] = { + : Iterator[(VertexHashMap[VD], Iterator[EdgeWithVertices[VD, ED]])] = { val split = s.asInstanceOf[EdgeWithVerticesPartition] @@ -81,6 +81,6 @@ class EdgeWithVerticesRDD[VD: Manifest, ED: Manifest]( e } } - (vmap, iter) + Iterator((vmap, iter)) } } diff --git a/graph/src/main/scala/spark/graph/Graph.scala b/graph/src/main/scala/spark/graph/Graph.scala index 4f1b3b44d525a..85345c28b081a 100644 --- a/graph/src/main/scala/spark/graph/Graph.scala +++ b/graph/src/main/scala/spark/graph/Graph.scala @@ -5,7 +5,7 @@ import scala.collection.mutable.ArrayBuffer import it.unimi.dsi.fastutil.ints.IntArrayList -import spark.{ClosureCleaner, HashPartitioner, SparkContext, RDD} +import spark.{ClosureCleaner, HashPartitioner, RDD} import spark.SparkContext._ import spark.graph.Graph.EdgePartition import spark.storage.StorageLevel @@ -37,37 +37,102 @@ class EdgeWithVertices[@specialized(Char, Int, Boolean, Byte, Long, Float, Doubl /** * A Graph RDD that supports computation on graphs. */ -class Graph[VD: Manifest, ED: Manifest]( - val rawVertices: RDD[Vertex[VD]], - val rawEdges: RDD[Edge[ED]]) { +class Graph[VD: ClassManifest, ED: ClassManifest] protected ( + _rawVertices: RDD[Vertex[VD]], + _rawEdges: RDD[Edge[ED]], + _rawVTable: RDD[(Vid, (VD, Array[Pid]))], + _rawETable: RDD[(Pid, EdgePartition[ED])]) { + + def this(vertices: RDD[Vertex[VD]], edges: RDD[Edge[ED]]) = this(vertices, edges, null, null) + + protected var _cached = false + + def cache(): Graph[VD, ED] = { + eTable.cache() + vTable.cache() + _cached = true + this + } var numEdgePartitions = 5 var numVertexPartitions = 5 - val vertexPartitioner = new HashPartitioner(numVertexPartitions) + protected val vertexPartitioner = new HashPartitioner(numVertexPartitions) - val edgePartitioner = new HashPartitioner(numEdgePartitions) + protected val edgePartitioner = new HashPartitioner(numEdgePartitions) - lazy val eTable: RDD[(Pid, EdgePartition[ED])] = Graph.createETable( - rawEdges, numEdgePartitions) + protected lazy val eTable: RDD[(Pid, EdgePartition[ED])] = { + if (_rawETable == null) { + Graph.createETable(_rawEdges, numEdgePartitions) + } else { + _rawETable + } + } - lazy val vTable: RDD[(Vid, (VD, Array[Pid]))] = Graph.createVTable( - rawVertices, eTable, numVertexPartitions) + protected lazy val vTable: RDD[(Vid, (VD, Array[Pid]))] = { + if (_rawVTable == null) { + Graph.createVTable(_rawVertices, eTable, numVertexPartitions) + } else { + _rawVTable + } + } - def vertices(): RDD[Vertex[VD]] = vTable.map { case(vid, (data, pids)) => new Vertex(vid, data) } + def vertices: RDD[Vertex[VD]] = { + if (!_cached && _rawVertices != null) { + _rawVertices + } else { + vTable.map { case(vid, (data, pids)) => new Vertex(vid, data) } + } + } - def edges(): RDD[Edge[ED]] = eTable.mapPartitions { iter => iter.next._2.iterator } + def edges: RDD[Edge[ED]] = { + if (!_cached && _rawEdges != null) { + _rawEdges + } else { + eTable.mapPartitions { iter => iter.next._2.iterator } + } + } + + def edgesWithVertices: RDD[EdgeWithVertices[VD, ED]] = { + (new EdgeWithVerticesRDD[VD, ED](vTable, eTable)).mapPartitions { part => part.next._2 } + } + + def mapVertices[VD2: ClassManifest](f: (Vertex[VD]) => Vertex[VD2]) = { + ClosureCleaner.clean(f) + new Graph(vertices.map(f), edges) + } + + def mapEdges[ED2: ClassManifest](f: (Edge[ED]) => Edge[ED2]) = { + ClosureCleaner.clean(f) + new Graph(vertices, edges.map(f)) + } + + def updateVertices[U: ClassManifest]( + updates: RDD[(Vid, U)], + updateFunc: (Vertex[VD], Seq[U]) => VD) + : Graph[VD, ED] = { + + ClosureCleaner.clean(updateFunc) + + val joined: RDD[(Vid, ((VD, Array[Pid]), Option[Seq[U]]))] = + vTable.leftOuterJoin(updates.groupByKey(vertexPartitioner)) + + val newVTable = (joined.mapPartitions({ iter => + iter.map { case (vid, ((vdata, pids), updates)) => + val newVdata = if (updates.isDefined) updateFunc(Vertex(vid, vdata), updates.get) else vdata + (vid, (newVdata, pids)) + } + }, preservesPartitioning = true)).cache() - def edgesWithVertices(): RDD[EdgeWithVertices[VD, ED]] = { - (new EdgeWithVerticesRDD(vTable, eTable)).mapPartitions { case(vmap, iter) => iter } + new Graph(null, null, newVTable, eTable) } def mapPartitions[U: ClassManifest]( - f: (VertexHashMap, Iterator[EdgeWithVertices[VD, ED]]) => Iterator[U], + f: (VertexHashMap[VD], Iterator[EdgeWithVertices[VD, ED]]) => Iterator[U], preservesPartitioning: Boolean = false): RDD[U] = { (new EdgeWithVerticesRDD(vTable, eTable)).mapPartitions({ part => val (vmap, iter) = part.next() - iter.mapPartitions(f) + f(vmap, iter) }, preservesPartitioning) } @@ -80,7 +145,8 @@ object Graph { * A partition of edges in 3 large columnar arrays. */ private[graph] - class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED: Manifest] { + class EdgePartition [@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED:ClassManifest] + { val srcIds: IntArrayList = new IntArrayList val dstIds: IntArrayList = new IntArrayList // TODO: Specialize data. @@ -117,7 +183,7 @@ object Graph { } private[graph] - def createVTable[VD: Manifest, ED: Manifest]( + def createVTable[VD: ClassManifest, ED: ClassManifest]( vertices: RDD[Vertex[VD]], eTable: RDD[(Pid, EdgePartition[ED])], numPartitions: Int) = { @@ -145,7 +211,6 @@ object Graph { case (vdata, None) => (vdata, Array.empty[Pid]) case (vdata, Some(pids)) => (vdata, pids.toArray) } - .cache() } /** @@ -157,7 +222,7 @@ object Graph { * containing all the edges in a partition. */ private[graph] - def createETable[ED: Manifest](edges: RDD[Edge[ED]], numPartitions: Int) + def createETable[ED: ClassManifest](edges: RDD[Edge[ED]], numPartitions: Int) : RDD[(Pid, EdgePartition[ED])] = { edges.map { e => @@ -170,7 +235,6 @@ object Graph { iter.foreach { case (_, (src, dst, data)) => edgePartition.add(src, dst, data) } Iterator((pid, edgePartition)) }, preservesPartitioning = true) - .cache() } } diff --git a/graph/src/main/scala/spark/graph/GraphLab.scala b/graph/src/main/scala/spark/graph/GraphLab.scala index ebf001937b312..3d3ce99a4b321 100644 --- a/graph/src/main/scala/spark/graph/GraphLab.scala +++ b/graph/src/main/scala/spark/graph/GraphLab.scala @@ -1,45 +1,58 @@ package spark.graph +import scala.collection.JavaConversions._ + +import spark.RDD + class GraphLab { - def iterateGAS[A: Manifest, VD: Manifest, ED: Manifest]( + def iterateGAS[A: ClassManifest, VD: ClassManifest, ED: ClassManifest]( graph: Graph[VD, ED], gather: (Vid, EdgeWithVertices[VD, ED]) => A, merge: (A, A) => A, default: A, apply: (Vertex[VD], A) => VD, numIter: Int, - gatherEdges: EdgeDirection = EdgeDirection.In) = { + gatherEdges: EdgeDirection.EdgeDirection = EdgeDirection.In) = { - val g = new Graph[(VD, A), ED](graph.rawVertices.map(v => (v, default)), graph.rawEdges) + var g = graph.mapVertices(v => Vertex(v.id, VDataWithAcc(v.data, default))).cache() var i = 0 while (i < numIter) { - val gatherTable = g.mapPartitions { case(vmap, iter) => + val accUpdates: RDD[(Vid, A)] = g.mapPartitions({ case(vmap, iter) => val edgeSansAcc = new EdgeWithVertices[VD, ED]() - iter.map { edge: EdgeWithVertices[(VD, A), ED] => + iter.map { edge: EdgeWithVertices[VDataWithAcc[VD, A], ED] => edgeSansAcc.data = edge.data - edgeSansAcc.src.data = edge.src.data._1 - edgeSansAcc.dst.data = edge.dst.data._1 + edgeSansAcc.src.data = edge.src.data.vdata + edgeSansAcc.dst.data = edge.dst.data.vdata edgeSansAcc.src.id = edge.src.id edgeSansAcc.dst.id = edge.dst.id if (gatherEdges == EdgeDirection.In || gatherEdges == EdgeDirection.Both) { - edge.dst.data._2 = merge(edge.dst.data._2, gather(edgeSansAcc.dst.id, edgeSansAcc)) + edge.dst.data.acc = merge(edge.dst.data.acc, gather(edgeSansAcc.dst.id, edgeSansAcc)) } if (gatherEdges == EdgeDirection.Out || gatherEdges == EdgeDirection.Both) { - edge.src.data._2 = merge(edge.src.data._2, gather(edgeSansAcc.src.id, edgeSansAcc)) + edge.src.data.acc = merge(edge.src.data.acc, gather(edgeSansAcc.src.id, edgeSansAcc)) } } - vmap.int2ObjectEntrySet().fastIterator().map{ case (vid, (vdata, acc)) => (vid, acc) } - }.reduceByKey(graph.vertexPartitioner, false) + vmap.int2ObjectEntrySet().fastIterator().map{ entry => + (entry.getIntKey(), entry.getValue().acc) + } + })(classManifest[(Int, A)]) - gatherTable + def applyFunc(v: Vertex[VDataWithAcc[VD, A]], updates: Seq[A]): VDataWithAcc[VD, A] = { + VDataWithAcc(apply(Vertex(v.id, v.data.vdata), updates.reduce(merge)), default) + } + g = g.updateVertices(accUpdates, applyFunc).cache() i += 1 } } } + + +private[graph] +sealed case class VDataWithAcc[VD: ClassManifest, A](var vdata: VD, var acc: A) diff --git a/graph/src/main/scala/spark/graph/package.scala b/graph/src/main/scala/spark/graph/package.scala index e900a27b27a38..fa63e488fc713 100644 --- a/graph/src/main/scala/spark/graph/package.scala +++ b/graph/src/main/scala/spark/graph/package.scala @@ -5,7 +5,7 @@ package object graph { type Vid = Int type Pid = Int type Status = Boolean - type VertexHashMap = it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap + type VertexHashMap[T] = it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap[T] /** * Return the default null-like value for a data type T. From fe42ad41bb6c424a97e3ce8bc185b2c7a7b24e86 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 3 Apr 2013 23:40:09 +0800 Subject: [PATCH 007/531] Commit a working version. --- graph/src/main/scala/spark/graph/Graph.scala | 46 +++++++++++++++++-- .../src/main/scala/spark/graph/GraphLab.scala | 2 +- .../src/main/scala/spark/graph/package.scala | 2 +- 3 files changed, 45 insertions(+), 5 deletions(-) diff --git a/graph/src/main/scala/spark/graph/Graph.scala b/graph/src/main/scala/spark/graph/Graph.scala index 85345c28b081a..877c391b6e2cc 100644 --- a/graph/src/main/scala/spark/graph/Graph.scala +++ b/graph/src/main/scala/spark/graph/Graph.scala @@ -6,6 +6,7 @@ import scala.collection.mutable.ArrayBuffer import it.unimi.dsi.fastutil.ints.IntArrayList import spark.{ClosureCleaner, HashPartitioner, RDD} +import spark.SparkContext import spark.SparkContext._ import spark.graph.Graph.EdgePartition import spark.storage.StorageLevel @@ -13,7 +14,10 @@ import spark.storage.StorageLevel case class Vertex[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) VD] ( var id: Vid = 0, - var data: VD = nullValue[VD]) + var data: VD = nullValue[VD]) { + + def this(tuple: Tuple2[Vid, VD]) = this(tuple._1, tuple._2) +} case class Edge[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED] ( @@ -141,11 +145,47 @@ class Graph[VD: ClassManifest, ED: ClassManifest] protected ( object Graph { + /** + * Load an edge list from file initializing the Graph RDD + */ + def textFile[ED: ClassManifest](sc: SparkContext, + fname: String, edgeParser: Array[String] => ED) = { + + // Parse the edge data table + val edges = sc.textFile(fname).map { line => + val lineArray = line.split("\\s+") + if(lineArray.length < 2) { + println("Invalid line: " + line) + assert(false) + } + val source = lineArray(0) + val target = lineArray(1) + val tail = lineArray.drop(2) + val edata = edgeParser(tail) + Edge(source.trim.toInt, target.trim.toInt, edata) + }.cache() + + // Parse the vertex data table + val vertices = edges.flatMap { edge => List((edge.src, 1), (edge.dst, 1)) } + .reduceByKey(_ + _) + .map(new Vertex(_)) + + val graph = new Graph[Int, ED](vertices, edges) + graph.cache() + + println("Loaded graph:" + + "\n\t#edges: " + graph.edges.count + + "\n\t#vertices: " + graph.vertices.count) + + graph + } + + /** * A partition of edges in 3 large columnar arrays. */ private[graph] - class EdgePartition [@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED:ClassManifest] + class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED:ClassManifest] { val srcIds: IntArrayList = new IntArrayList val dstIds: IntArrayList = new IntArrayList @@ -170,7 +210,7 @@ object Graph { private var edge = new Edge[ED] private var pos = 0 - override def hasNext: Boolean = pos < size + override def hasNext: Boolean = pos < EdgePartition.this.size override def next(): Edge[ED] = { edge.src = srcIds.get(pos) diff --git a/graph/src/main/scala/spark/graph/GraphLab.scala b/graph/src/main/scala/spark/graph/GraphLab.scala index 3d3ce99a4b321..4892826bf448b 100644 --- a/graph/src/main/scala/spark/graph/GraphLab.scala +++ b/graph/src/main/scala/spark/graph/GraphLab.scala @@ -5,7 +5,7 @@ import scala.collection.JavaConversions._ import spark.RDD -class GraphLab { +object GraphLab { def iterateGAS[A: ClassManifest, VD: ClassManifest, ED: ClassManifest]( graph: Graph[VD, ED], diff --git a/graph/src/main/scala/spark/graph/package.scala b/graph/src/main/scala/spark/graph/package.scala index fa63e488fc713..6db050e6e16cb 100644 --- a/graph/src/main/scala/spark/graph/package.scala +++ b/graph/src/main/scala/spark/graph/package.scala @@ -4,7 +4,7 @@ package object graph { type Vid = Int type Pid = Int - type Status = Boolean + type VertexHashMap[T] = it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap[T] /** From 0123c9d6a123f3e2c3e416ce402e7cf4c9900042 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Wed, 3 Apr 2013 08:43:08 -0700 Subject: [PATCH 008/531] Changed GraphLab class to object and added graph loading from text file. --- graph/src/main/scala/spark/graph/Graph.scala | 36 +++++++++++++++++++ .../src/main/scala/spark/graph/GraphLab.scala | 2 +- 2 files changed, 37 insertions(+), 1 deletion(-) diff --git a/graph/src/main/scala/spark/graph/Graph.scala b/graph/src/main/scala/spark/graph/Graph.scala index 4f1b3b44d525a..6e662ab1fbbb9 100644 --- a/graph/src/main/scala/spark/graph/Graph.scala +++ b/graph/src/main/scala/spark/graph/Graph.scala @@ -172,5 +172,41 @@ object Graph { }, preservesPartitioning = true) .cache() } + + /** + * Load a graph from a text file. + */ + def textFile[ED: Manifest](sc: SparkContext, + fname: String, edgeParser: Array[String] => ED) = { + + // Parse the edge data table + val edges = sc.textFile(fname).map( + line => { + val lineArray = line.split("\\s+") + if(lineArray.length < 2) { + println("Invalid line: " + line) + assert(false) + } + val source = lineArray(0) + val target = lineArray(1) + val tail = lineArray.drop(2) + val edata = edgeParser(tail) + Edge(source.trim.toInt, target.trim.toInt, edata) + }).cache + + // Parse the vertex data table + val vertices = edges.flatMap { + case (source, target, _) => List((source, 1), (target, 1)) + }.reduceByKey(_ + _).map(pair => Vertex(piar._1, pair._2)) + + val graph = new Graph[Int, ED](vertices, edges) + + println("Loaded graph:" + + "\n\t#edges: " + graph.numEdges + + "\n\t#vertices: " + graph.numVertices) + graph + } + + } diff --git a/graph/src/main/scala/spark/graph/GraphLab.scala b/graph/src/main/scala/spark/graph/GraphLab.scala index ebf001937b312..d90fdc3369bf5 100644 --- a/graph/src/main/scala/spark/graph/GraphLab.scala +++ b/graph/src/main/scala/spark/graph/GraphLab.scala @@ -1,7 +1,7 @@ package spark.graph -class GraphLab { +object GraphLab { def iterateGAS[A: Manifest, VD: Manifest, ED: Manifest]( graph: Graph[VD, ED], From 4291da1481e86f6d1c22f06e118002b44dbfed07 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 4 Apr 2013 00:07:54 +0800 Subject: [PATCH 009/531] Allow returning different vertex data type in updateVertices. Please enter the commit message for your changes. Lines starting --- graph/src/main/scala/spark/graph/Graph.scala | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/graph/src/main/scala/spark/graph/Graph.scala b/graph/src/main/scala/spark/graph/Graph.scala index 877c391b6e2cc..77f16979a45d4 100644 --- a/graph/src/main/scala/spark/graph/Graph.scala +++ b/graph/src/main/scala/spark/graph/Graph.scala @@ -111,22 +111,23 @@ class Graph[VD: ClassManifest, ED: ClassManifest] protected ( new Graph(vertices, edges.map(f)) } - def updateVertices[U: ClassManifest]( + def updateVertices[U: ClassManifest, VD2: ClassManifest]( updates: RDD[(Vid, U)], - updateFunc: (Vertex[VD], Seq[U]) => VD) - : Graph[VD, ED] = { + updateFunc: (Vertex[VD], Seq[U]) => VD2) + : Graph[VD2, ED] = { ClosureCleaner.clean(updateFunc) val joined: RDD[(Vid, ((VD, Array[Pid]), Option[Seq[U]]))] = vTable.leftOuterJoin(updates.groupByKey(vertexPartitioner)) - val newVTable = (joined.mapPartitions({ iter => + val newVTable = joined.mapPartitions({ iter => iter.map { case (vid, ((vdata, pids), updates)) => - val newVdata = if (updates.isDefined) updateFunc(Vertex(vid, vdata), updates.get) else vdata + val u = if (updates.isDefined) updates.get else Seq.empty + val newVdata = updateFunc(Vertex(vid, vdata), u) (vid, (newVdata, pids)) } - }, preservesPartitioning = true)).cache() + }, preservesPartitioning = true).cache() new Graph(null, null, newVTable, eTable) } From 39cac0ae65f78e92fc8da68c0aa72c80410cbfd0 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Wed, 3 Apr 2013 16:22:37 -0700 Subject: [PATCH 010/531] Fixed iterateGAS to return a graph, added some minor features to graph and some additional todo items, and added the Analytics code from the internal SparkGraph prototype --- .../main/scala/spark/graph/Analytics.scala | 456 ++++++++++++++++++ graph/src/main/scala/spark/graph/Graph.scala | 15 +- .../src/main/scala/spark/graph/GraphLab.scala | 3 + 3 files changed, 471 insertions(+), 3 deletions(-) create mode 100644 graph/src/main/scala/spark/graph/Analytics.scala diff --git a/graph/src/main/scala/spark/graph/Analytics.scala b/graph/src/main/scala/spark/graph/Analytics.scala new file mode 100644 index 0000000000000..48f20875594a7 --- /dev/null +++ b/graph/src/main/scala/spark/graph/Analytics.scala @@ -0,0 +1,456 @@ +package spark.graph + +import spark._ +import spark.SparkContext._ +// import com.esotericsoftware.kryo._ + +// import breeze.linalg._ + + + +// class AnalyticsKryoRegistrator extends KryoRegistrator { +// def registerClasses(kryo: Kryo) { +// println("registering kryo") +// kryo.register(classOf[(Int,Float,Float)]) +// Graph.kryoRegister[(Int,Float,Float), Float](kryo) +// Graph.kryoRegister[(Int,Float), Float](kryo) +// Graph.kryoRegister[Int, Float](kryo) +// Graph.kryoRegister[Float, Float](kryo) +// kryo.setReferences(false); +// } +// } + + +object Analytics { + + /** + * Compute the PageRank of a graph returning the pagerank of each vertex as an RDD + */ + def pagerank[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], maxIter: Int) = { + // Compute the out degree of each vertex + val outDegree = graph.edges.map { + case Edge(src, target, data) => (src, 1) + }.reduceByKey(_ + _) + val pagerankGraph = graph.updateVertices[Int, (Int, Float)](outDegree, + (vertex, degIter) => (degIter.sum, 1.0F) + ) + GraphLab.iterateGAS[Float, (Int, Float), ED](pagerankGraph, + (me_id, edge) => edge.src.data._2 / edge.dst.data._1, // gather + (a: Float, b: Float) => a + b, // merge + 0F, + (vertex, a: Float) => (vertex.data._1, (0.15F + 0.85F * a)), // apply + maxIter).vertices.map { case Vertex(id, (degree, rank)) => (id, rank) } + // println("Computed graph: #edges: " + graph_ret.numEdges + " #vertices" + graph_ret.numVertices) + // graph_ret.vertices.take(10).foreach(println) + } + +/* + + /** + * Compute the connected component membership of each vertex + * and return an RDD with the vertex value containing the + * lowest vertex id in the connected component containing + * that vertex. + */ + def connectedComponents[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], numIter: Int) = { + + val vertices = graph.vertices.mapPartitions(iter => iter.map { case (vid, _) => (vid, vid) }) + val edges = graph.edges // .mapValues(v => None) + val ccGraph = new Graph(vertices, edges) + + + ccGraph.iterateStatic( + (me_id, edge) => edge.otherVertex(me_id).data, // gather + (a: Int, b: Int) => math.min(a, b), // merge + Integer.MAX_VALUE, + (v, a: Int) => math.min(v.data, a), // apply + numIter, + gatherEdges = EdgeDirection.Both).vertices + // + // graph_ret.vertices.`.foreach(println) + // graph_ret.edges.take(10).foreach(println) + } + + + /** + * Compute the shortest path to a set of markers + */ + def shortestPath[VD: Manifest, ED: Manifest](graph: Graph[VD, Float], + sources: List[Int], numIter: Int) = { + val sourceSet = sources.toSet + val vertices = graph.vertices.mapPartitions( + iter => iter.map { + case (vid, _) => (vid, (if(sourceSet.contains(vid)) 0.0F else Float.MaxValue) ) + }); + + val edges = graph.edges // .mapValues(v => None) + val spGraph = new Graph(vertices, edges) + + val niterations = Int.MaxValue + spGraph.iterateStatic( + (me_id, edge) => edge.otherVertex(me_id).data + edge.data, // gather + (a: Float, b: Float) => math.min(a, b), // merge + Float.MaxValue, + (v, a: Float) => math.min(v.data, a), // apply + numIter, + gatherEdges = EdgeDirection.In).vertices + } + +*/ + + + // /** + // * Compute the PageRank of a graph returning the pagerank of each vertex as an RDD + // */ + // def dynamicPageRank[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], + // tol: Float, maxIter: Int = 10) = { + // graph.edges.cache + // // Compute the out degree of each vertex + // val outDegree = graph.edges.map { case (src, target, data) => (src, 1)}.reduceByKey(_ + _) + // val vertices = graph.vertices.leftOuterJoin(outDegree).mapValues { + // case (_, Some(deg)) => (deg, 1.0F, 1.0F) + // case (_, None) => (0, 1.0F, 1.0F) + // }.cache + + // val edges = graph.edges + // val pageRankGraph = new Graph(vertices, edges) + // pageRankGraph.numVPart = graph.numVPart + // pageRankGraph.numEPart = graph.numEPart + + // // Run PageRank + // pageRankGraph.iterateDynamic( + // (me_id, edge) => edge.source.data._2 / edge.source.data._1, // gather + // (a: Float, b: Float) => a + b, // merge + // 0F, + // (vertex, a: Float) => (vertex.data._1, (0.15F + 0.85F * a), vertex.data._2), // apply + // (me_id, edge) => math.abs(edge.source.data._2 - edge.source.data._1) > tol, // scatter + // maxIter).vertices.mapValues { case (degree, rank, oldRank) => rank } + // // println("Computed graph: #edges: " + graph_ret.numEdges + " #vertices" + graph_ret.numVertices) + // // graph_ret.vertices.take(10).foreach(println) + // } + + + + + + // /** + // * Compute the connected component membership of each vertex + // * and return an RDD with the vertex value containing the + // * lowest vertex id in the connected component containing + // * that vertex. + // */ + // def dynamicConnectedComponents[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], + // numIter: Int = Int.MaxValue) = { + + // val vertices = graph.vertices.mapPartitions(iter => iter.map { case (vid, _) => (vid, vid) }) + // val edges = graph.edges // .mapValues(v => None) + // val ccGraph = new Graph(vertices, edges) + + // ccGraph.iterateDynamic( + // (me_id, edge) => edge.otherVertex(me_id).data, // gather + // (a: Int, b: Int) => math.min(a, b), // merge + // Integer.MAX_VALUE, + // (v, a: Int) => math.min(v.data, a), // apply + // (me_id, edge) => edge.otherVertex(me_id).data > edge.vertex(me_id).data, // scatter + // numIter, + // gatherEdges = EdgeDirection.Both, + // scatterEdges = EdgeDirection.Both).vertices + // // + // // graph_ret.vertices.collect.foreach(println) + // // graph_ret.edges.take(10).foreach(println) + // } + + + + + // /** + // * Compute the shortest path to a set of markers + // */ + // def dynamicShortestPath[VD: Manifest, ED: Manifest](graph: Graph[VD, Float], + // sources: List[Int], numIter: Int) = { + // val sourceSet = sources.toSet + // val vertices = graph.vertices.mapPartitions( + // iter => iter.map { + // case (vid, _) => (vid, (if(sourceSet.contains(vid)) 0.0F else Float.MaxValue) ) + // }); + + // val edges = graph.edges // .mapValues(v => None) + // val spGraph = new Graph(vertices, edges) + + // val niterations = Int.MaxValue + // spGraph.iterateDynamic( + // (me_id, edge) => edge.otherVertex(me_id).data + edge.data, // gather + // (a: Float, b: Float) => math.min(a, b), // merge + // Float.MaxValue, + // (v, a: Float) => math.min(v.data, a), // apply + // (me_id, edge) => edge.vertex(me_id).data + edge.data < edge.otherVertex(me_id).data, // scatter + // numIter, + // gatherEdges = EdgeDirection.In, + // scatterEdges = EdgeDirection.Out).vertices + // } + + + + + // /** + // * + // */ + // def alternatingLeastSquares[VD: ClassManifest, ED: ClassManifest](graph: Graph[VD, Double], + // latentK: Int, lambda: Double, numIter: Int) = { + // val vertices = graph.vertices.mapPartitions( _.map { + // case (vid, _) => (vid, Array.fill(latentK){ scala.util.Random.nextDouble() } ) + // }).cache + // val maxUser = graph.edges.map(_._1).reduce(math.max(_,_)) + // val edges = graph.edges // .mapValues(v => None) + // val alsGraph = new Graph(vertices, edges) + // alsGraph.numVPart = graph.numVPart + // alsGraph.numEPart = graph.numEPart + + // val niterations = Int.MaxValue + // alsGraph.iterateDynamic[(Array[Double], Array[Double])]( + // (me_id, edge) => { // gather + // val X = edge.otherVertex(me_id).data + // val y = edge.data + // val Xy = X.map(_ * y) + // val XtX = (for(i <- 0 until latentK; j <- i until latentK) yield(X(i) * X(j))).toArray + // (Xy, XtX) + // }, + // (a, b) => { + // // The difference between the while loop and the zip is a FACTOR OF TWO in overall + // // runtime + // var i = 0 + // while(i < a._1.length) { a._1(i) += b._1(i); i += 1 } + // i = 0 + // while(i < a._2.length) { a._2(i) += b._2(i); i += 1 } + // a + // // (a._1.zip(b._1).map{ case (q,r) => q+r }, a._2.zip(b._2).map{ case (q,r) => q+r }) + // }, + // (Array.empty[Double], Array.empty[Double]), // default value is empty + // (vertex, accum) => { // apply + // val XyArray = accum._1 + // val XtXArray = accum._2 + // if(XyArray.isEmpty) vertex.data // no neighbors + // else { + // val XtX = DenseMatrix.tabulate(latentK,latentK){ (i,j) => + // (if(i < j) XtXArray(i + (j+1)*j/2) else XtXArray(i + (j+1)*j/2)) + + // (if(i == j) lambda else 1.0F) //regularization + // } + // val Xy = DenseMatrix.create(latentK,1,XyArray) + // val w = XtX \ Xy + // w.data + // } + // }, + // (me_id, edge) => true, + // numIter, + // gatherEdges = EdgeDirection.Both, + // scatterEdges = EdgeDirection.Both, + // vertex => vertex.id < maxUser).vertices + // } + + + + def main(args: Array[String]) = { + val host = args(0) + val taskType = args(1) + val fname = args(2) + val options = args.drop(3).map { arg => + arg.dropWhile(_ == '-').split('=') match { + case Array(opt, v) => (opt -> v) + case _ => throw new IllegalArgumentException("Invalid argument: " + arg) + } + } + + // System.setProperty("spark.serializer", "spark.KryoSerializer") + // //System.setProperty("spark.shuffle.compress", "false") + // System.setProperty("spark.kryo.registrator", "spark.graphlab.AnalyticsKryoRegistrator") + + + +/* + taskType match { + case "pagerank" => { + + var numIter = Int.MaxValue + var isDynamic = false + var tol:Float = 0.001F + var outFname = "" + var numVPart = 4 + var numEPart = 4 + + options.foreach{ + case ("numIter", v) => numIter = v.toInt + case ("dynamic", v) => isDynamic = v.toBoolean + case ("tol", v) => tol = v.toFloat + case ("output", v) => outFname = v + case ("numVPart", v) => numVPart = v.toInt + case ("numEPart", v) => numEPart = v.toInt + case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) + } + + if(!isDynamic && numIter == Int.MaxValue) { + println("Set number of iterations!") + exit(1) + } + println("======================================") + println("| PageRank |") + println("--------------------------------------") + println(" Using parameters:") + println(" \tDynamic: " + isDynamic) + if(isDynamic) println(" \t |-> Tolerance: " + tol) + println(" \tNumIter: " + numIter) + println("======================================") + + val sc = new SparkContext(host, "PageRank(" + fname + ")") + val graph = Graph.textFile(sc, fname, a => 1.0F) + graph.numVertexPartitions = numVPart + graph.numEdgePartitions = numEPart + val startTime = System.currentTimeMillis + + Analytics.pageRank(graph, numIter) + // val pr = if(isDynamic) Analytics.dynamicPageRank(graph, tol, numIter) + // else Analytics.pageRank(graph, numIter) + println("Total rank: " + pr.map(_._2).reduce(_+_)) + if(!outFname.isEmpty) { + println("Saving pageranks of pages to " + outFname) + pr.map(p => p._1 + "\t" + p._2).saveAsTextFile(outFname) + } + println("Runtime: " + ((System.currentTimeMillis - startTime)/1000.0) + " seconds") + sc.stop() + } + + case "cc" => { + + var numIter = Int.MaxValue + var isDynamic = false + + options.foreach{ + case ("numIter", v) => numIter = v.toInt + case ("dynamic", v) => isDynamic = v.toBoolean + case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) + } + + if(!isDynamic && numIter == Int.MaxValue) { + println("Set number of iterations!") + exit(1) + } + println("======================================") + println("| Connected Components |") + println("--------------------------------------") + println(" Using parameters:") + println(" \tDynamic: " + isDynamic) + println(" \tNumIter: " + numIter) + println("======================================") + + val sc = new SparkContext(host, "ConnectedComponents(" + fname + ")") + val graph = Graph.textFile(sc, fname, a => 1.0F) + Analytics.connectedComponents(graph, numIter) + // val cc = if(isDynamic) Analytics.dynamicConnectedComponents(graph, numIter) + // else Analytics.connectedComponents(graph, numIter) + println("Components: " + cc.map(_._2).distinct()) + + sc.stop() + } + + // case "shortestpath" => { + + // var numIter = Int.MaxValue + // var isDynamic = true + // var sources: List[Int] = List.empty + + // options.foreach{ + // case ("numIter", v) => numIter = v.toInt + // case ("dynamic", v) => isDynamic = v.toBoolean + // case ("source", v) => sources ++= List(v.toInt) + // case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) + // } + + + // if(!isDynamic && numIter == Int.MaxValue) { + // println("Set number of iterations!") + // exit(1) + // } + + // if(sources.isEmpty) { + // println("No sources provided!") + // exit(1) + // } + + // println("======================================") + // println("| Shortest Path |") + // println("--------------------------------------") + // println(" Using parameters:") + // println(" \tDynamic: " + isDynamic) + // println(" \tNumIter: " + numIter) + // println(" \tSources: [" + sources.mkString(", ") + "]") + // println("======================================") + + // val sc = new SparkContext(host, "ShortestPath(" + fname + ")") + // val graph = Graph.textFile(sc, fname, a => (if(a.isEmpty) 1.0F else a(0).toFloat ) ) + // val cc = if(isDynamic) Analytics.dynamicShortestPath(graph, sources, numIter) + // else Analytics.shortestPath(graph, sources, numIter) + // println("Longest Path: " + cc.map(_._2).reduce(math.max(_,_))) + + // sc.stop() + // } + + + // case "als" => { + + // var numIter = 5 + // var lambda = 0.01 + // var latentK = 10 + // var usersFname = "usersFactors.tsv" + // var moviesFname = "moviesFname.tsv" + // var numVPart = 4 + // var numEPart = 4 + + // options.foreach{ + // case ("numIter", v) => numIter = v.toInt + // case ("lambda", v) => lambda = v.toDouble + // case ("latentK", v) => latentK = v.toInt + // case ("usersFname", v) => usersFname = v + // case ("moviesFname", v) => moviesFname = v + // case ("numVPart", v) => numVPart = v.toInt + // case ("numEPart", v) => numEPart = v.toInt + // case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) + // } + + // println("======================================") + // println("| Alternating Least Squares |") + // println("--------------------------------------") + // println(" Using parameters:") + // println(" \tNumIter: " + numIter) + // println(" \tLambda: " + lambda) + // println(" \tLatentK: " + latentK) + // println(" \tusersFname: " + usersFname) + // println(" \tmoviesFname: " + moviesFname) + // println("======================================") + + // val sc = new SparkContext(host, "ALS(" + fname + ")") + // val graph = Graph.textFile(sc, fname, a => a(0).toDouble ) + // graph.numVPart = numVPart + // graph.numEPart = numEPart + + // val maxUser = graph.edges.map(_._1).reduce(math.max(_,_)) + // val minMovie = graph.edges.map(_._2).reduce(math.min(_,_)) + // assert(maxUser < minMovie) + + // val factors = Analytics.alternatingLeastSquares(graph, latentK, lambda, numIter).cache + // factors.filter(_._1 <= maxUser).map(r => r._1 + "\t" + r._2.mkString("\t")) + // .saveAsTextFile(usersFname) + // factors.filter(_._1 >= minMovie).map(r => r._1 + "\t" + r._2.mkString("\t")) + // .saveAsTextFile(moviesFname) + + // sc.stop() + // } + + + case _ => { + println("Invalid task type.") + } + } + */ + } + + +} diff --git a/graph/src/main/scala/spark/graph/Graph.scala b/graph/src/main/scala/spark/graph/Graph.scala index bb321156ca87b..caf31e04aed50 100644 --- a/graph/src/main/scala/spark/graph/Graph.scala +++ b/graph/src/main/scala/spark/graph/Graph.scala @@ -58,13 +58,22 @@ class Graph[VD: ClassManifest, ED: ClassManifest] protected ( this } + /// Todo: Should theses be set on construction and passed onto derived graphs? var numEdgePartitions = 5 var numVertexPartitions = 5 + /// Todo: Should these be passed onto derived graphs? protected val vertexPartitioner = new HashPartitioner(numVertexPartitions) - protected val edgePartitioner = new HashPartitioner(numEdgePartitions) + + + lazy val numEdges = edges.count() + lazy val numVertices = vertices.count() + lazy val inDegrees = edges.map{ case Edge(src, target, _) => (target, 1) }.reduceByKey(_+_) + lazy val outDegrees = edges.map{ case Edge(src, target, _) => (src, 1) }.reduceByKey(_+_) + + protected lazy val eTable: RDD[(Pid, EdgePartition[ED])] = { if (_rawETable == null) { Graph.createETable(_rawEdges, numEdgePartitions) @@ -301,8 +310,8 @@ object Graph { // Parse the vertex data table val vertices = edges.flatMap { - case (source, target, _) => List((source, 1), (target, 1)) - }.reduceByKey(_ + _).map(pair => Vertex(piar._1, pair._2)) + case Edge(source, target, _) => List((source, 1), (target, 1)) + }.reduceByKey(_ + _).map(pair => Vertex(pair._1, pair._2)) val graph = new Graph[Int, ED](vertices, edges) diff --git a/graph/src/main/scala/spark/graph/GraphLab.scala b/graph/src/main/scala/spark/graph/GraphLab.scala index ce5b3f6e19b94..b3127c64d36c6 100644 --- a/graph/src/main/scala/spark/graph/GraphLab.scala +++ b/graph/src/main/scala/spark/graph/GraphLab.scala @@ -48,6 +48,9 @@ object GraphLab { i += 1 } + // return the transformed graph with the accumulator removed + g.mapVertices(v => Vertex(v.id, v.data.vdata)) + } } From 91e1227edb42d35a6a811aa7768c301a1c022f48 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Wed, 3 Apr 2013 23:04:29 -0700 Subject: [PATCH 011/531] completed port of several analytics packages as well as analytics main --- .../main/scala/spark/graph/Analytics.scala | 134 ++++++++---------- graph/src/main/scala/spark/graph/Graph.scala | 35 +---- .../src/main/scala/spark/graph/GraphLab.scala | 6 +- 3 files changed, 61 insertions(+), 114 deletions(-) diff --git a/graph/src/main/scala/spark/graph/Analytics.scala b/graph/src/main/scala/spark/graph/Analytics.scala index 48f20875594a7..b5ffaf1b344fa 100644 --- a/graph/src/main/scala/spark/graph/Analytics.scala +++ b/graph/src/main/scala/spark/graph/Analytics.scala @@ -26,25 +26,19 @@ object Analytics { /** * Compute the PageRank of a graph returning the pagerank of each vertex as an RDD */ - def pagerank[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], maxIter: Int) = { + def pagerank[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], numIter: Int) = { // Compute the out degree of each vertex - val outDegree = graph.edges.map { - case Edge(src, target, data) => (src, 1) - }.reduceByKey(_ + _) - val pagerankGraph = graph.updateVertices[Int, (Int, Float)](outDegree, + val pagerankGraph = graph.updateVertices[Int, (Int, Float)](graph.outDegrees, (vertex, degIter) => (degIter.sum, 1.0F) ) - GraphLab.iterateGAS[Float, (Int, Float), ED](pagerankGraph, + GraphLab.iterateGAS[(Int, Float), ED, Float](pagerankGraph)( (me_id, edge) => edge.src.data._2 / edge.dst.data._1, // gather (a: Float, b: Float) => a + b, // merge 0F, (vertex, a: Float) => (vertex.data._1, (0.15F + 0.85F * a)), // apply - maxIter).vertices.map { case Vertex(id, (degree, rank)) => (id, rank) } - // println("Computed graph: #edges: " + graph_ret.numEdges + " #vertices" + graph_ret.numVertices) - // graph_ret.vertices.take(10).foreach(println) + numIter).vertices.map{ case Vertex(id, (outDeg, r)) => Vertex(id, r) } } -/* /** * Compute the connected component membership of each vertex @@ -53,41 +47,26 @@ object Analytics { * that vertex. */ def connectedComponents[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], numIter: Int) = { - - val vertices = graph.vertices.mapPartitions(iter => iter.map { case (vid, _) => (vid, vid) }) - val edges = graph.edges // .mapValues(v => None) - val ccGraph = new Graph(vertices, edges) - - - ccGraph.iterateStatic( + val ccGraph = graph.mapVertices { case Vertex(vid, _) => Vertex(vid, vid) } + GraphLab.iterateGAS[Int, ED, Int](ccGraph)( (me_id, edge) => edge.otherVertex(me_id).data, // gather (a: Int, b: Int) => math.min(a, b), // merge Integer.MAX_VALUE, (v, a: Int) => math.min(v.data, a), // apply numIter, gatherEdges = EdgeDirection.Both).vertices - // - // graph_ret.vertices.`.foreach(println) - // graph_ret.edges.take(10).foreach(println) } /** * Compute the shortest path to a set of markers */ - def shortestPath[VD: Manifest, ED: Manifest](graph: Graph[VD, Float], - sources: List[Int], numIter: Int) = { + def shortestPath[VD: Manifest](graph: Graph[VD, Float], sources: List[Int], numIter: Int) = { val sourceSet = sources.toSet - val vertices = graph.vertices.mapPartitions( - iter => iter.map { - case (vid, _) => (vid, (if(sourceSet.contains(vid)) 0.0F else Float.MaxValue) ) - }); - - val edges = graph.edges // .mapValues(v => None) - val spGraph = new Graph(vertices, edges) - - val niterations = Int.MaxValue - spGraph.iterateStatic( + val spGraph = graph.mapVertices { + case Vertex(vid, _) => Vertex(vid, (if(sourceSet.contains(vid)) 0.0F else Float.MaxValue)) + } + GraphLab.iterateGAS[Float, Float, Float](spGraph)( (me_id, edge) => edge.otherVertex(me_id).data + edge.data, // gather (a: Float, b: Float) => math.min(a, b), // merge Float.MaxValue, @@ -96,7 +75,7 @@ object Analytics { gatherEdges = EdgeDirection.In).vertices } -*/ + // /** @@ -266,7 +245,7 @@ object Analytics { -/* + taskType match { case "pagerank" => { @@ -289,7 +268,7 @@ object Analytics { if(!isDynamic && numIter == Int.MaxValue) { println("Set number of iterations!") - exit(1) + sys.exit(1) } println("======================================") println("| PageRank |") @@ -306,13 +285,13 @@ object Analytics { graph.numEdgePartitions = numEPart val startTime = System.currentTimeMillis - Analytics.pageRank(graph, numIter) - // val pr = if(isDynamic) Analytics.dynamicPageRank(graph, tol, numIter) - // else Analytics.pageRank(graph, numIter) - println("Total rank: " + pr.map(_._2).reduce(_+_)) + val pr = Analytics.pagerank(graph, numIter) + // val pr = if(isDynamic) Analytics.dynamicPagerank(graph, tol, numIter) + // else Analytics.pagerank(graph, numIter) + println("Total rank: " + pr.map{ case Vertex(id,r) => r }.reduce(_+_) ) if(!outFname.isEmpty) { println("Saving pageranks of pages to " + outFname) - pr.map(p => p._1 + "\t" + p._2).saveAsTextFile(outFname) + pr.map{case Vertex(id, r) => id + "\t" + r}.saveAsTextFile(outFname) } println("Runtime: " + ((System.currentTimeMillis - startTime)/1000.0) + " seconds") sc.stop() @@ -331,7 +310,7 @@ object Analytics { if(!isDynamic && numIter == Int.MaxValue) { println("Set number of iterations!") - exit(1) + sys.exit(1) } println("======================================") println("| Connected Components |") @@ -343,55 +322,56 @@ object Analytics { val sc = new SparkContext(host, "ConnectedComponents(" + fname + ")") val graph = Graph.textFile(sc, fname, a => 1.0F) - Analytics.connectedComponents(graph, numIter) + val cc = Analytics.connectedComponents(graph, numIter) // val cc = if(isDynamic) Analytics.dynamicConnectedComponents(graph, numIter) // else Analytics.connectedComponents(graph, numIter) - println("Components: " + cc.map(_._2).distinct()) + println("Components: " + cc.map(_.data).distinct()) sc.stop() } - // case "shortestpath" => { + case "shortestpath" => { - // var numIter = Int.MaxValue - // var isDynamic = true - // var sources: List[Int] = List.empty + var numIter = Int.MaxValue + var isDynamic = true + var sources: List[Int] = List.empty - // options.foreach{ - // case ("numIter", v) => numIter = v.toInt - // case ("dynamic", v) => isDynamic = v.toBoolean - // case ("source", v) => sources ++= List(v.toInt) - // case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) - // } + options.foreach{ + case ("numIter", v) => numIter = v.toInt + case ("dynamic", v) => isDynamic = v.toBoolean + case ("source", v) => sources ++= List(v.toInt) + case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) + } - // if(!isDynamic && numIter == Int.MaxValue) { - // println("Set number of iterations!") - // exit(1) - // } + if(!isDynamic && numIter == Int.MaxValue) { + println("Set number of iterations!") + sys.exit(1) + } - // if(sources.isEmpty) { - // println("No sources provided!") - // exit(1) - // } + if(sources.isEmpty) { + println("No sources provided!") + sys.exit(1) + } - // println("======================================") - // println("| Shortest Path |") - // println("--------------------------------------") - // println(" Using parameters:") - // println(" \tDynamic: " + isDynamic) - // println(" \tNumIter: " + numIter) - // println(" \tSources: [" + sources.mkString(", ") + "]") - // println("======================================") + println("======================================") + println("| Shortest Path |") + println("--------------------------------------") + println(" Using parameters:") + println(" \tDynamic: " + isDynamic) + println(" \tNumIter: " + numIter) + println(" \tSources: [" + sources.mkString(", ") + "]") + println("======================================") - // val sc = new SparkContext(host, "ShortestPath(" + fname + ")") - // val graph = Graph.textFile(sc, fname, a => (if(a.isEmpty) 1.0F else a(0).toFloat ) ) - // val cc = if(isDynamic) Analytics.dynamicShortestPath(graph, sources, numIter) - // else Analytics.shortestPath(graph, sources, numIter) - // println("Longest Path: " + cc.map(_._2).reduce(math.max(_,_))) + val sc = new SparkContext(host, "ShortestPath(" + fname + ")") + val graph = Graph.textFile(sc, fname, a => (if(a.isEmpty) 1.0F else a(0).toFloat ) ) + val sp = Analytics.shortestPath(graph, sources, numIter) + // val cc = if(isDynamic) Analytics.dynamicShortestPath(graph, sources, numIter) + // else Analytics.shortestPath(graph, sources, numIter) + println("Longest Path: " + sp.map(_.data).reduce(math.max(_,_))) - // sc.stop() - // } + sc.stop() + } // case "als" => { @@ -449,7 +429,7 @@ object Analytics { println("Invalid task type.") } } - */ + } diff --git a/graph/src/main/scala/spark/graph/Graph.scala b/graph/src/main/scala/spark/graph/Graph.scala index caf31e04aed50..f8e16f24a3de6 100644 --- a/graph/src/main/scala/spark/graph/Graph.scala +++ b/graph/src/main/scala/spark/graph/Graph.scala @@ -159,7 +159,7 @@ object Graph { * Load an edge list from file initializing the Graph RDD */ def textFile[ED: ClassManifest](sc: SparkContext, - fname: String, edgeParser: Array[String] => ED) = { + fname: String, edgeParser: Array[String] => ED ) = { // Parse the edge data table val edges = sc.textFile(fname).map { line => @@ -287,39 +287,6 @@ object Graph { }, preservesPartitioning = true) } - /** - * Load a graph from a text file. - */ - def textFile[ED: Manifest](sc: SparkContext, - fname: String, edgeParser: Array[String] => ED) = { - - // Parse the edge data table - val edges = sc.textFile(fname).map( - line => { - val lineArray = line.split("\\s+") - if(lineArray.length < 2) { - println("Invalid line: " + line) - assert(false) - } - val source = lineArray(0) - val target = lineArray(1) - val tail = lineArray.drop(2) - val edata = edgeParser(tail) - Edge(source.trim.toInt, target.trim.toInt, edata) - }).cache - - // Parse the vertex data table - val vertices = edges.flatMap { - case Edge(source, target, _) => List((source, 1), (target, 1)) - }.reduceByKey(_ + _).map(pair => Vertex(pair._1, pair._2)) - - val graph = new Graph[Int, ED](vertices, edges) - - println("Loaded graph:" + - "\n\t#edges: " + graph.numEdges + - "\n\t#vertices: " + graph.numVertices) - graph - } } diff --git a/graph/src/main/scala/spark/graph/GraphLab.scala b/graph/src/main/scala/spark/graph/GraphLab.scala index b3127c64d36c6..4bc7c4d2e5948 100644 --- a/graph/src/main/scala/spark/graph/GraphLab.scala +++ b/graph/src/main/scala/spark/graph/GraphLab.scala @@ -6,14 +6,14 @@ import spark.RDD object GraphLab { - def iterateGAS[A: ClassManifest, VD: ClassManifest, ED: ClassManifest]( - graph: Graph[VD, ED], + def iterateGAS[VD: ClassManifest, ED: ClassManifest, A: ClassManifest](graph: Graph[VD, ED])( gather: (Vid, EdgeWithVertices[VD, ED]) => A, merge: (A, A) => A, default: A, apply: (Vertex[VD], A) => VD, numIter: Int, - gatherEdges: EdgeDirection.EdgeDirection = EdgeDirection.In) = { + gatherEdges: EdgeDirection.EdgeDirection = EdgeDirection.In) : + Graph[VD, ED] = { var g = graph.mapVertices(v => Vertex(v.id, VDataWithAcc(v.data, default))).cache() From 28d0557fd807cd930a070e67df6208abd7078ba2 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Wed, 3 Apr 2013 23:23:07 -0700 Subject: [PATCH 012/531] added graph generator to run additional experiments --- graph/src/main/scala/spark/graph/Graph.scala | 52 ++++++++++++++++---- 1 file changed, 43 insertions(+), 9 deletions(-) diff --git a/graph/src/main/scala/spark/graph/Graph.scala b/graph/src/main/scala/spark/graph/Graph.scala index f8e16f24a3de6..cc73cb089b539 100644 --- a/graph/src/main/scala/spark/graph/Graph.scala +++ b/graph/src/main/scala/spark/graph/Graph.scala @@ -175,21 +175,55 @@ object Graph { Edge(source.trim.toInt, target.trim.toInt, edata) }.cache() - // Parse the vertex data table - val vertices = edges.flatMap { edge => List((edge.src, 1), (edge.dst, 1)) } - .reduceByKey(_ + _) - .map(new Vertex(_)) + val graph = fromEdges(edges) + println("Loaded graph:" + + "\n\t#edges: " + graph.numEdges + + "\n\t#vertices: " + graph.numVertices) + + graph + } - val graph = new Graph[Int, ED](vertices, edges) - graph.cache() - println("Loaded graph:" + - "\n\t#edges: " + graph.edges.count + - "\n\t#vertices: " + graph.vertices.count) + def fromEdges[ED: ClassManifest](edges: RDD[Edge[ED]]): Graph[Int, ED] = { + val vertices = edges.flatMap { edge => List((edge.src, 1), (edge.dst, 1)) } + .reduceByKey(_ + _) + .map{ case (vid, degree) => Vertex(vid, degree) } + new Graph[Int, ED](vertices, edges).cache + } + /** + * Make k-cycles + */ + def kCycles(sc: SparkContext, numCycles: Int = 3, size: Int = 3) = { + // Construct the edges + val edges = sc.parallelize(for (i <- 0 until numCycles; j <- 0 until size) yield { + val offset = i * numCycles + val source = offset + j + val target = offset + ((j + 1) % size) + Edge(source, target, i * numCycles + j) + }) + // Change vertex data to be the lowest vertex id of the vertex in that cycle + val graph = fromEdges(edges).mapVertices{ + case Vertex(id, degree) => Vertex(id, (id/numCycles) * numCycles) + } graph } + /** + * Make a regular grid graph + **/ + def grid(sc: SparkContext, numRows: Int = 5, numCols: Int = 5) = { + def coord(vid: Int) = (vid % numRows, vid / numRows) + val vertices = sc.parallelize( 0 until (numRows * numCols) ).map( + vid => Vertex(vid, coord(vid))) + def index(r: Int, c:Int) = (r + c * numRows) + val edges = vertices.flatMap{ case Vertex(vid, (r,c)) => + (if(r+1 < numRows) List(Edge(vid, index(r+1,c), 1.0F)) else List.empty) ++ + (if(c+1 < numCols) List(Edge(vid, index(r,c+1), 1.0F)) else List.empty) + } + new Graph(vertices, edges) + } + /** * A partition of edges in 3 large columnar arrays. From 1671abf47c40fde52132edd80adcdfab2cfbf820 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 5 Apr 2013 01:47:59 +0800 Subject: [PATCH 013/531] Implemented mapReduceNeighborhood in Graph and used that to implement gather in GraphLab. --- .../main/scala/spark/graph/Analytics.scala | 4 +- .../spark/graph/EdgeWithVerticesRDD.scala | 30 ++--- graph/src/main/scala/spark/graph/Graph.scala | 105 ++++++++++++++---- .../src/main/scala/spark/graph/GraphLab.scala | 56 +++------- .../src/main/scala/spark/graph/package.scala | 1 - 5 files changed, 109 insertions(+), 87 deletions(-) diff --git a/graph/src/main/scala/spark/graph/Analytics.scala b/graph/src/main/scala/spark/graph/Analytics.scala index b5ffaf1b344fa..9d8c0338ed1d2 100644 --- a/graph/src/main/scala/spark/graph/Analytics.scala +++ b/graph/src/main/scala/spark/graph/Analytics.scala @@ -54,7 +54,7 @@ object Analytics { Integer.MAX_VALUE, (v, a: Int) => math.min(v.data, a), // apply numIter, - gatherEdges = EdgeDirection.Both).vertices + gatherDirection = EdgeDirection.Both).vertices } @@ -72,7 +72,7 @@ object Analytics { Float.MaxValue, (v, a: Float) => math.min(v.data, a), // apply numIter, - gatherEdges = EdgeDirection.In).vertices + gatherDirection = EdgeDirection.In).vertices } diff --git a/graph/src/main/scala/spark/graph/EdgeWithVerticesRDD.scala b/graph/src/main/scala/spark/graph/EdgeWithVerticesRDD.scala index f200f7cbf668c..f804c792c20c0 100644 --- a/graph/src/main/scala/spark/graph/EdgeWithVerticesRDD.scala +++ b/graph/src/main/scala/spark/graph/EdgeWithVerticesRDD.scala @@ -7,7 +7,8 @@ import spark.graph.Graph.EdgePartition private[graph] -class EdgeWithVerticesPartition(idx: Int, val eTablePartition: Partition) extends Partition { +class EdgeWithVerticesPartition(idx: Int, val vPart: Partition, val ePart: Partition) + extends Partition { override val index: Int = idx override def hashCode(): Int = idx } @@ -18,34 +19,24 @@ class EdgeWithVerticesPartition(idx: Int, val eTablePartition: Partition) extend */ private[graph] class EdgeWithVerticesRDD[VD: ClassManifest, ED: ClassManifest]( - @transient vTable: RDD[(Vid, (VD, Array[Pid]))], + vTableReplicated: RDD[(Vid, VD)], eTable: RDD[(Pid, EdgePartition[ED])]) extends RDD[(VertexHashMap[VD], Iterator[EdgeWithVertices[VD, ED]])](eTable.context, Nil) { - @transient - private val shuffleDependency = { - // Join vid2pid and vTable, generate a shuffle dependency on the joined result, and get - // the shuffle id so we can use it on the slave. - val vTableReplicated = vTable.flatMap { case (vid, (vdata, pids)) => - pids.iterator.map { pid => (pid, (vid, vdata)) } - } - new ShuffleDependency(vTableReplicated, eTable.partitioner.get) - } - - private val shuffleId = shuffleDependency.shuffleId + assert(vTableReplicated.partitioner == eTable.partitioner) override def getDependencies: List[Dependency[_]] = { - List(new OneToOneDependency(eTable), shuffleDependency) + List(new OneToOneDependency(eTable), new OneToOneDependency(vTableReplicated)) } override def getPartitions = Array.tabulate[Partition](eTable.partitions.size) { - i => new EdgeWithVerticesPartition(i, eTable.partitions(i)): Partition + i => new EdgeWithVerticesPartition(i, eTable.partitions(i), vTableReplicated.partitions(i)) } override val partitioner = eTable.partitioner override def getPreferredLocations(s: Partition) = - eTable.preferredLocations(s.asInstanceOf[EdgeWithVerticesPartition].eTablePartition) + eTable.preferredLocations(s.asInstanceOf[EdgeWithVerticesPartition].ePart) override def compute(s: Partition, context: TaskContext) : Iterator[(VertexHashMap[VD], Iterator[EdgeWithVertices[VD, ED]])] = { @@ -55,12 +46,9 @@ class EdgeWithVerticesRDD[VD: ClassManifest, ED: ClassManifest]( // Fetch the vertices and put them in a hashmap. // TODO: use primitive hashmaps for primitive VD types. val vmap = new VertexHashMap[VD]//(1000000) - val fetcher = SparkEnv.get.shuffleFetcher - fetcher.fetch[Pid, (Vid, VD)](shuffleId, split.index, context.taskMetrics).foreach { - case (pid, (vid, vdata)) => vmap.put(vid, vdata) - } + vTableReplicated.iterator(split.vPart, context).foreach { v => vmap.put(v._1, v._2) } - val (pid, edgePartition) = eTable.iterator(split.eTablePartition, context).next() + val (pid, edgePartition) = eTable.iterator(split.ePart, context).next() .asInstanceOf[(Pid, EdgePartition[ED])] // Return an iterator that looks up the hash map to find matching vertices for each edge. diff --git a/graph/src/main/scala/spark/graph/Graph.scala b/graph/src/main/scala/spark/graph/Graph.scala index cc73cb089b539..968f44e98b412 100644 --- a/graph/src/main/scala/spark/graph/Graph.scala +++ b/graph/src/main/scala/spark/graph/Graph.scala @@ -5,7 +5,7 @@ import scala.collection.mutable.ArrayBuffer import it.unimi.dsi.fastutil.ints.IntArrayList -import spark.{ClosureCleaner, HashPartitioner, RDD} +import spark.{ClosureCleaner, HashPartitioner, Partitioner, RDD} import spark.SparkContext import spark.SparkContext._ import spark.graph.Graph.EdgePartition @@ -17,6 +17,8 @@ case class Vertex[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) VD var data: VD = nullValue[VD]) { def this(tuple: Tuple2[Vid, VD]) = this(tuple._1, tuple._2) + + def tuple = (id, data) } @@ -38,6 +40,13 @@ class EdgeWithVertices[@specialized(Char, Int, Boolean, Byte, Long, Float, Doubl } +private[graph] +case class MutableTuple2[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) U, + @specialized(Char, Int, Boolean, Byte, Long, Float, Double) V]( + var _1: U, var _2: V) + + + /** * A Graph RDD that supports computation on graphs. */ @@ -58,7 +67,7 @@ class Graph[VD: ClassManifest, ED: ClassManifest] protected ( this } - /// Todo: Should theses be set on construction and passed onto derived graphs? + /// Todo: Should theses be set on construction and passed onto derived graphs? Yes. var numEdgePartitions = 5 var numVertexPartitions = 5 @@ -66,13 +75,17 @@ class Graph[VD: ClassManifest, ED: ClassManifest] protected ( protected val vertexPartitioner = new HashPartitioner(numVertexPartitions) protected val edgePartitioner = new HashPartitioner(numEdgePartitions) - - lazy val numEdges = edges.count() + lazy val numVertices = vertices.count() - lazy val inDegrees = edges.map{ case Edge(src, target, _) => (target, 1) }.reduceByKey(_+_) - lazy val outDegrees = edges.map{ case Edge(src, target, _) => (src, 1) }.reduceByKey(_+_) + lazy val inDegrees = edges + .map { case Edge(src, target, _) => (target, 1) } + .reduceByKey(vertexPartitioner, _ + _) + + lazy val outDegrees = edges + .map { case Edge(src, target, _) => (src, 1) } + .reduceByKey(vertexPartitioner, _ + _) protected lazy val eTable: RDD[(Pid, EdgePartition[ED])] = { if (_rawETable == null) { @@ -90,6 +103,17 @@ class Graph[VD: ClassManifest, ED: ClassManifest] protected ( } } + protected lazy val vTableReplicated: RDD[(Vid, VD)] = { + // Join vid2pid and vTable, generate a shuffle dependency on the joined result, and get + // the shuffle id so we can use it on the slave. + vTable + .flatMap { case (vid, (vdata, pids)) => pids.iterator.map { pid => (pid, (vid, vdata)) } } + .partitionBy(edgePartitioner) + .mapPartitions( + { part => part.map { case(pid, (vid, vdata)) => (vid, vdata) } }, + preservesPartitioning = true) + } + def vertices: RDD[Vertex[VD]] = { if (!_cached && _rawVertices != null) { _rawVertices @@ -107,7 +131,7 @@ class Graph[VD: ClassManifest, ED: ClassManifest] protected ( } def edgesWithVertices: RDD[EdgeWithVertices[VD, ED]] = { - (new EdgeWithVerticesRDD[VD, ED](vTable, eTable)).mapPartitions { part => part.next._2 } + (new EdgeWithVerticesRDD(vTableReplicated, eTable)).mapPartitions { part => part.next._2 } } def mapVertices[VD2: ClassManifest](f: (Vertex[VD]) => Vertex[VD2]) = { @@ -120,20 +144,55 @@ class Graph[VD: ClassManifest, ED: ClassManifest] protected ( new Graph(vertices, edges.map(f)) } + def mapReduceNeighborhood[VD2: ClassManifest]( + mapFunc: (Vid, EdgeWithVertices[VD, ED]) => VD2, + reduceFunc: (VD2, VD2) => VD2, + default: VD2, + gatherDirection: EdgeDirection.EdgeDirection): RDD[(Vid, VD2)] = { + + ClosureCleaner.clean(mapFunc) + ClosureCleaner.clean(reduceFunc) + //ClosureCleaner.clean(default) + + val newVTable = vTableReplicated.mapPartitions({ part => + part.map { v => (v._1, MutableTuple2(v._2, default)) } + }, preservesPartitioning = true) + + (new EdgeWithVerticesRDD[MutableTuple2[VD, VD2], ED](newVTable, eTable)) + .mapPartitions { part => + val (vmap, edges) = part.next() + val edgeSansAcc = new EdgeWithVertices[VD, ED]() + edges.map { edge: EdgeWithVertices[MutableTuple2[VD, VD2], ED] => + edgeSansAcc.data = edge.data + edgeSansAcc.src.data = edge.src.data._1 + edgeSansAcc.dst.data = edge.dst.data._1 + edgeSansAcc.src.id = edge.src.id + edgeSansAcc.dst.id = edge.dst.id + if (gatherDirection == EdgeDirection.In || gatherDirection == EdgeDirection.Both) { + edge.dst.data._2 = reduceFunc(edge.dst.data._2, mapFunc(edgeSansAcc.dst.id, edgeSansAcc)) + } + if (gatherDirection == EdgeDirection.Out || gatherDirection == EdgeDirection.Both) { + edge.src.data._2 = reduceFunc(edge.src.data._2, mapFunc(edgeSansAcc.src.id, edgeSansAcc)) + } + } + + vmap.int2ObjectEntrySet().fastIterator().map{ entry => + (entry.getIntKey(), entry.getValue()._2) + } + } + .combineByKey((v: VD2) => v, reduceFunc, null, vertexPartitioner, false) + } + def updateVertices[U: ClassManifest, VD2: ClassManifest]( updates: RDD[(Vid, U)], - updateFunc: (Vertex[VD], Seq[U]) => VD2) + updateFunc: (Vertex[VD], Option[U]) => VD2) : Graph[VD2, ED] = { ClosureCleaner.clean(updateFunc) - val joined: RDD[(Vid, ((VD, Array[Pid]), Option[Seq[U]]))] = - vTable.leftOuterJoin(updates.groupByKey(vertexPartitioner)) - - val newVTable = joined.mapPartitions({ iter => - iter.map { case (vid, ((vdata, pids), updates)) => - val u = if (updates.isDefined) updates.get else Seq.empty - val newVdata = updateFunc(Vertex(vid, vdata), u) + val newVTable = vTable.leftOuterJoin(updates).mapPartitions({ iter => + iter.map { case (vid, ((vdata, pids), update)) => + val newVdata = updateFunc(Vertex(vid, vdata), update) (vid, (newVdata, pids)) } }, preservesPartitioning = true).cache() @@ -141,14 +200,14 @@ class Graph[VD: ClassManifest, ED: ClassManifest] protected ( new Graph(null, null, newVTable, eTable) } - def mapPartitions[U: ClassManifest]( - f: (VertexHashMap[VD], Iterator[EdgeWithVertices[VD, ED]]) => Iterator[U], - preservesPartitioning: Boolean = false): RDD[U] = { - (new EdgeWithVerticesRDD(vTable, eTable)).mapPartitions({ part => - val (vmap, iter) = part.next() - f(vmap, iter) - }, preservesPartitioning) - } + // def mapPartitions[U: ClassManifest]( + // f: (VertexHashMap[VD], Iterator[EdgeWithVertices[VD, ED]]) => Iterator[U], + // preservesPartitioning: Boolean = false): RDD[U] = { + // (new EdgeWithVerticesRDD(vTable, eTable)).mapPartitions({ part => + // val (vmap, iter) = part.next() + // f(vmap, iter) + // }, preservesPartitioning) + // } } diff --git a/graph/src/main/scala/spark/graph/GraphLab.scala b/graph/src/main/scala/spark/graph/GraphLab.scala index 4bc7c4d2e5948..eb992c464aaa2 100644 --- a/graph/src/main/scala/spark/graph/GraphLab.scala +++ b/graph/src/main/scala/spark/graph/GraphLab.scala @@ -6,55 +6,31 @@ import spark.RDD object GraphLab { - def iterateGAS[VD: ClassManifest, ED: ClassManifest, A: ClassManifest](graph: Graph[VD, ED])( - gather: (Vid, EdgeWithVertices[VD, ED]) => A, - merge: (A, A) => A, - default: A, - apply: (Vertex[VD], A) => VD, - numIter: Int, - gatherEdges: EdgeDirection.EdgeDirection = EdgeDirection.In) : - Graph[VD, ED] = { - - var g = graph.mapVertices(v => Vertex(v.id, VDataWithAcc(v.data, default))).cache() + def iterateGAS[VD: ClassManifest, ED: ClassManifest, A: ClassManifest]( + graph: Graph[VD, ED])( + gather: (Vid, EdgeWithVertices[VD, ED]) => A, + merge: (A, A) => A, + default: A, + apply: (Vertex[VD], A) => VD, + numIter: Int, + gatherDirection: EdgeDirection.EdgeDirection = EdgeDirection.In) + : Graph[VD, ED] = { + + var g = graph.cache() var i = 0 while (i < numIter) { - val accUpdates: RDD[(Vid, A)] = g.mapPartitions({ case(vmap, iter) => - val edgeSansAcc = new EdgeWithVertices[VD, ED]() - iter.map { edge: EdgeWithVertices[VDataWithAcc[VD, A], ED] => - edgeSansAcc.data = edge.data - edgeSansAcc.src.data = edge.src.data.vdata - edgeSansAcc.dst.data = edge.dst.data.vdata - edgeSansAcc.src.id = edge.src.id - edgeSansAcc.dst.id = edge.dst.id - if (gatherEdges == EdgeDirection.In || gatherEdges == EdgeDirection.Both) { - edge.dst.data.acc = merge(edge.dst.data.acc, gather(edgeSansAcc.dst.id, edgeSansAcc)) - } - if (gatherEdges == EdgeDirection.Out || gatherEdges == EdgeDirection.Both) { - edge.src.data.acc = merge(edge.src.data.acc, gather(edgeSansAcc.src.id, edgeSansAcc)) - } - } - - vmap.int2ObjectEntrySet().fastIterator().map{ entry => - (entry.getIntKey(), entry.getValue().acc) - } - })(classManifest[(Int, A)]) - - def applyFunc(v: Vertex[VDataWithAcc[VD, A]], updates: Seq[A]): VDataWithAcc[VD, A] = { - VDataWithAcc(apply(Vertex(v.id, v.data.vdata), updates.reduce(merge)), default) - } + val accUpdates: RDD[(Vid, A)] = g.mapReduceNeighborhood( + gather, merge, default, gatherDirection) + + def applyFunc(v: Vertex[VD], update: Option[A]): VD = { apply(v, update.get) } g = g.updateVertices(accUpdates, applyFunc).cache() i += 1 } - // return the transformed graph with the accumulator removed - g.mapVertices(v => Vertex(v.id, v.data.vdata)) + g } } - - -private[graph] -sealed case class VDataWithAcc[VD: ClassManifest, A](var vdata: VD, var acc: A) diff --git a/graph/src/main/scala/spark/graph/package.scala b/graph/src/main/scala/spark/graph/package.scala index 6db050e6e16cb..e7ec3f6e862fd 100644 --- a/graph/src/main/scala/spark/graph/package.scala +++ b/graph/src/main/scala/spark/graph/package.scala @@ -11,5 +11,4 @@ package object graph { * Return the default null-like value for a data type T. */ def nullValue[T] = null.asInstanceOf[T] - } From 93eca18a62cc34fee044c5e5e4b01a53353cdd27 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Thu, 4 Apr 2013 15:35:07 -0700 Subject: [PATCH 014/531] fixing a silly bug whereby the pagerank equation was implemented incorrectly (divided by degree of dst instead of source). --- graph/src/main/scala/spark/graph/Analytics.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/graph/src/main/scala/spark/graph/Analytics.scala b/graph/src/main/scala/spark/graph/Analytics.scala index 9d8c0338ed1d2..27b8e371d4a37 100644 --- a/graph/src/main/scala/spark/graph/Analytics.scala +++ b/graph/src/main/scala/spark/graph/Analytics.scala @@ -32,7 +32,7 @@ object Analytics { (vertex, degIter) => (degIter.sum, 1.0F) ) GraphLab.iterateGAS[(Int, Float), ED, Float](pagerankGraph)( - (me_id, edge) => edge.src.data._2 / edge.dst.data._1, // gather + (me_id, edge) => edge.src.data._2 / edge.src.data._1, // gather (a: Float, b: Float) => a + b, // merge 0F, (vertex, a: Float) => (vertex.data._1, (0.15F + 0.85F * a)), // apply From db45cf3a497ffdde17a4964d8c31da6e995ad03a Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Thu, 4 Apr 2013 15:37:02 -0700 Subject: [PATCH 015/531] Fixing several bugs in mapReduceNeighborhood. First a map is used instead of a foreach which for mysterious reasons meant that the map never seems to be executed? Switching to a foreach causes a null pointer exception since the body of the foreach did not properly initialize the temporary EdgeWithVertex data structure. --- .../spark/graph/EdgeWithVerticesRDD.scala | 4 ++++ graph/src/main/scala/spark/graph/Graph.scala | 22 ++++++++++++------- 2 files changed, 18 insertions(+), 8 deletions(-) diff --git a/graph/src/main/scala/spark/graph/EdgeWithVerticesRDD.scala b/graph/src/main/scala/spark/graph/EdgeWithVerticesRDD.scala index f804c792c20c0..c27ce586e9d75 100644 --- a/graph/src/main/scala/spark/graph/EdgeWithVerticesRDD.scala +++ b/graph/src/main/scala/spark/graph/EdgeWithVerticesRDD.scala @@ -61,9 +61,13 @@ class EdgeWithVerticesRDD[VD: ClassManifest, ED: ClassManifest]( override def hasNext: Boolean = pos < edgePartition.size override def next() = { e.src.id = edgePartition.srcIds.getInt(pos) + // assert(vmap.containsKey(e.src.id)) e.src.data = vmap.get(e.src.id) + e.dst.id = edgePartition.dstIds.getInt(pos) + // assert(vmap.containsKey(e.dst.id)) e.dst.data = vmap.get(e.dst.id) + e.data = edgePartition.data(pos) pos += 1 e diff --git a/graph/src/main/scala/spark/graph/Graph.scala b/graph/src/main/scala/spark/graph/Graph.scala index 968f44e98b412..1117cb5a68f1b 100644 --- a/graph/src/main/scala/spark/graph/Graph.scala +++ b/graph/src/main/scala/spark/graph/Graph.scala @@ -79,13 +79,17 @@ class Graph[VD: ClassManifest, ED: ClassManifest] protected ( lazy val numVertices = vertices.count() - lazy val inDegrees = edges + lazy val inDegrees = mapReduceNeighborhood[Vid]((vid, edge) => 1, _+_, 0, EdgeDirection.In) + + lazy val outDegrees = mapReduceNeighborhood[Vid]((vid,edge) => 1, _+_, 0, EdgeDirection.Out) + + lazy val degrees = mapReduceNeighborhood[Vid]((vid,edge) => 1, _+_, 0, EdgeDirection.Both) + + + edges .map { case Edge(src, target, _) => (target, 1) } .reduceByKey(vertexPartitioner, _ + _) - lazy val outDegrees = edges - .map { case Edge(src, target, _) => (src, 1) } - .reduceByKey(vertexPartitioner, _ + _) protected lazy val eTable: RDD[(Pid, EdgePartition[ED])] = { if (_rawETable == null) { @@ -144,6 +148,7 @@ class Graph[VD: ClassManifest, ED: ClassManifest] protected ( new Graph(vertices, edges.map(f)) } + def mapReduceNeighborhood[VD2: ClassManifest]( mapFunc: (Vid, EdgeWithVertices[VD, ED]) => VD2, reduceFunc: (VD2, VD2) => VD2, @@ -162,7 +167,9 @@ class Graph[VD: ClassManifest, ED: ClassManifest] protected ( .mapPartitions { part => val (vmap, edges) = part.next() val edgeSansAcc = new EdgeWithVertices[VD, ED]() - edges.map { edge: EdgeWithVertices[MutableTuple2[VD, VD2], ED] => + edgeSansAcc.src = new Vertex[VD] + edgeSansAcc.dst = new Vertex[VD] + edges.foreach { edge: EdgeWithVertices[MutableTuple2[VD, VD2], ED] => edgeSansAcc.data = edge.data edgeSansAcc.src.data = edge.src.data._1 edgeSansAcc.dst.data = edge.dst.data._1 @@ -175,14 +182,13 @@ class Graph[VD: ClassManifest, ED: ClassManifest] protected ( edge.src.data._2 = reduceFunc(edge.src.data._2, mapFunc(edgeSansAcc.src.id, edgeSansAcc)) } } - vmap.int2ObjectEntrySet().fastIterator().map{ entry => (entry.getIntKey(), entry.getValue()._2) } - } - .combineByKey((v: VD2) => v, reduceFunc, null, vertexPartitioner, false) + }.combineByKey((v: VD2) => v, reduceFunc, null, vertexPartitioner, false) } + def updateVertices[U: ClassManifest, VD2: ClassManifest]( updates: RDD[(Vid, U)], updateFunc: (Vertex[VD], Option[U]) => VD2) From cb99fc193c489ad3b481c165fd0c23cd596808df Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Thu, 4 Apr 2013 17:56:16 -0700 Subject: [PATCH 016/531] Added graphlab style implementation of Pregel and a more sophisticated version of mapReduceNeighborhoodFilter --- graph/src/main/scala/spark/graph/Graph.scala | 55 +++++++++++++++++++ .../src/main/scala/spark/graph/GraphLab.scala | 53 +++++++++++++----- graph/src/main/scala/spark/graph/Pregel.scala | 34 ++++++++++++ 3 files changed, 129 insertions(+), 13 deletions(-) create mode 100644 graph/src/main/scala/spark/graph/Pregel.scala diff --git a/graph/src/main/scala/spark/graph/Graph.scala b/graph/src/main/scala/spark/graph/Graph.scala index 1117cb5a68f1b..b264a5e209858 100644 --- a/graph/src/main/scala/spark/graph/Graph.scala +++ b/graph/src/main/scala/spark/graph/Graph.scala @@ -189,6 +189,61 @@ class Graph[VD: ClassManifest, ED: ClassManifest] protected ( } + + def mapReduceNeighborhoodFilter[VD2: ClassManifest]( + mapFunc: (Vid, EdgeWithVertices[VD, ED]) => Option[VD2], + reduceFunc: (VD2, VD2) => VD2, + gatherDirection: EdgeDirection.EdgeDirection): RDD[(Vid, VD2)] = { + + ClosureCleaner.clean(mapFunc) + ClosureCleaner.clean(reduceFunc) + + val newVTable = vTableReplicated.mapPartitions({ part => + part.map { v => (v._1, MutableTuple2(v._2, Option.empty[VD2])) } + }, preservesPartitioning = true) + + (new EdgeWithVerticesRDD[MutableTuple2[VD, Option[VD2]], ED](newVTable, eTable)) + .mapPartitions { part => + val (vmap, edges) = part.next() + val edgeSansAcc = new EdgeWithVertices[VD, ED]() + edgeSansAcc.src = new Vertex[VD] + edgeSansAcc.dst = new Vertex[VD] + edges.foreach { edge: EdgeWithVertices[MutableTuple2[VD, Option[VD2]], ED] => + edgeSansAcc.data = edge.data + edgeSansAcc.src.data = edge.src.data._1 + edgeSansAcc.dst.data = edge.dst.data._1 + edgeSansAcc.src.id = edge.src.id + edgeSansAcc.dst.id = edge.dst.id + if (gatherDirection == EdgeDirection.In || gatherDirection == EdgeDirection.Both) { + edge.dst.data._2 = + if(edge.dst.data._2.isEmpty) mapFunc(edgeSansAcc.dst.id, edgeSansAcc) + else { + val tmp = mapFunc(edgeSansAcc.dst.id, edgeSansAcc) + if(!tmp.isEmpty) Some(reduceFunc(edge.dst.data._2.get, tmp.get)) + else edge.dst.data._2 + } + } + if (gatherDirection == EdgeDirection.Out || gatherDirection == EdgeDirection.Both) { + edge.dst.data._2 = + if(edge.dst.data._2.isEmpty) mapFunc(edgeSansAcc.src.id, edgeSansAcc) + else { + val tmp = mapFunc(edgeSansAcc.src.id, edgeSansAcc) + if(!tmp.isEmpty) Some(reduceFunc(edge.src.data._2.get, tmp.get)) + else edge.src.data._2 + } + } + } + vmap.int2ObjectEntrySet().fastIterator().filter{!_.getValue()._2.isEmpty}.map{ entry => + (entry.getIntKey(), entry.getValue()._2) + } + } + .map{ case (vid, aOpt) => (vid, aOpt.get) } + .combineByKey((v: VD2) => v, reduceFunc, null, vertexPartitioner, false) + + } + + + def updateVertices[U: ClassManifest, VD2: ClassManifest]( updates: RDD[(Vid, U)], updateFunc: (Vertex[VD], Option[U]) => VD2) diff --git a/graph/src/main/scala/spark/graph/GraphLab.scala b/graph/src/main/scala/spark/graph/GraphLab.scala index eb992c464aaa2..8899d9688a860 100644 --- a/graph/src/main/scala/spark/graph/GraphLab.scala +++ b/graph/src/main/scala/spark/graph/GraphLab.scala @@ -7,30 +7,57 @@ import spark.RDD object GraphLab { def iterateGAS[VD: ClassManifest, ED: ClassManifest, A: ClassManifest]( - graph: Graph[VD, ED])( - gather: (Vid, EdgeWithVertices[VD, ED]) => A, - merge: (A, A) => A, - default: A, - apply: (Vertex[VD], A) => VD, - numIter: Int, - gatherDirection: EdgeDirection.EdgeDirection = EdgeDirection.In) - : Graph[VD, ED] = { + rawGraph: Graph[VD, ED])( + gather: (Vid, EdgeWithVertices[VD, ED]) => A, + merge: (A, A) => A, + default: A, + apply: (Vertex[VD], A) => VD, + numIter: Int, + gatherDirection: EdgeDirection.EdgeDirection = EdgeDirection.In) : Graph[VD, ED] = { - var g = graph.cache() + var graph = rawGraph.cache() var i = 0 while (i < numIter) { - val accUpdates: RDD[(Vid, A)] = g.mapReduceNeighborhood( - gather, merge, default, gatherDirection) + val accUpdates: RDD[(Vid, A)] = + graph.mapReduceNeighborhood(gather, merge, default, gatherDirection) def applyFunc(v: Vertex[VD], update: Option[A]): VD = { apply(v, update.get) } - g = g.updateVertices(accUpdates, applyFunc).cache() + graph = graph.updateVertices(accUpdates, applyFunc).cache() i += 1 } + graph + } + + + def iterateGASOption[VD: ClassManifest, ED: ClassManifest, A: ClassManifest]( + rawGraph: Graph[VD, ED])( + gather: (Vid, EdgeWithVertices[VD, ED]) => A, + merge: (A, A) => A, + apply: (Vertex[VD], Option[A]) => VD, + numIter: Int, + gatherDirection: EdgeDirection.EdgeDirection = EdgeDirection.In) : Graph[VD, ED] = { + + var graph = rawGraph.cache() + + def someGather(vid: Vid, edge: EdgeWithVertices[VD,ED]) = Some(gather(vid, edge)) + + var i = 0 + while (i < numIter) { + + val accUpdates: RDD[(Vid, A)] = + graph.mapReduceNeighborhoodFilter(someGather, merge, gatherDirection) - g + def applyFunc(v: Vertex[VD], update: Option[A]): VD = { apply(v, update) } + graph = graph.updateVertices(accUpdates, applyFunc).cache() + + i += 1 + } + graph } + + } diff --git a/graph/src/main/scala/spark/graph/Pregel.scala b/graph/src/main/scala/spark/graph/Pregel.scala new file mode 100644 index 0000000000000..6c551c21ddf8e --- /dev/null +++ b/graph/src/main/scala/spark/graph/Pregel.scala @@ -0,0 +1,34 @@ +package spark.graph + +import scala.collection.JavaConversions._ +import spark.RDD + + +object Pregel { + + def iterate[VD: ClassManifest, ED: ClassManifest, A: ClassManifest]( + rawGraph: Graph[VD, ED])( + vprog: ( Vertex[VD], A) => VD, + sendMsg: (Vid, EdgeWithVertices[VD, ED]) => Option[A], + mergeMsg: (A, A) => A, + numIter: Int) : Graph[VD, ED] = { + + var graph = rawGraph.cache + var i = 0 + while (i < numIter) { + + val msgs: RDD[(Vid, A)] = + graph.mapReduceNeighborhoodFilter(sendMsg, mergeMsg, EdgeDirection.In) + + def runProg(v: Vertex[VD], msg: Option[A]): VD = + if(msg.isEmpty) v.data else vprog(v, msg.get) + + graph = graph.updateVertices(msgs, runProg).cache() + + i += 1 + } + graph + + } + +} From 0667986c8eb1f325d807eb9def0440bafdb7b44d Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Thu, 4 Apr 2013 17:59:08 -0700 Subject: [PATCH 017/531] Minor tweak to pregel semantics to give reverse edge illusion on send message. --- graph/src/main/scala/spark/graph/Pregel.scala | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/graph/src/main/scala/spark/graph/Pregel.scala b/graph/src/main/scala/spark/graph/Pregel.scala index 6c551c21ddf8e..9ddae64b03e54 100644 --- a/graph/src/main/scala/spark/graph/Pregel.scala +++ b/graph/src/main/scala/spark/graph/Pregel.scala @@ -15,10 +15,14 @@ object Pregel { var graph = rawGraph.cache var i = 0 + + def reverseGather(vid: Vid, edge: EdgeWithVertices[VD,ED]) = + sendMsg(edge.otherVertex(vid).id, edge) + while (i < numIter) { val msgs: RDD[(Vid, A)] = - graph.mapReduceNeighborhoodFilter(sendMsg, mergeMsg, EdgeDirection.In) + graph.mapReduceNeighborhoodFilter(reverseGather, mergeMsg, EdgeDirection.In) def runProg(v: Vertex[VD], msg: Option[A]): VD = if(msg.isEmpty) v.data else vprog(v, msg.get) From 4a2b8aa55781a0aaa4a63b5da806fc72c58f9837 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Thu, 4 Apr 2013 18:59:56 -0700 Subject: [PATCH 018/531] added dynamic graphlab but I have not yet tested it. --- .../src/main/scala/spark/graph/GraphLab.scala | 82 ++++++++++++++++++- 1 file changed, 79 insertions(+), 3 deletions(-) diff --git a/graph/src/main/scala/spark/graph/GraphLab.scala b/graph/src/main/scala/spark/graph/GraphLab.scala index 8899d9688a860..d4fdb76040128 100644 --- a/graph/src/main/scala/spark/graph/GraphLab.scala +++ b/graph/src/main/scala/spark/graph/GraphLab.scala @@ -32,6 +32,8 @@ object GraphLab { } + + def iterateGASOption[VD: ClassManifest, ED: ClassManifest, A: ClassManifest]( rawGraph: Graph[VD, ED])( gather: (Vid, EdgeWithVertices[VD, ED]) => A, @@ -49,9 +51,7 @@ object GraphLab { val accUpdates: RDD[(Vid, A)] = graph.mapReduceNeighborhoodFilter(someGather, merge, gatherDirection) - - def applyFunc(v: Vertex[VD], update: Option[A]): VD = { apply(v, update) } - graph = graph.updateVertices(accUpdates, applyFunc).cache() + graph = graph.updateVertices(accUpdates, apply).cache() i += 1 } @@ -60,4 +60,80 @@ object GraphLab { + def iterateDynamic[VD: ClassManifest, ED: ClassManifest, A: ClassManifest]( + rawGraph: Graph[VD, ED])( + rawGather: (Vid, EdgeWithVertices[VD, ED]) => A, + merge: (A, A) => A, + rawApply: (Vertex[VD], Option[A]) => VD, + rawScatter: (Vid, EdgeWithVertices[VD, ED]) => Boolean, + numIter: Int, + gatherDirection: EdgeDirection.EdgeDirection = EdgeDirection.In, + rawScatterDirection: EdgeDirection.EdgeDirection = EdgeDirection.Out) : Graph[VD, ED] = { + + var graph = rawGraph.mapVertices{ case Vertex(id,data) => Vertex(id, (true, data)) }.cache() + + def gather(vid: Vid, e: EdgeWithVertices[(Boolean, VD), ED]) = { + if(e.vertex(vid).data._1) { + val edge = new EdgeWithVertices[VD,ED] + edge.src = Vertex(e.src.id, e.src.data._2) + edge.dst = Vertex(e.dst.id, e.dst.data._2) + Some(rawGather(vid, edge)) + } else { + None + } + } + + def apply(v: Vertex[(Boolean, VD)], accum: Option[A]) = { + if(v.data._1) (true, rawApply(Vertex(v.id, v.data._2), accum)) + else (false, v.data._2) + } + + def scatter(rawVid: Vid, e: EdgeWithVertices[(Boolean, VD),ED]) = { + val vid = e.otherVertex(rawVid).id + if(e.vertex(vid).data._1) { + val edge = new EdgeWithVertices[VD,ED] + edge.src = Vertex(e.src.id, e.src.data._2) + edge.dst = Vertex(e.dst.id, e.dst.data._2) + Some(rawScatter(vid, edge)) + } else { + None + } + } + + val scatterDirection = rawScatterDirection match { + case EdgeDirection.In => EdgeDirection.Out + case EdgeDirection.Out => EdgeDirection.In + case _ => rawScatterDirection + } + + def applyActive(v: Vertex[(Boolean, VD)], accum: Option[Boolean]) = + (accum.getOrElse(false), v.data._2) + + + + var i = 0 + var numActive = graph.numVertices + while (i < numIter && numActive > 0) { + + val accUpdates: RDD[(Vid, A)] = + graph.mapReduceNeighborhoodFilter(gather, merge, gatherDirection) + + graph = graph.updateVertices(accUpdates, apply).cache() + + val activeVertices: RDD[(Vid, Boolean)] = + graph.mapReduceNeighborhoodFilter(scatter, _ || _, scatterDirection) + + graph = graph.updateVertices(activeVertices, applyActive).cache() + + numActive = graph.vertices.map(v => if(v.data._1) 1 else 0).reduce( _ + _ ) + + println("Number active vertices: " + numActive) + + i += 1 + } + graph.mapVertices(v => Vertex(v.id, v.data._2)) + } + + + } From b53761357019b1c0ce3dd3094876dcee702aff37 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Thu, 4 Apr 2013 19:18:15 -0700 Subject: [PATCH 019/531] added dynamic graphlab --- .../main/scala/spark/graph/Analytics.scala | 81 +++++++++---------- .../src/main/scala/spark/graph/GraphLab.scala | 42 +++++----- 2 files changed, 57 insertions(+), 66 deletions(-) diff --git a/graph/src/main/scala/spark/graph/Analytics.scala b/graph/src/main/scala/spark/graph/Analytics.scala index 27b8e371d4a37..40f378fbc2106 100644 --- a/graph/src/main/scala/spark/graph/Analytics.scala +++ b/graph/src/main/scala/spark/graph/Analytics.scala @@ -23,6 +23,7 @@ import spark.SparkContext._ object Analytics { + /** * Compute the PageRank of a graph returning the pagerank of each vertex as an RDD */ @@ -31,15 +32,36 @@ object Analytics { val pagerankGraph = graph.updateVertices[Int, (Int, Float)](graph.outDegrees, (vertex, degIter) => (degIter.sum, 1.0F) ) - GraphLab.iterateGAS[(Int, Float), ED, Float](pagerankGraph)( + GraphLab.iterateGA[(Int, Float), ED, Float](pagerankGraph)( (me_id, edge) => edge.src.data._2 / edge.src.data._1, // gather (a: Float, b: Float) => a + b, // merge - 0F, - (vertex, a: Float) => (vertex.data._1, (0.15F + 0.85F * a)), // apply - numIter).vertices.map{ case Vertex(id, (outDeg, r)) => Vertex(id, r) } + (vertex, a: Option[Float]) => (vertex.data._1, (0.15F + 0.85F * a.getOrElse(0F))), // apply + numIter).mapVertices{ case Vertex(id, (outDeg, r)) => Vertex(id, r) } + } + + + /** + * Compute the PageRank of a graph returning the pagerank of each vertex as an RDD + */ + def dynamicPageRank[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], + tol: Float, maxIter: Int = 10) = { + // Compute the out degree of each vertex + val pagerankGraph = graph.updateVertices[Int, (Int, Float, Float)](graph.outDegrees, + (vertex, degIter) => (degIter.sum, 1.0F, 1.0F) + ) + + // Run PageRank + GraphLab.iterateGAS(pagerankGraph)( + (me_id, edge) => edge.src.data._2 / edge.dst.data._1, // gather + (a: Float, b: Float) => a + b, + (vertex, a: Option[Float]) => + (vertex.data._1, (0.15F + 0.85F * a.getOrElse(0F)), vertex.data._2), // apply + (me_id, edge) => math.abs(edge.src.data._2 - edge.dst.data._1) > tol, // scatter + maxIter).mapVertices { case Vertex(vid, data) => Vertex(vid, data._2) } } + /** * Compute the connected component membership of each vertex * and return an RDD with the vertex value containing the @@ -48,13 +70,12 @@ object Analytics { */ def connectedComponents[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], numIter: Int) = { val ccGraph = graph.mapVertices { case Vertex(vid, _) => Vertex(vid, vid) } - GraphLab.iterateGAS[Int, ED, Int](ccGraph)( + GraphLab.iterateGA[Int, ED, Int](ccGraph)( (me_id, edge) => edge.otherVertex(me_id).data, // gather (a: Int, b: Int) => math.min(a, b), // merge - Integer.MAX_VALUE, - (v, a: Int) => math.min(v.data, a), // apply + (v, a: Option[Int]) => math.min(v.data, a.getOrElse(Integer.MAX_VALUE)), // apply numIter, - gatherDirection = EdgeDirection.Both).vertices + gatherDirection = EdgeDirection.Both) } @@ -66,47 +87,17 @@ object Analytics { val spGraph = graph.mapVertices { case Vertex(vid, _) => Vertex(vid, (if(sourceSet.contains(vid)) 0.0F else Float.MaxValue)) } - GraphLab.iterateGAS[Float, Float, Float](spGraph)( + GraphLab.iterateGA[Float, Float, Float](spGraph)( (me_id, edge) => edge.otherVertex(me_id).data + edge.data, // gather (a: Float, b: Float) => math.min(a, b), // merge - Float.MaxValue, - (v, a: Float) => math.min(v.data, a), // apply + (v, a: Option[Float]) => math.min(v.data, a.getOrElse(Float.MaxValue)), // apply numIter, - gatherDirection = EdgeDirection.In).vertices + gatherDirection = EdgeDirection.In) } - // /** - // * Compute the PageRank of a graph returning the pagerank of each vertex as an RDD - // */ - // def dynamicPageRank[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], - // tol: Float, maxIter: Int = 10) = { - // graph.edges.cache - // // Compute the out degree of each vertex - // val outDegree = graph.edges.map { case (src, target, data) => (src, 1)}.reduceByKey(_ + _) - // val vertices = graph.vertices.leftOuterJoin(outDegree).mapValues { - // case (_, Some(deg)) => (deg, 1.0F, 1.0F) - // case (_, None) => (0, 1.0F, 1.0F) - // }.cache - - // val edges = graph.edges - // val pageRankGraph = new Graph(vertices, edges) - // pageRankGraph.numVPart = graph.numVPart - // pageRankGraph.numEPart = graph.numEPart - - // // Run PageRank - // pageRankGraph.iterateDynamic( - // (me_id, edge) => edge.source.data._2 / edge.source.data._1, // gather - // (a: Float, b: Float) => a + b, // merge - // 0F, - // (vertex, a: Float) => (vertex.data._1, (0.15F + 0.85F * a), vertex.data._2), // apply - // (me_id, edge) => math.abs(edge.source.data._2 - edge.source.data._1) > tol, // scatter - // maxIter).vertices.mapValues { case (degree, rank, oldRank) => rank } - // // println("Computed graph: #edges: " + graph_ret.numEdges + " #vertices" + graph_ret.numVertices) - // // graph_ret.vertices.take(10).foreach(println) - // } @@ -288,10 +279,10 @@ object Analytics { val pr = Analytics.pagerank(graph, numIter) // val pr = if(isDynamic) Analytics.dynamicPagerank(graph, tol, numIter) // else Analytics.pagerank(graph, numIter) - println("Total rank: " + pr.map{ case Vertex(id,r) => r }.reduce(_+_) ) + println("Total rank: " + pr.vertices.map{ case Vertex(id,r) => r }.reduce(_+_) ) if(!outFname.isEmpty) { println("Saving pageranks of pages to " + outFname) - pr.map{case Vertex(id, r) => id + "\t" + r}.saveAsTextFile(outFname) + pr.vertices.map{case Vertex(id, r) => id + "\t" + r}.saveAsTextFile(outFname) } println("Runtime: " + ((System.currentTimeMillis - startTime)/1000.0) + " seconds") sc.stop() @@ -325,7 +316,7 @@ object Analytics { val cc = Analytics.connectedComponents(graph, numIter) // val cc = if(isDynamic) Analytics.dynamicConnectedComponents(graph, numIter) // else Analytics.connectedComponents(graph, numIter) - println("Components: " + cc.map(_.data).distinct()) + println("Components: " + cc.vertices.map(_.data).distinct()) sc.stop() } @@ -368,7 +359,7 @@ object Analytics { val sp = Analytics.shortestPath(graph, sources, numIter) // val cc = if(isDynamic) Analytics.dynamicShortestPath(graph, sources, numIter) // else Analytics.shortestPath(graph, sources, numIter) - println("Longest Path: " + sp.map(_.data).reduce(math.max(_,_))) + println("Longest Path: " + sp.vertices.map(_.data).reduce(math.max(_,_))) sc.stop() } diff --git a/graph/src/main/scala/spark/graph/GraphLab.scala b/graph/src/main/scala/spark/graph/GraphLab.scala index d4fdb76040128..fa69e1ad9bdfc 100644 --- a/graph/src/main/scala/spark/graph/GraphLab.scala +++ b/graph/src/main/scala/spark/graph/GraphLab.scala @@ -6,35 +6,35 @@ import spark.RDD object GraphLab { - def iterateGAS[VD: ClassManifest, ED: ClassManifest, A: ClassManifest]( - rawGraph: Graph[VD, ED])( - gather: (Vid, EdgeWithVertices[VD, ED]) => A, - merge: (A, A) => A, - default: A, - apply: (Vertex[VD], A) => VD, - numIter: Int, - gatherDirection: EdgeDirection.EdgeDirection = EdgeDirection.In) : Graph[VD, ED] = { + // def iterateGA[VD: ClassManifest, ED: ClassManifest, A: ClassManifest]( + // rawGraph: Graph[VD, ED])( + // gather: (Vid, EdgeWithVertices[VD, ED]) => A, + // merge: (A, A) => A, + // default: A, + // apply: (Vertex[VD], A) => VD, + // numIter: Int, + // gatherDirection: EdgeDirection.EdgeDirection = EdgeDirection.In) : Graph[VD, ED] = { - var graph = rawGraph.cache() + // var graph = rawGraph.cache() - var i = 0 - while (i < numIter) { + // var i = 0 + // while (i < numIter) { - val accUpdates: RDD[(Vid, A)] = - graph.mapReduceNeighborhood(gather, merge, default, gatherDirection) + // val accUpdates: RDD[(Vid, A)] = + // graph.mapReduceNeighborhood(gather, merge, default, gatherDirection) - def applyFunc(v: Vertex[VD], update: Option[A]): VD = { apply(v, update.get) } - graph = graph.updateVertices(accUpdates, applyFunc).cache() + // def applyFunc(v: Vertex[VD], update: Option[A]): VD = { apply(v, update.get) } + // graph = graph.updateVertices(accUpdates, applyFunc).cache() - i += 1 - } - graph - } + // i += 1 + // } + // graph + // } - def iterateGASOption[VD: ClassManifest, ED: ClassManifest, A: ClassManifest]( + def iterateGA[VD: ClassManifest, ED: ClassManifest, A: ClassManifest]( rawGraph: Graph[VD, ED])( gather: (Vid, EdgeWithVertices[VD, ED]) => A, merge: (A, A) => A, @@ -60,7 +60,7 @@ object GraphLab { - def iterateDynamic[VD: ClassManifest, ED: ClassManifest, A: ClassManifest]( + def iterateGAS[VD: ClassManifest, ED: ClassManifest, A: ClassManifest]( rawGraph: Graph[VD, ED])( rawGather: (Vid, EdgeWithVertices[VD, ED]) => A, merge: (A, A) => A, From d510045d8c156beafaa3e4832fac739ff5309240 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Thu, 4 Apr 2013 19:24:06 -0700 Subject: [PATCH 020/531] fixing the the silly bug in the dynamic pagerank gather function --- graph/src/main/scala/spark/graph/Analytics.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/graph/src/main/scala/spark/graph/Analytics.scala b/graph/src/main/scala/spark/graph/Analytics.scala index 40f378fbc2106..18abbe4f2fd9a 100644 --- a/graph/src/main/scala/spark/graph/Analytics.scala +++ b/graph/src/main/scala/spark/graph/Analytics.scala @@ -43,7 +43,7 @@ object Analytics { /** * Compute the PageRank of a graph returning the pagerank of each vertex as an RDD */ - def dynamicPageRank[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], + def dynamicPagerank[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], tol: Float, maxIter: Int = 10) = { // Compute the out degree of each vertex val pagerankGraph = graph.updateVertices[Int, (Int, Float, Float)](graph.outDegrees, @@ -52,7 +52,7 @@ object Analytics { // Run PageRank GraphLab.iterateGAS(pagerankGraph)( - (me_id, edge) => edge.src.data._2 / edge.dst.data._1, // gather + (me_id, edge) => edge.src.data._2 / edge.src.data._1, // gather (a: Float, b: Float) => a + b, (vertex, a: Option[Float]) => (vertex.data._1, (0.15F + 0.85F * a.getOrElse(0F)), vertex.data._2), // apply From ad908f75457dc898987104ded9eacf6118361e9e Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Thu, 4 Apr 2013 19:31:14 -0700 Subject: [PATCH 021/531] added pregel pagerank --- graph/src/main/scala/spark/graph/Analytics.scala | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/graph/src/main/scala/spark/graph/Analytics.scala b/graph/src/main/scala/spark/graph/Analytics.scala index 18abbe4f2fd9a..c5ac94e6bf50c 100644 --- a/graph/src/main/scala/spark/graph/Analytics.scala +++ b/graph/src/main/scala/spark/graph/Analytics.scala @@ -40,6 +40,22 @@ object Analytics { } + /** + * Compute the PageRank of a graph returning the pagerank of each vertex as an RDD + */ + def pregelPagerank[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], numIter: Int) = { + // Compute the out degree of each vertex + val pagerankGraph = graph.updateVertices[Int, (Int, Float)](graph.outDegrees, + (vertex, degIter) => (degIter.sum, 1.0F) + ) + Pregel.iterate[(Int, Float), ED, Float](pagerankGraph)( + (vertex, a: Float) => (vertex.data._1, (0.15F + 0.85F * a)), // apply + (me_id, edge) => Some(edge.src.data._2 / edge.src.data._1), // gather + (a: Float, b: Float) => a + b, // merge + numIter).mapVertices{ case Vertex(id, (outDeg, r)) => Vertex(id, r) } + } + + /** * Compute the PageRank of a graph returning the pagerank of each vertex as an RDD */ From d5b0f4dfa646ae58e5bc1d89909ba327d4740402 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Thu, 4 Apr 2013 19:54:14 -0700 Subject: [PATCH 022/531] added a function to collect the neighborhood of a vertex as well as the skeleton of the program for classic pregel --- graph/src/main/scala/spark/graph/Graph.scala | 23 ++++++++++++++++++- .../src/main/scala/spark/graph/GraphLab.scala | 13 +++++------ graph/src/main/scala/spark/graph/Pregel.scala | 21 ++++++++++++++++- 3 files changed, 48 insertions(+), 9 deletions(-) diff --git a/graph/src/main/scala/spark/graph/Graph.scala b/graph/src/main/scala/spark/graph/Graph.scala index b264a5e209858..31cf34b4e156f 100644 --- a/graph/src/main/scala/spark/graph/Graph.scala +++ b/graph/src/main/scala/spark/graph/Graph.scala @@ -37,6 +37,9 @@ class EdgeWithVertices[@specialized(Char, Int, Boolean, Byte, Long, Float, Doubl def otherVertex(vid: Vid): Vertex[VD] = if (src.id == vid) dst else src def vertex(vid: Vid): Vertex[VD] = if (src.id == vid) src else dst + + def relativeDirection(vid: Vid): EdgeDirection.EdgeDirection = + if(vid == src.id) EdgeDirection.Out else EdgeDirection.In } @@ -134,6 +137,11 @@ class Graph[VD: ClassManifest, ED: ClassManifest] protected ( } } + def reverse: Graph[VD,ED] = { + new Graph(vertices, edges.map{ case Edge(s,t,e) => Edge(t,s,e) }) + } + + def edgesWithVertices: RDD[EdgeWithVertices[VD, ED]] = { (new EdgeWithVerticesRDD(vTableReplicated, eTable)).mapPartitions { part => part.next._2 } } @@ -143,6 +151,7 @@ class Graph[VD: ClassManifest, ED: ClassManifest] protected ( new Graph(vertices.map(f), edges) } + def mapEdges[ED2: ClassManifest](f: (Edge[ED]) => Edge[ED2]) = { ClosureCleaner.clean(f) new Graph(vertices, edges.map(f)) @@ -190,7 +199,19 @@ class Graph[VD: ClassManifest, ED: ClassManifest] protected ( - def mapReduceNeighborhoodFilter[VD2: ClassManifest]( + def collectNeighborIds(edgeDirection: EdgeDirection.EdgeDirection) : RDD[(Vid, Array[Vid])] = { + mapReduceNeighborhood[Array[Vid]]( + (vid, edge) => Array(edge.otherVertex(vid).id), + (a,b) => a ++ b, + Array.empty[Vid], + edgeDirection) + } + + /** + * Same as mapReduceNeighborhood but map function can return none and there is no default value. + * As a consequence the resulting table may be much smaller than the set of vertices. + */ + def flatMapReduceNeighborhood[VD2: ClassManifest]( mapFunc: (Vid, EdgeWithVertices[VD, ED]) => Option[VD2], reduceFunc: (VD2, VD2) => VD2, gatherDirection: EdgeDirection.EdgeDirection): RDD[(Vid, VD2)] = { diff --git a/graph/src/main/scala/spark/graph/GraphLab.scala b/graph/src/main/scala/spark/graph/GraphLab.scala index fa69e1ad9bdfc..61e97094777a0 100644 --- a/graph/src/main/scala/spark/graph/GraphLab.scala +++ b/graph/src/main/scala/spark/graph/GraphLab.scala @@ -34,8 +34,7 @@ object GraphLab { - def iterateGA[VD: ClassManifest, ED: ClassManifest, A: ClassManifest]( - rawGraph: Graph[VD, ED])( + def iterateGA[VD: ClassManifest, ED: ClassManifest, A: ClassManifest](rawGraph: Graph[VD, ED])( gather: (Vid, EdgeWithVertices[VD, ED]) => A, merge: (A, A) => A, apply: (Vertex[VD], Option[A]) => VD, @@ -50,7 +49,7 @@ object GraphLab { while (i < numIter) { val accUpdates: RDD[(Vid, A)] = - graph.mapReduceNeighborhoodFilter(someGather, merge, gatherDirection) + graph.flatMapReduceNeighborhood(someGather, merge, gatherDirection) graph = graph.updateVertices(accUpdates, apply).cache() i += 1 @@ -60,8 +59,7 @@ object GraphLab { - def iterateGAS[VD: ClassManifest, ED: ClassManifest, A: ClassManifest]( - rawGraph: Graph[VD, ED])( + def iterateGAS[VD: ClassManifest, ED: ClassManifest, A: ClassManifest](rawGraph: Graph[VD, ED])( rawGather: (Vid, EdgeWithVertices[VD, ED]) => A, merge: (A, A) => A, rawApply: (Vertex[VD], Option[A]) => VD, @@ -100,6 +98,7 @@ object GraphLab { } } + // Scatter is basically a gather in the opposite direction so we reverse the edge direction val scatterDirection = rawScatterDirection match { case EdgeDirection.In => EdgeDirection.Out case EdgeDirection.Out => EdgeDirection.In @@ -116,12 +115,12 @@ object GraphLab { while (i < numIter && numActive > 0) { val accUpdates: RDD[(Vid, A)] = - graph.mapReduceNeighborhoodFilter(gather, merge, gatherDirection) + graph.flatMapReduceNeighborhood(gather, merge, gatherDirection) graph = graph.updateVertices(accUpdates, apply).cache() val activeVertices: RDD[(Vid, Boolean)] = - graph.mapReduceNeighborhoodFilter(scatter, _ || _, scatterDirection) + graph.flatMapReduceNeighborhood(scatter, _ || _, scatterDirection) graph = graph.updateVertices(activeVertices, applyActive).cache() diff --git a/graph/src/main/scala/spark/graph/Pregel.scala b/graph/src/main/scala/spark/graph/Pregel.scala index 9ddae64b03e54..851edfdd077ab 100644 --- a/graph/src/main/scala/spark/graph/Pregel.scala +++ b/graph/src/main/scala/spark/graph/Pregel.scala @@ -22,7 +22,7 @@ object Pregel { while (i < numIter) { val msgs: RDD[(Vid, A)] = - graph.mapReduceNeighborhoodFilter(reverseGather, mergeMsg, EdgeDirection.In) + graph.flatMapReduceNeighborhood(reverseGather, mergeMsg, EdgeDirection.In) def runProg(v: Vertex[VD], msg: Option[A]): VD = if(msg.isEmpty) v.data else vprog(v, msg.get) @@ -35,4 +35,23 @@ object Pregel { } + + def iterateOriginal[VD: ClassManifest, ED: ClassManifest, A: ClassManifest]( + rawGraph: Graph[VD, ED])( + vprog: ( Vertex[VD], A, Seq[Vid]) => Seq[(Vid, A)], + mergeMsg: (A, A) => A, + numIter: Int) : Graph[VD, ED] = { + + var graph = rawGraph.cache + var i = 0 + + val outNbrIds : RDD[(Vid, Array[Vid])] = graph.collectNeighborIds(EdgeDirection.Out) + + /// Todo implement + /// vprog takes the vertex, the message (A), and list of out neighbor ids + + graph + + } + } From 71348563515a3f97119c2723f20ae034eed57ea4 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 5 Apr 2013 12:54:59 +0800 Subject: [PATCH 023/531] numVertexPartitions and numEdgePartitions are now part of the constructor and are immutable. Also done some cleanups. --- .../main/scala/spark/graph/Analytics.scala | 4 +- graph/src/main/scala/spark/graph/Graph.scala | 293 +++++++++--------- 2 files changed, 156 insertions(+), 141 deletions(-) diff --git a/graph/src/main/scala/spark/graph/Analytics.scala b/graph/src/main/scala/spark/graph/Analytics.scala index c5ac94e6bf50c..5b0e5221ba946 100644 --- a/graph/src/main/scala/spark/graph/Analytics.scala +++ b/graph/src/main/scala/spark/graph/Analytics.scala @@ -287,9 +287,7 @@ object Analytics { println("======================================") val sc = new SparkContext(host, "PageRank(" + fname + ")") - val graph = Graph.textFile(sc, fname, a => 1.0F) - graph.numVertexPartitions = numVPart - graph.numEdgePartitions = numEPart + val graph = Graph.textFile(sc, fname, a => 1.0F).withPartitioner(numVPart, numEPart) val startTime = System.currentTimeMillis val pr = Analytics.pagerank(graph, numIter) diff --git a/graph/src/main/scala/spark/graph/Graph.scala b/graph/src/main/scala/spark/graph/Graph.scala index 31cf34b4e156f..1b1929cc260c6 100644 --- a/graph/src/main/scala/spark/graph/Graph.scala +++ b/graph/src/main/scala/spark/graph/Graph.scala @@ -49,78 +49,51 @@ case class MutableTuple2[@specialized(Char, Int, Boolean, Byte, Long, Float, Dou var _1: U, var _2: V) - /** * A Graph RDD that supports computation on graphs. */ class Graph[VD: ClassManifest, ED: ClassManifest] protected ( + val numVertexPartitions: Int, + val numEdgePartitions: Int, _rawVertices: RDD[Vertex[VD]], _rawEdges: RDD[Edge[ED]], _rawVTable: RDD[(Vid, (VD, Array[Pid]))], _rawETable: RDD[(Pid, EdgePartition[ED])]) { - def this(vertices: RDD[Vertex[VD]], edges: RDD[Edge[ED]]) = this(vertices, edges, null, null) - - protected var _cached = false - - def cache(): Graph[VD, ED] = { - eTable.cache() - vTable.cache() - _cached = true - this + def this(vertices: RDD[Vertex[VD]], edges: RDD[Edge[ED]]) = { + this( + Graph.DEFAULT_NUM_VERTEX_PARTITIONS, Graph.DEFAULT_NUM_EDGE_PARTITIONS, + vertices, edges, + null, null) } - /// Todo: Should theses be set on construction and passed onto derived graphs? Yes. - var numEdgePartitions = 5 - var numVertexPartitions = 5 - - /// Todo: Should these be passed onto derived graphs? - protected val vertexPartitioner = new HashPartitioner(numVertexPartitions) - protected val edgePartitioner = new HashPartitioner(numEdgePartitions) - - lazy val numEdges = edges.count() - - lazy val numVertices = vertices.count() - - lazy val inDegrees = mapReduceNeighborhood[Vid]((vid, edge) => 1, _+_, 0, EdgeDirection.In) - - lazy val outDegrees = mapReduceNeighborhood[Vid]((vid,edge) => 1, _+_, 0, EdgeDirection.Out) - - lazy val degrees = mapReduceNeighborhood[Vid]((vid,edge) => 1, _+_, 0, EdgeDirection.Both) - - - edges - .map { case Edge(src, target, _) => (target, 1) } - .reduceByKey(vertexPartitioner, _ + _) - - - protected lazy val eTable: RDD[(Pid, EdgePartition[ED])] = { - if (_rawETable == null) { - Graph.createETable(_rawEdges, numEdgePartitions) + def withPartitioner(numVertexPartitions: Int, numEdgePartitions: Int): Graph[VD, ED] = { + if (_cached) { + val newgraph = new Graph(numVertexPartitions, numEdgePartitions, null, null, _rawVTable, _rawETable) + newgraph.cache() } else { - _rawETable + new Graph(numVertexPartitions, numEdgePartitions, _rawVertices, _rawEdges, null, null) } } - protected lazy val vTable: RDD[(Vid, (VD, Array[Pid]))] = { - if (_rawVTable == null) { - Graph.createVTable(_rawVertices, eTable, numVertexPartitions) - } else { - _rawVTable - } + def withVertexPartitioner(numVertexPartitions: Int = Graph.DEFAULT_NUM_VERTEX_PARTITIONS) = { + withPartitioner(numVertexPartitions, numEdgePartitions) } - protected lazy val vTableReplicated: RDD[(Vid, VD)] = { - // Join vid2pid and vTable, generate a shuffle dependency on the joined result, and get - // the shuffle id so we can use it on the slave. - vTable - .flatMap { case (vid, (vdata, pids)) => pids.iterator.map { pid => (pid, (vid, vdata)) } } - .partitionBy(edgePartitioner) - .mapPartitions( - { part => part.map { case(pid, (vid, vdata)) => (vid, vdata) } }, - preservesPartitioning = true) + def withEdgePartitioner(numEdgePartitions: Int = Graph.DEFAULT_NUM_EDGE_PARTITIONS) = { + withPartitioner(numVertexPartitions, numEdgePartitions) } + protected var _cached = false + + def cache(): Graph[VD, ED] = { + eTable.cache() + vTable.cache() + _cached = true + this + } + + /** Return a RDD of vertices. */ def vertices: RDD[Vertex[VD]] = { if (!_cached && _rawVertices != null) { _rawVertices @@ -129,6 +102,7 @@ class Graph[VD: ClassManifest, ED: ClassManifest] protected ( } } + /** Return a RDD of edges. */ def edges: RDD[Edge[ED]] = { if (!_cached && _rawEdges != null) { _rawEdges @@ -137,26 +111,45 @@ class Graph[VD: ClassManifest, ED: ClassManifest] protected ( } } - def reverse: Graph[VD,ED] = { - new Graph(vertices, edges.map{ case Edge(s,t,e) => Edge(t,s,e) }) + /** Return a RDD that brings edges with its source and destination vertices together. */ + def edgesWithVertices: RDD[EdgeWithVertices[VD, ED]] = { + (new EdgeWithVerticesRDD(vTableReplicated, eTable)).mapPartitions { part => part.next._2 } } + lazy val numEdges: Long = edges.count() - def edgesWithVertices: RDD[EdgeWithVertices[VD, ED]] = { - (new EdgeWithVerticesRDD(vTableReplicated, eTable)).mapPartitions { part => part.next._2 } + lazy val numVertices: Long = vertices.count() + + lazy val inDegrees = mapReduceNeighborhood[Vid]((vid, edge) => 1, _+_, 0, EdgeDirection.In) + + lazy val outDegrees = mapReduceNeighborhood[Vid]((vid,edge) => 1, _+_, 0, EdgeDirection.Out) + + lazy val degrees = mapReduceNeighborhood[Vid]((vid,edge) => 1, _+_, 0, EdgeDirection.Both) + + /** Return a new graph with its edge directions reversed. */ + lazy val reverse: Graph[VD,ED] = { + newGraph(vertices, edges.map{ case Edge(s, t, e) => Edge(t, s, e) }) } - def mapVertices[VD2: ClassManifest](f: (Vertex[VD]) => Vertex[VD2]) = { - ClosureCleaner.clean(f) - new Graph(vertices.map(f), edges) + def collectNeighborIds(edgeDirection: EdgeDirection.EdgeDirection) : RDD[(Vid, Array[Vid])] = { + mapReduceNeighborhood[Array[Vid]]( + (vid, edge) => Array(edge.otherVertex(vid).id), + (a, b) => a ++ b, + Array.empty[Vid], + edgeDirection) } + def mapVertices[VD2: ClassManifest](f: (Vertex[VD]) => Vertex[VD2]): Graph[VD2, ED] = { + newGraph(vertices.map(f), edges) + } - def mapEdges[ED2: ClassManifest](f: (Edge[ED]) => Edge[ED2]) = { - ClosureCleaner.clean(f) - new Graph(vertices, edges.map(f)) + def mapEdges[ED2: ClassManifest](f: (Edge[ED]) => Edge[ED2]): Graph[VD, ED2] = { + newGraph(vertices, edges.map(f)) } + ////////////////////////////////////////////////////////////////////////////////////////////////// + // Lower level transformation methods + ////////////////////////////////////////////////////////////////////////////////////////////////// def mapReduceNeighborhood[VD2: ClassManifest]( mapFunc: (Vid, EdgeWithVertices[VD, ED]) => VD2, @@ -166,7 +159,6 @@ class Graph[VD: ClassManifest, ED: ClassManifest] protected ( ClosureCleaner.clean(mapFunc) ClosureCleaner.clean(reduceFunc) - //ClosureCleaner.clean(default) val newVTable = vTableReplicated.mapPartitions({ part => part.map { v => (v._1, MutableTuple2(v._2, default)) } @@ -178,38 +170,29 @@ class Graph[VD: ClassManifest, ED: ClassManifest] protected ( val edgeSansAcc = new EdgeWithVertices[VD, ED]() edgeSansAcc.src = new Vertex[VD] edgeSansAcc.dst = new Vertex[VD] - edges.foreach { edge: EdgeWithVertices[MutableTuple2[VD, VD2], ED] => - edgeSansAcc.data = edge.data - edgeSansAcc.src.data = edge.src.data._1 - edgeSansAcc.dst.data = edge.dst.data._1 - edgeSansAcc.src.id = edge.src.id - edgeSansAcc.dst.id = edge.dst.id + edges.foreach { e: EdgeWithVertices[MutableTuple2[VD, VD2], ED] => + edgeSansAcc.data = e.data + edgeSansAcc.src.data = e.src.data._1 + edgeSansAcc.dst.data = e.dst.data._1 + edgeSansAcc.src.id = e.src.id + edgeSansAcc.dst.id = e.dst.id if (gatherDirection == EdgeDirection.In || gatherDirection == EdgeDirection.Both) { - edge.dst.data._2 = reduceFunc(edge.dst.data._2, mapFunc(edgeSansAcc.dst.id, edgeSansAcc)) + e.dst.data._2 = reduceFunc(e.dst.data._2, mapFunc(edgeSansAcc.dst.id, edgeSansAcc)) } if (gatherDirection == EdgeDirection.Out || gatherDirection == EdgeDirection.Both) { - edge.src.data._2 = reduceFunc(edge.src.data._2, mapFunc(edgeSansAcc.src.id, edgeSansAcc)) + e.src.data._2 = reduceFunc(e.src.data._2, mapFunc(edgeSansAcc.src.id, edgeSansAcc)) } } vmap.int2ObjectEntrySet().fastIterator().map{ entry => (entry.getIntKey(), entry.getValue()._2) } - }.combineByKey((v: VD2) => v, reduceFunc, null, vertexPartitioner, false) - } - - - - def collectNeighborIds(edgeDirection: EdgeDirection.EdgeDirection) : RDD[(Vid, Array[Vid])] = { - mapReduceNeighborhood[Array[Vid]]( - (vid, edge) => Array(edge.otherVertex(vid).id), - (a,b) => a ++ b, - Array.empty[Vid], - edgeDirection) + } + .combineByKey((v: VD2) => v, reduceFunc, null, vertexPartitioner, false) } /** * Same as mapReduceNeighborhood but map function can return none and there is no default value. - * As a consequence the resulting table may be much smaller than the set of vertices. + * As a consequence, the resulting table may be much smaller than the set of vertices. */ def flatMapReduceNeighborhood[VD2: ClassManifest]( mapFunc: (Vid, EdgeWithVertices[VD, ED]) => Option[VD2], @@ -229,28 +212,28 @@ class Graph[VD: ClassManifest, ED: ClassManifest] protected ( val edgeSansAcc = new EdgeWithVertices[VD, ED]() edgeSansAcc.src = new Vertex[VD] edgeSansAcc.dst = new Vertex[VD] - edges.foreach { edge: EdgeWithVertices[MutableTuple2[VD, Option[VD2]], ED] => - edgeSansAcc.data = edge.data - edgeSansAcc.src.data = edge.src.data._1 - edgeSansAcc.dst.data = edge.dst.data._1 - edgeSansAcc.src.id = edge.src.id - edgeSansAcc.dst.id = edge.dst.id + edges.foreach { e: EdgeWithVertices[MutableTuple2[VD, Option[VD2]], ED] => + edgeSansAcc.data = e.data + edgeSansAcc.src.data = e.src.data._1 + edgeSansAcc.dst.data = e.dst.data._1 + edgeSansAcc.src.id = e.src.id + edgeSansAcc.dst.id = e.dst.id if (gatherDirection == EdgeDirection.In || gatherDirection == EdgeDirection.Both) { - edge.dst.data._2 = - if(edge.dst.data._2.isEmpty) mapFunc(edgeSansAcc.dst.id, edgeSansAcc) - else { + e.dst.data._2 = + if (e.dst.data._2.isEmpty) { + mapFunc(edgeSansAcc.dst.id, edgeSansAcc) + } else { val tmp = mapFunc(edgeSansAcc.dst.id, edgeSansAcc) - if(!tmp.isEmpty) Some(reduceFunc(edge.dst.data._2.get, tmp.get)) - else edge.dst.data._2 + if (!tmp.isEmpty) Some(reduceFunc(e.dst.data._2.get, tmp.get)) else e.dst.data._2 } } if (gatherDirection == EdgeDirection.Out || gatherDirection == EdgeDirection.Both) { - edge.dst.data._2 = - if(edge.dst.data._2.isEmpty) mapFunc(edgeSansAcc.src.id, edgeSansAcc) - else { + e.dst.data._2 = + if (e.dst.data._2.isEmpty) { + mapFunc(edgeSansAcc.src.id, edgeSansAcc) + } else { val tmp = mapFunc(edgeSansAcc.src.id, edgeSansAcc) - if(!tmp.isEmpty) Some(reduceFunc(edge.src.data._2.get, tmp.get)) - else edge.src.data._2 + if (!tmp.isEmpty) Some(reduceFunc(e.src.data._2.get, tmp.get)) else e.src.data._2 } } } @@ -260,11 +243,8 @@ class Graph[VD: ClassManifest, ED: ClassManifest] protected ( } .map{ case (vid, aOpt) => (vid, aOpt.get) } .combineByKey((v: VD2) => v, reduceFunc, null, vertexPartitioner, false) - } - - def updateVertices[U: ClassManifest, VD2: ClassManifest]( updates: RDD[(Vid, U)], updateFunc: (Vertex[VD], Option[U]) => VD2) @@ -279,7 +259,7 @@ class Graph[VD: ClassManifest, ED: ClassManifest] protected ( } }, preservesPartitioning = true).cache() - new Graph(null, null, newVTable, eTable) + new Graph(newVTable.partitioner.size, eTable.partitioner.size, null, null, newVTable, eTable) } // def mapPartitions[U: ClassManifest]( @@ -291,11 +271,54 @@ class Graph[VD: ClassManifest, ED: ClassManifest] protected ( // }, preservesPartitioning) // } + ////////////////////////////////////////////////////////////////////////////////////////////////// + // Internals hidden from callers + ////////////////////////////////////////////////////////////////////////////////////////////////// + + // TODO: Support non-hash partitioning schemes. + protected val vertexPartitioner = new HashPartitioner(numVertexPartitions) + protected val edgePartitioner = new HashPartitioner(numEdgePartitions) + + /** Create a new graph but keep the current partitioning scheme. */ + protected def newGraph[VD2: ClassManifest, ED2: ClassManifest]( + vertices: RDD[Vertex[VD2]], edges: RDD[Edge[ED2]]): Graph[VD2, ED2] = { + (new Graph[VD2, ED2](vertices, edges)).withPartitioner(numVertexPartitions, numEdgePartitions) + } + + protected lazy val eTable: RDD[(Pid, EdgePartition[ED])] = { + if (_rawETable == null) { + Graph.createETable(_rawEdges, numEdgePartitions) + } else { + _rawETable + } + } + + protected lazy val vTable: RDD[(Vid, (VD, Array[Pid]))] = { + if (_rawVTable == null) { + Graph.createVTable(_rawVertices, eTable, numVertexPartitions) + } else { + _rawVTable + } + } + + protected lazy val vTableReplicated: RDD[(Vid, VD)] = { + // Join vid2pid and vTable, generate a shuffle dependency on the joined result, and get + // the shuffle id so we can use it on the slave. + vTable + .flatMap { case (vid, (vdata, pids)) => pids.iterator.map { pid => (pid, (vid, vdata)) } } + .partitionBy(edgePartitioner) + .mapPartitions( + { part => part.map { case(pid, (vid, vdata)) => (vid, vdata) } }, + preservesPartitioning = true) + } } object Graph { + val DEFAULT_NUM_VERTEX_PARTITIONS = 5 + val DEFAULT_NUM_EDGE_PARTITIONS = 5 + /** * Load an edge list from file initializing the Graph RDD */ @@ -324,12 +347,11 @@ object Graph { graph } - def fromEdges[ED: ClassManifest](edges: RDD[Edge[ED]]): Graph[Int, ED] = { val vertices = edges.flatMap { edge => List((edge.src, 1), (edge.dst, 1)) } .reduceByKey(_ + _) .map{ case (vid, degree) => Vertex(vid, degree) } - new Graph[Int, ED](vertices, edges).cache + (new Graph[Int, ED](vertices, edges)) } /** @@ -370,7 +392,7 @@ object Graph { * A partition of edges in 3 large columnar arrays. */ private[graph] - class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED:ClassManifest] + class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED: ClassManifest] { val srcIds: IntArrayList = new IntArrayList val dstIds: IntArrayList = new IntArrayList @@ -407,11 +429,34 @@ object Graph { } } - private[graph] - def createVTable[VD: ClassManifest, ED: ClassManifest]( + /** + * Create the edge table RDD, which is much more efficient for Java heap storage than the + * normal edges data structure (RDD[(Vid, Vid, ED)]). + * + * The edge table contains multiple partitions, and each partition contains only one RDD + * key-value pair: the key is the partition id, and the value is an EdgePartition object + * containing all the edges in a partition. + */ + protected def createETable[ED: ClassManifest](edges: RDD[Edge[ED]], numPartitions: Int) + : RDD[(Pid, EdgePartition[ED])] = { + edges + .map { e => + // Random partitioning based on the source vertex id. + (math.abs(e.src) % numPartitions, (e.src, e.dst, e.data)) + } + .partitionBy(new HashPartitioner(numPartitions)) + .mapPartitionsWithIndex({ (pid, iter) => + val edgePartition = new Graph.EdgePartition[ED] + iter.foreach { case (_, (src, dst, data)) => edgePartition.add(src, dst, data) } + Iterator((pid, edgePartition)) + }, preservesPartitioning = true) + } + + protected def createVTable[VD: ClassManifest, ED: ClassManifest]( vertices: RDD[Vertex[VD]], eTable: RDD[(Pid, EdgePartition[ED])], - numPartitions: Int) = { + numPartitions: Int) + : RDD[(Vid, (VD, Array[Pid]))] = { val partitioner = new HashPartitioner(numPartitions) // A key-value RDD. The key is a vertex id, and the value is a list of @@ -437,32 +482,4 @@ object Graph { case (vdata, Some(pids)) => (vdata, pids.toArray) } } - - /** - * Create the edge table RDD, which is much more efficient for Java heap storage than the - * normal edges data structure (RDD[(Vid, Vid, ED)]). - * - * The edge table contains multiple partitions, and each partition contains only one RDD - * key-value pair: the key is the partition id, and the value is an EdgePartition object - * containing all the edges in a partition. - */ - private[graph] - def createETable[ED: ClassManifest](edges: RDD[Edge[ED]], numPartitions: Int) - : RDD[(Pid, EdgePartition[ED])] = { - - edges.map { e => - // Random partitioning based on the source vertex id. - (math.abs(e.src) % numPartitions, (e.src, e.dst, e.data)) - } - .partitionBy(new HashPartitioner(numPartitions)) - .mapPartitionsWithIndex({ (pid, iter) => - val edgePartition = new Graph.EdgePartition[ED] - iter.foreach { case (_, (src, dst, data)) => edgePartition.add(src, dst, data) } - Iterator((pid, edgePartition)) - }, preservesPartitioning = true) - } - - - } - From 3d7a4b1feffd745ebe447ad7e625a122e7f86fcb Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Thu, 4 Apr 2013 23:17:27 -0700 Subject: [PATCH 024/531] More realistic version of Pregel. --- graph/src/main/scala/spark/graph/Analytics.scala | 5 +++-- graph/src/main/scala/spark/graph/Pregel.scala | 8 +++++--- 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/graph/src/main/scala/spark/graph/Analytics.scala b/graph/src/main/scala/spark/graph/Analytics.scala index 5b0e5221ba946..8d37b2f26443f 100644 --- a/graph/src/main/scala/spark/graph/Analytics.scala +++ b/graph/src/main/scala/spark/graph/Analytics.scala @@ -30,7 +30,7 @@ object Analytics { def pagerank[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], numIter: Int) = { // Compute the out degree of each vertex val pagerankGraph = graph.updateVertices[Int, (Int, Float)](graph.outDegrees, - (vertex, degIter) => (degIter.sum, 1.0F) + (vertex, deg) => (deg.getOrElse(0), 1.0F) ) GraphLab.iterateGA[(Int, Float), ED, Float](pagerankGraph)( (me_id, edge) => edge.src.data._2 / edge.src.data._1, // gather @@ -46,12 +46,13 @@ object Analytics { def pregelPagerank[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], numIter: Int) = { // Compute the out degree of each vertex val pagerankGraph = graph.updateVertices[Int, (Int, Float)](graph.outDegrees, - (vertex, degIter) => (degIter.sum, 1.0F) + (vertex, deg) => (deg.getOrElse(0), 1.0F) ) Pregel.iterate[(Int, Float), ED, Float](pagerankGraph)( (vertex, a: Float) => (vertex.data._1, (0.15F + 0.85F * a)), // apply (me_id, edge) => Some(edge.src.data._2 / edge.src.data._1), // gather (a: Float, b: Float) => a + b, // merge + 1.0F, numIter).mapVertices{ case Vertex(id, (outDeg, r)) => Vertex(id, r) } } diff --git a/graph/src/main/scala/spark/graph/Pregel.scala b/graph/src/main/scala/spark/graph/Pregel.scala index 851edfdd077ab..1011e3bf1277a 100644 --- a/graph/src/main/scala/spark/graph/Pregel.scala +++ b/graph/src/main/scala/spark/graph/Pregel.scala @@ -11,6 +11,7 @@ object Pregel { vprog: ( Vertex[VD], A) => VD, sendMsg: (Vid, EdgeWithVertices[VD, ED]) => Option[A], mergeMsg: (A, A) => A, + initialMsg: A, numIter: Int) : Graph[VD, ED] = { var graph = rawGraph.cache @@ -19,16 +20,17 @@ object Pregel { def reverseGather(vid: Vid, edge: EdgeWithVertices[VD,ED]) = sendMsg(edge.otherVertex(vid).id, edge) - while (i < numIter) { + var msgs: RDD[(Vid, A)] = graph.vertices.map{ v => (v.id, initialMsg) } - val msgs: RDD[(Vid, A)] = - graph.flatMapReduceNeighborhood(reverseGather, mergeMsg, EdgeDirection.In) + while (i < numIter) { def runProg(v: Vertex[VD], msg: Option[A]): VD = if(msg.isEmpty) v.data else vprog(v, msg.get) graph = graph.updateVertices(msgs, runProg).cache() + msgs = graph.flatMapReduceNeighborhood(reverseGather, mergeMsg, EdgeDirection.In) + i += 1 } graph From 9764e579b879ff6a483019418a5377fb07fc7cc5 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 5 Apr 2013 14:21:24 +0800 Subject: [PATCH 025/531] Minor cleanup. --- .../main/scala/spark/graph/Analytics.scala | 27 ---------------- graph/src/main/scala/spark/graph/Graph.scala | 31 ++++++------------- .../spark/graph/GraphKryoRegistrator.scala | 18 +++++------ .../src/main/scala/spark/graph/package.scala | 7 +++++ 4 files changed, 23 insertions(+), 60 deletions(-) diff --git a/graph/src/main/scala/spark/graph/Analytics.scala b/graph/src/main/scala/spark/graph/Analytics.scala index 5b0e5221ba946..5c16ba8175d36 100644 --- a/graph/src/main/scala/spark/graph/Analytics.scala +++ b/graph/src/main/scala/spark/graph/Analytics.scala @@ -2,25 +2,9 @@ package spark.graph import spark._ import spark.SparkContext._ -// import com.esotericsoftware.kryo._ // import breeze.linalg._ - - -// class AnalyticsKryoRegistrator extends KryoRegistrator { -// def registerClasses(kryo: Kryo) { -// println("registering kryo") -// kryo.register(classOf[(Int,Float,Float)]) -// Graph.kryoRegister[(Int,Float,Float), Float](kryo) -// Graph.kryoRegister[(Int,Float), Float](kryo) -// Graph.kryoRegister[Int, Float](kryo) -// Graph.kryoRegister[Float, Float](kryo) -// kryo.setReferences(false); -// } -// } - - object Analytics { @@ -111,14 +95,6 @@ object Analytics { gatherDirection = EdgeDirection.In) } - - - - - - - - // /** // * Compute the connected component membership of each vertex // * and return an RDD with the vertex value containing the @@ -250,9 +226,6 @@ object Analytics { // //System.setProperty("spark.shuffle.compress", "false") // System.setProperty("spark.kryo.registrator", "spark.graphlab.AnalyticsKryoRegistrator") - - - taskType match { case "pagerank" => { diff --git a/graph/src/main/scala/spark/graph/Graph.scala b/graph/src/main/scala/spark/graph/Graph.scala index 1b1929cc260c6..7b1111ae77b2c 100644 --- a/graph/src/main/scala/spark/graph/Graph.scala +++ b/graph/src/main/scala/spark/graph/Graph.scala @@ -43,12 +43,6 @@ class EdgeWithVertices[@specialized(Char, Int, Boolean, Byte, Long, Float, Doubl } -private[graph] -case class MutableTuple2[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) U, - @specialized(Char, Int, Boolean, Byte, Long, Float, Double) V]( - var _1: U, var _2: V) - - /** * A Graph RDD that supports computation on graphs. */ @@ -61,26 +55,23 @@ class Graph[VD: ClassManifest, ED: ClassManifest] protected ( _rawETable: RDD[(Pid, EdgePartition[ED])]) { def this(vertices: RDD[Vertex[VD]], edges: RDD[Edge[ED]]) = { - this( - Graph.DEFAULT_NUM_VERTEX_PARTITIONS, Graph.DEFAULT_NUM_EDGE_PARTITIONS, - vertices, edges, - null, null) + this(vertices.partitions.size, edges.partitions.size, vertices, edges, null, null) } def withPartitioner(numVertexPartitions: Int, numEdgePartitions: Int): Graph[VD, ED] = { if (_cached) { - val newgraph = new Graph(numVertexPartitions, numEdgePartitions, null, null, _rawVTable, _rawETable) - newgraph.cache() + (new Graph(numVertexPartitions, numEdgePartitions, null, null, _rawVTable, _rawETable)) + .cache() } else { new Graph(numVertexPartitions, numEdgePartitions, _rawVertices, _rawEdges, null, null) } } - def withVertexPartitioner(numVertexPartitions: Int = Graph.DEFAULT_NUM_VERTEX_PARTITIONS) = { + def withVertexPartitioner(numVertexPartitions: Int) = { withPartitioner(numVertexPartitions, numEdgePartitions) } - def withEdgePartitioner(numEdgePartitions: Int = Graph.DEFAULT_NUM_EDGE_PARTITIONS) = { + def withEdgePartitioner(numEdgePartitions: Int) = { withPartitioner(numVertexPartitions, numEdgePartitions) } @@ -139,11 +130,11 @@ class Graph[VD: ClassManifest, ED: ClassManifest] protected ( edgeDirection) } - def mapVertices[VD2: ClassManifest](f: (Vertex[VD]) => Vertex[VD2]): Graph[VD2, ED] = { + def mapVertices[VD2: ClassManifest](f: Vertex[VD] => Vertex[VD2]): Graph[VD2, ED] = { newGraph(vertices.map(f), edges) } - def mapEdges[ED2: ClassManifest](f: (Edge[ED]) => Edge[ED2]): Graph[VD, ED2] = { + def mapEdges[ED2: ClassManifest](f: Edge[ED] => Edge[ED2]): Graph[VD, ED2] = { newGraph(vertices, edges.map(f)) } @@ -237,7 +228,7 @@ class Graph[VD: ClassManifest, ED: ClassManifest] protected ( } } } - vmap.int2ObjectEntrySet().fastIterator().filter{!_.getValue()._2.isEmpty}.map{ entry => + vmap.int2ObjectEntrySet().fastIterator().filter(!_.getValue()._2.isEmpty).map{ entry => (entry.getIntKey(), entry.getValue()._2) } } @@ -316,14 +307,10 @@ class Graph[VD: ClassManifest, ED: ClassManifest] protected ( object Graph { - val DEFAULT_NUM_VERTEX_PARTITIONS = 5 - val DEFAULT_NUM_EDGE_PARTITIONS = 5 - /** * Load an edge list from file initializing the Graph RDD */ - def textFile[ED: ClassManifest](sc: SparkContext, - fname: String, edgeParser: Array[String] => ED ) = { + def textFile[ED: ClassManifest](sc: SparkContext, fname: String, edgeParser: Array[String] => ED) = { // Parse the edge data table val edges = sc.textFile(fname).map { line => diff --git a/graph/src/main/scala/spark/graph/GraphKryoRegistrator.scala b/graph/src/main/scala/spark/graph/GraphKryoRegistrator.scala index e72e500fb8fc0..e1cb77f1141ac 100644 --- a/graph/src/main/scala/spark/graph/GraphKryoRegistrator.scala +++ b/graph/src/main/scala/spark/graph/GraphKryoRegistrator.scala @@ -8,21 +8,17 @@ import spark.KryoRegistrator class GraphKryoRegistrator extends KryoRegistrator { def registerClasses(kryo: Kryo) { - kryo.register(classOf[(Int, Float, Float)]) - registerClass[(Int, Float, Float), Float](kryo) - registerClass[(Int, Float), Float](kryo) - registerClass[Int, Float](kryo) - registerClass[Float, Float](kryo) + //kryo.register(classOf[(Int, Float, Float)]) + registerClass[Int, Int, Int](kryo) // This avoids a large number of hash table lookups. kryo.setReferences(false) } - private def registerClass[VD: Manifest, ED: Manifest](kryo: Kryo) { - //kryo.register(classManifest[VD].erasure) - // kryo.register(classManifest[ED].erasure) - kryo.register(classOf[(Vid, Vid, ED)]) - kryo.register(classOf[(Vid, ED)]) - //kryo.register(classOf[EdgeBlockRecord[ED]]) + private def registerClass[VD: Manifest, ED: Manifest, VD2: Manifest](kryo: Kryo) { + kryo.register(classOf[Vertex[VD]]) + kryo.register(classOf[Edge[ED]]) + kryo.register(classOf[MutableTuple2[VD, VD2]]) + kryo.register(classOf[(Vid, VD2)]) } } diff --git a/graph/src/main/scala/spark/graph/package.scala b/graph/src/main/scala/spark/graph/package.scala index e7ec3f6e862fd..cf1b23ca5d4e1 100644 --- a/graph/src/main/scala/spark/graph/package.scala +++ b/graph/src/main/scala/spark/graph/package.scala @@ -11,4 +11,11 @@ package object graph { * Return the default null-like value for a data type T. */ def nullValue[T] = null.asInstanceOf[T] + + + private[graph] + case class MutableTuple2[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) U, + @specialized(Char, Int, Boolean, Byte, Long, Float, Double) V]( + var _1: U, var _2: V) + } From c973e564b92d18d756d9b31e5ef9ffb01d090e1b Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 5 Apr 2013 14:46:37 +0800 Subject: [PATCH 026/531] Changed EdgeDirection from Enumeration to case classes. --- .../scala/spark/graph/EdgeDirection.scala | 14 +++- graph/src/main/scala/spark/graph/Graph.scala | 13 ++-- .../src/main/scala/spark/graph/GraphLab.scala | 65 +++++++------------ 3 files changed, 42 insertions(+), 50 deletions(-) diff --git a/graph/src/main/scala/spark/graph/EdgeDirection.scala b/graph/src/main/scala/spark/graph/EdgeDirection.scala index a5bd9749bfbfe..a0c52c7038b3c 100644 --- a/graph/src/main/scala/spark/graph/EdgeDirection.scala +++ b/graph/src/main/scala/spark/graph/EdgeDirection.scala @@ -1,9 +1,17 @@ package spark.graph -object EdgeDirection extends Enumeration { +sealed abstract class EdgeDirection { + def reverse: EdgeDirection = this match { + case EdgeDirection.In => EdgeDirection.In + case EdgeDirection.Out => EdgeDirection.Out + case EdgeDirection.Both => EdgeDirection.Both + } +} - type EdgeDirection = Value - val None, In, Out, Both = Value +object EdgeDirection { + case object In extends EdgeDirection + case object Out extends EdgeDirection + case object Both extends EdgeDirection } diff --git a/graph/src/main/scala/spark/graph/Graph.scala b/graph/src/main/scala/spark/graph/Graph.scala index 7b1111ae77b2c..e46e755cee6e6 100644 --- a/graph/src/main/scala/spark/graph/Graph.scala +++ b/graph/src/main/scala/spark/graph/Graph.scala @@ -38,8 +38,9 @@ class EdgeWithVertices[@specialized(Char, Int, Boolean, Byte, Long, Float, Doubl def vertex(vid: Vid): Vertex[VD] = if (src.id == vid) src else dst - def relativeDirection(vid: Vid): EdgeDirection.EdgeDirection = - if(vid == src.id) EdgeDirection.Out else EdgeDirection.In + def relativeDirection(vid: Vid): EdgeDirection = { + if (vid == src.id) EdgeDirection.Out else EdgeDirection.In + } } @@ -122,7 +123,7 @@ class Graph[VD: ClassManifest, ED: ClassManifest] protected ( newGraph(vertices, edges.map{ case Edge(s, t, e) => Edge(t, s, e) }) } - def collectNeighborIds(edgeDirection: EdgeDirection.EdgeDirection) : RDD[(Vid, Array[Vid])] = { + def collectNeighborIds(edgeDirection: EdgeDirection) : RDD[(Vid, Array[Vid])] = { mapReduceNeighborhood[Array[Vid]]( (vid, edge) => Array(edge.otherVertex(vid).id), (a, b) => a ++ b, @@ -146,7 +147,7 @@ class Graph[VD: ClassManifest, ED: ClassManifest] protected ( mapFunc: (Vid, EdgeWithVertices[VD, ED]) => VD2, reduceFunc: (VD2, VD2) => VD2, default: VD2, - gatherDirection: EdgeDirection.EdgeDirection): RDD[(Vid, VD2)] = { + gatherDirection: EdgeDirection): RDD[(Vid, VD2)] = { ClosureCleaner.clean(mapFunc) ClosureCleaner.clean(reduceFunc) @@ -188,7 +189,7 @@ class Graph[VD: ClassManifest, ED: ClassManifest] protected ( def flatMapReduceNeighborhood[VD2: ClassManifest]( mapFunc: (Vid, EdgeWithVertices[VD, ED]) => Option[VD2], reduceFunc: (VD2, VD2) => VD2, - gatherDirection: EdgeDirection.EdgeDirection): RDD[(Vid, VD2)] = { + gatherDirection: EdgeDirection): RDD[(Vid, VD2)] = { ClosureCleaner.clean(mapFunc) ClosureCleaner.clean(reduceFunc) @@ -372,7 +373,7 @@ object Graph { (if(c+1 < numCols) List(Edge(vid, index(r,c+1), 1.0F)) else List.empty) } new Graph(vertices, edges) - } + } /** diff --git a/graph/src/main/scala/spark/graph/GraphLab.scala b/graph/src/main/scala/spark/graph/GraphLab.scala index 61e97094777a0..2a906d73d5a99 100644 --- a/graph/src/main/scala/spark/graph/GraphLab.scala +++ b/graph/src/main/scala/spark/graph/GraphLab.scala @@ -31,42 +31,38 @@ object GraphLab { // graph // } - - - - def iterateGA[VD: ClassManifest, ED: ClassManifest, A: ClassManifest](rawGraph: Graph[VD, ED])( - gather: (Vid, EdgeWithVertices[VD, ED]) => A, - merge: (A, A) => A, - apply: (Vertex[VD], Option[A]) => VD, + def iterateGA[VD: ClassManifest, ED: ClassManifest, A: ClassManifest](graph: Graph[VD, ED])( + gatherFunc: (Vid, EdgeWithVertices[VD, ED]) => A, + mergeFunc: (A, A) => A, + applyFunc: (Vertex[VD], Option[A]) => VD, numIter: Int, - gatherDirection: EdgeDirection.EdgeDirection = EdgeDirection.In) : Graph[VD, ED] = { + gatherDirection: EdgeDirection = EdgeDirection.In) : Graph[VD, ED] = { - var graph = rawGraph.cache() + var g = graph.cache() - def someGather(vid: Vid, edge: EdgeWithVertices[VD,ED]) = Some(gather(vid, edge)) + def someGather(vid: Vid, edge: EdgeWithVertices[VD, ED]) = Some(gatherFunc(vid, edge)) var i = 0 while (i < numIter) { val accUpdates: RDD[(Vid, A)] = - graph.flatMapReduceNeighborhood(someGather, merge, gatherDirection) - graph = graph.updateVertices(accUpdates, apply).cache() + g.flatMapReduceNeighborhood(someGather, mergeFunc, gatherDirection) + + g = g.updateVertices(accUpdates, applyFunc).cache() i += 1 } - graph + g } - - def iterateGAS[VD: ClassManifest, ED: ClassManifest, A: ClassManifest](rawGraph: Graph[VD, ED])( - rawGather: (Vid, EdgeWithVertices[VD, ED]) => A, - merge: (A, A) => A, - rawApply: (Vertex[VD], Option[A]) => VD, - rawScatter: (Vid, EdgeWithVertices[VD, ED]) => Boolean, + gatherFunc: (Vid, EdgeWithVertices[VD, ED]) => A, + mergeFunc: (A, A) => A, + applyFunc: (Vertex[VD], Option[A]) => VD, + scatterFunc: (Vid, EdgeWithVertices[VD, ED]) => Boolean, numIter: Int, - gatherDirection: EdgeDirection.EdgeDirection = EdgeDirection.In, - rawScatterDirection: EdgeDirection.EdgeDirection = EdgeDirection.Out) : Graph[VD, ED] = { + gatherDirection: EdgeDirection = EdgeDirection.In, + scatterDirection: EdgeDirection = EdgeDirection.Out) : Graph[VD, ED] = { var graph = rawGraph.mapVertices{ case Vertex(id,data) => Vertex(id, (true, data)) }.cache() @@ -75,14 +71,14 @@ object GraphLab { val edge = new EdgeWithVertices[VD,ED] edge.src = Vertex(e.src.id, e.src.data._2) edge.dst = Vertex(e.dst.id, e.dst.data._2) - Some(rawGather(vid, edge)) + Some(gatherFunc(vid, edge)) } else { None } } def apply(v: Vertex[(Boolean, VD)], accum: Option[A]) = { - if(v.data._1) (true, rawApply(Vertex(v.id, v.data._2), accum)) + if(v.data._1) (true, applyFunc(Vertex(v.id, v.data._2), accum)) else (false, v.data._2) } @@ -92,47 +88,34 @@ object GraphLab { val edge = new EdgeWithVertices[VD,ED] edge.src = Vertex(e.src.id, e.src.data._2) edge.dst = Vertex(e.dst.id, e.dst.data._2) - Some(rawScatter(vid, edge)) + Some(scatterFunc(vid, edge)) } else { None } } - // Scatter is basically a gather in the opposite direction so we reverse the edge direction - val scatterDirection = rawScatterDirection match { - case EdgeDirection.In => EdgeDirection.Out - case EdgeDirection.Out => EdgeDirection.In - case _ => rawScatterDirection - } - def applyActive(v: Vertex[(Boolean, VD)], accum: Option[Boolean]) = (accum.getOrElse(false), v.data._2) - - var i = 0 var numActive = graph.numVertices while (i < numIter && numActive > 0) { val accUpdates: RDD[(Vid, A)] = - graph.flatMapReduceNeighborhood(gather, merge, gatherDirection) + graph.flatMapReduceNeighborhood(gather, mergeFunc, gatherDirection) graph = graph.updateVertices(accUpdates, apply).cache() + // Scatter is basically a gather in the opposite direction so we reverse the edge direction val activeVertices: RDD[(Vid, Boolean)] = - graph.flatMapReduceNeighborhood(scatter, _ || _, scatterDirection) + graph.flatMapReduceNeighborhood(scatter, _ || _, scatterDirection.reverse) graph = graph.updateVertices(activeVertices, applyActive).cache() - numActive = graph.vertices.map(v => if(v.data._1) 1 else 0).reduce( _ + _ ) - + numActive = graph.vertices.map(v => if (v.data._1) 1 else 0).reduce( _ + _ ) println("Number active vertices: " + numActive) - i += 1 } graph.mapVertices(v => Vertex(v.id, v.data._2)) } - - - } From e5dd61e720520dabea6f0b4d84eb135175c17f6a Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 5 Apr 2013 14:47:39 +0800 Subject: [PATCH 027/531] Rename rawGraph to graph. --- .../src/main/scala/spark/graph/GraphLab.scala | 19 ++++++++++--------- 1 file changed, 10 insertions(+), 9 deletions(-) diff --git a/graph/src/main/scala/spark/graph/GraphLab.scala b/graph/src/main/scala/spark/graph/GraphLab.scala index 2a906d73d5a99..b0efdadce92c7 100644 --- a/graph/src/main/scala/spark/graph/GraphLab.scala +++ b/graph/src/main/scala/spark/graph/GraphLab.scala @@ -55,7 +55,7 @@ object GraphLab { g } - def iterateGAS[VD: ClassManifest, ED: ClassManifest, A: ClassManifest](rawGraph: Graph[VD, ED])( + def iterateGAS[VD: ClassManifest, ED: ClassManifest, A: ClassManifest](graph: Graph[VD, ED])( gatherFunc: (Vid, EdgeWithVertices[VD, ED]) => A, mergeFunc: (A, A) => A, applyFunc: (Vertex[VD], Option[A]) => VD, @@ -64,7 +64,7 @@ object GraphLab { gatherDirection: EdgeDirection = EdgeDirection.In, scatterDirection: EdgeDirection = EdgeDirection.Out) : Graph[VD, ED] = { - var graph = rawGraph.mapVertices{ case Vertex(id,data) => Vertex(id, (true, data)) }.cache() + var g = graph.mapVertices{ case Vertex(id,data) => Vertex(id, (true, data)) }.cache() def gather(vid: Vid, e: EdgeWithVertices[(Boolean, VD), ED]) = { if(e.vertex(vid).data._1) { @@ -98,24 +98,25 @@ object GraphLab { (accum.getOrElse(false), v.data._2) var i = 0 - var numActive = graph.numVertices + var numActive = g.numVertices while (i < numIter && numActive > 0) { val accUpdates: RDD[(Vid, A)] = - graph.flatMapReduceNeighborhood(gather, mergeFunc, gatherDirection) + g.flatMapReduceNeighborhood(gather, mergeFunc, gatherDirection) - graph = graph.updateVertices(accUpdates, apply).cache() + g = g.updateVertices(accUpdates, apply).cache() // Scatter is basically a gather in the opposite direction so we reverse the edge direction val activeVertices: RDD[(Vid, Boolean)] = - graph.flatMapReduceNeighborhood(scatter, _ || _, scatterDirection.reverse) + g.flatMapReduceNeighborhood(scatter, _ || _, scatterDirection.reverse) - graph = graph.updateVertices(activeVertices, applyActive).cache() + g = g.updateVertices(activeVertices, applyActive).cache() - numActive = graph.vertices.map(v => if (v.data._1) 1 else 0).reduce( _ + _ ) + numActive = g.vertices.map(v => if (v.data._1) 1 else 0).reduce( _ + _ ) println("Number active vertices: " + numActive) i += 1 } - graph.mapVertices(v => Vertex(v.id, v.data._2)) + + g.mapVertices(v => Vertex(v.id, v.data._2)) } } From 092708e57e47673636083f65a5b64700e15d361b Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Thu, 4 Apr 2013 23:50:53 -0700 Subject: [PATCH 028/531] better parsing of graph text files --- graph/src/main/scala/spark/graph/Graph.scala | 24 +++++++++++--------- 1 file changed, 13 insertions(+), 11 deletions(-) diff --git a/graph/src/main/scala/spark/graph/Graph.scala b/graph/src/main/scala/spark/graph/Graph.scala index 7b1111ae77b2c..4bd3cc73e3fc9 100644 --- a/graph/src/main/scala/spark/graph/Graph.scala +++ b/graph/src/main/scala/spark/graph/Graph.scala @@ -313,17 +313,19 @@ object Graph { def textFile[ED: ClassManifest](sc: SparkContext, fname: String, edgeParser: Array[String] => ED) = { // Parse the edge data table - val edges = sc.textFile(fname).map { line => - val lineArray = line.split("\\s+") - if(lineArray.length < 2) { - println("Invalid line: " + line) - assert(false) - } - val source = lineArray(0) - val target = lineArray(1) - val tail = lineArray.drop(2) - val edata = edgeParser(tail) - Edge(source.trim.toInt, target.trim.toInt, edata) + val edges = sc.textFile(fname).flatMap { line => + if(!line.isEmpty && line(0) != '#') { + val lineArray = line.split("\\s+") + if(lineArray.length < 2) { + println("Invalid line: " + line) + assert(false) + } + val source = lineArray(0) + val target = lineArray(1) + val tail = lineArray.drop(2) + val edata = edgeParser(tail) + Array(Edge(source.trim.toInt, target.trim.toInt, edata)) + } else { Array.empty[Edge[ED]] } }.cache() val graph = fromEdges(edges) From d40c1d51229070660cb0c2f0d8fd04954c9d8ab7 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 5 Apr 2013 15:33:50 +0800 Subject: [PATCH 029/531] Added unit test and fix a partitioner problem. --- .../spark/graph/EdgeWithVerticesRDD.scala | 3 ++ graph/src/main/scala/spark/graph/Graph.scala | 2 +- .../test/scala/spark/graph/GraphSuite.scala | 41 +++++++++++++++++ .../scala/spark/graph/LocalSparkContext.scala | 44 +++++++++++++++++++ 4 files changed, 89 insertions(+), 1 deletion(-) create mode 100644 graph/src/test/scala/spark/graph/GraphSuite.scala create mode 100644 graph/src/test/scala/spark/graph/LocalSparkContext.scala diff --git a/graph/src/main/scala/spark/graph/EdgeWithVerticesRDD.scala b/graph/src/main/scala/spark/graph/EdgeWithVerticesRDD.scala index c27ce586e9d75..91e4d0e01718f 100644 --- a/graph/src/main/scala/spark/graph/EdgeWithVerticesRDD.scala +++ b/graph/src/main/scala/spark/graph/EdgeWithVerticesRDD.scala @@ -23,6 +23,9 @@ class EdgeWithVerticesRDD[VD: ClassManifest, ED: ClassManifest]( eTable: RDD[(Pid, EdgePartition[ED])]) extends RDD[(VertexHashMap[VD], Iterator[EdgeWithVertices[VD, ED]])](eTable.context, Nil) { + println(vTableReplicated.partitioner.get.numPartitions) + println(eTable.partitioner.get.numPartitions) + assert(vTableReplicated.partitioner == eTable.partitioner) override def getDependencies: List[Dependency[_]] = { diff --git a/graph/src/main/scala/spark/graph/Graph.scala b/graph/src/main/scala/spark/graph/Graph.scala index c22982d862c2b..766ef1520613c 100644 --- a/graph/src/main/scala/spark/graph/Graph.scala +++ b/graph/src/main/scala/spark/graph/Graph.scala @@ -251,7 +251,7 @@ class Graph[VD: ClassManifest, ED: ClassManifest] protected ( } }, preservesPartitioning = true).cache() - new Graph(newVTable.partitioner.size, eTable.partitioner.size, null, null, newVTable, eTable) + new Graph(newVTable.partitions.size, eTable.partitions.size, null, null, newVTable, eTable) } // def mapPartitions[U: ClassManifest]( diff --git a/graph/src/test/scala/spark/graph/GraphSuite.scala b/graph/src/test/scala/spark/graph/GraphSuite.scala new file mode 100644 index 0000000000000..11b3dd1298d0c --- /dev/null +++ b/graph/src/test/scala/spark/graph/GraphSuite.scala @@ -0,0 +1,41 @@ +package spark.graph + +import org.scalatest.FunSuite + +import spark.SparkContext + + +class GraphSuite extends FunSuite with LocalSparkContext { + + test("graph partitioner") { + sc = new SparkContext("local", "test") + val vertices = sc.parallelize(Seq(Vertex(1, "one"), Vertex(2, "two"))) + val edges = sc.parallelize(Seq(Edge(1, 2, "onlyedge"))) + var g = new Graph(vertices, edges) + + g = g.withPartitioner(4, 7) + assert(g.numVertexPartitions === 4) + assert(g.numEdgePartitions === 7) + + g = g.withVertexPartitioner(5) + assert(g.numVertexPartitions === 5) + + g = g.withEdgePartitioner(8) + assert(g.numEdgePartitions === 8) + + g = g.mapVertices(x => x) + assert(g.numVertexPartitions === 5) + assert(g.numEdgePartitions === 8) + + g = g.mapEdges(x => x) + assert(g.numVertexPartitions === 5) + assert(g.numEdgePartitions === 8) + + val updates = sc.parallelize(Seq((1, " more"))) + g = g.updateVertices( + updates, + (v, u: Option[String]) => if (u.isDefined) v.data + u.get else v.data) + assert(g.numVertexPartitions === 5) + assert(g.numEdgePartitions === 8) + } +} diff --git a/graph/src/test/scala/spark/graph/LocalSparkContext.scala b/graph/src/test/scala/spark/graph/LocalSparkContext.scala new file mode 100644 index 0000000000000..cd160cbf0c377 --- /dev/null +++ b/graph/src/test/scala/spark/graph/LocalSparkContext.scala @@ -0,0 +1,44 @@ +package spark.graph + +import org.scalatest.Suite +import org.scalatest.BeforeAndAfterEach + +import spark.SparkContext + + +/** Manages a local `sc` {@link SparkContext} variable, correctly stopping it after each test. */ +trait LocalSparkContext extends BeforeAndAfterEach { self: Suite => + + @transient var sc: SparkContext = _ + + override def afterEach() { + resetSparkContext() + super.afterEach() + } + + def resetSparkContext() = { + if (sc != null) { + LocalSparkContext.stop(sc) + sc = null + } + } + +} + +object LocalSparkContext { + def stop(sc: SparkContext) { + sc.stop() + // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown + System.clearProperty("spark.driver.port") + } + + /** Runs `f` by passing in `sc` and ensures that `sc` is stopped. */ + def withSpark[T](sc: SparkContext)(f: SparkContext => T) = { + try { + f(sc) + } finally { + stop(sc) + } + } + +} \ No newline at end of file From 822d9c5b70f3ad8fad1bb23075d29246ff31a38d Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 5 Apr 2013 16:16:04 +0800 Subject: [PATCH 030/531] Rename rawGraph to graph in Pregel. --- graph/src/main/scala/spark/graph/Pregel.scala | 17 +++++++---------- .../src/test/scala/spark/graph/GraphSuite.scala | 5 +++++ 2 files changed, 12 insertions(+), 10 deletions(-) diff --git a/graph/src/main/scala/spark/graph/Pregel.scala b/graph/src/main/scala/spark/graph/Pregel.scala index 1011e3bf1277a..4bd881063495b 100644 --- a/graph/src/main/scala/spark/graph/Pregel.scala +++ b/graph/src/main/scala/spark/graph/Pregel.scala @@ -6,35 +6,32 @@ import spark.RDD object Pregel { - def iterate[VD: ClassManifest, ED: ClassManifest, A: ClassManifest]( - rawGraph: Graph[VD, ED])( + def iterate[VD: ClassManifest, ED: ClassManifest, A: ClassManifest](graph: Graph[VD, ED])( vprog: ( Vertex[VD], A) => VD, sendMsg: (Vid, EdgeWithVertices[VD, ED]) => Option[A], mergeMsg: (A, A) => A, initialMsg: A, numIter: Int) : Graph[VD, ED] = { - var graph = rawGraph.cache + var g = graph.cache var i = 0 def reverseGather(vid: Vid, edge: EdgeWithVertices[VD,ED]) = sendMsg(edge.otherVertex(vid).id, edge) - var msgs: RDD[(Vid, A)] = graph.vertices.map{ v => (v.id, initialMsg) } + var msgs: RDD[(Vid, A)] = g.vertices.map{ v => (v.id, initialMsg) } while (i < numIter) { - def runProg(v: Vertex[VD], msg: Option[A]): VD = - if(msg.isEmpty) v.data else vprog(v, msg.get) + def runProg(v: Vertex[VD], msg: Option[A]): VD = if(msg.isEmpty) v.data else vprog(v, msg.get) - graph = graph.updateVertices(msgs, runProg).cache() + g = g.updateVertices(msgs, runProg).cache() - msgs = graph.flatMapReduceNeighborhood(reverseGather, mergeMsg, EdgeDirection.In) + msgs = g.flatMapReduceNeighborhood(reverseGather, mergeMsg, EdgeDirection.In) i += 1 } - graph - + g } diff --git a/graph/src/test/scala/spark/graph/GraphSuite.scala b/graph/src/test/scala/spark/graph/GraphSuite.scala index 11b3dd1298d0c..64a7aa063b057 100644 --- a/graph/src/test/scala/spark/graph/GraphSuite.scala +++ b/graph/src/test/scala/spark/graph/GraphSuite.scala @@ -37,5 +37,10 @@ class GraphSuite extends FunSuite with LocalSparkContext { (v, u: Option[String]) => if (u.isDefined) v.data + u.get else v.data) assert(g.numVertexPartitions === 5) assert(g.numEdgePartitions === 8) + + g = g.reverse + assert(g.numVertexPartitions === 5) + assert(g.numEdgePartitions === 8) + } } From 9eec3178352e9c95a66329182694c8fb0b66642a Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 5 Apr 2013 23:22:55 +0800 Subject: [PATCH 031/531] Minor cleanup. --- .../main/scala/spark/graph/Analytics.scala | 19 ++++------------ graph/src/main/scala/spark/graph/Graph.scala | 22 +++++++++++++------ 2 files changed, 19 insertions(+), 22 deletions(-) diff --git a/graph/src/main/scala/spark/graph/Analytics.scala b/graph/src/main/scala/spark/graph/Analytics.scala index d7be886daf02a..8f2844f34ff9d 100644 --- a/graph/src/main/scala/spark/graph/Analytics.scala +++ b/graph/src/main/scala/spark/graph/Analytics.scala @@ -7,7 +7,6 @@ import spark.SparkContext._ object Analytics { - /** * Compute the PageRank of a graph returning the pagerank of each vertex as an RDD */ @@ -23,7 +22,6 @@ object Analytics { numIter).mapVertices{ case Vertex(id, (outDeg, r)) => Vertex(id, r) } } - /** * Compute the PageRank of a graph returning the pagerank of each vertex as an RDD */ @@ -40,7 +38,6 @@ object Analytics { numIter).mapVertices{ case Vertex(id, (outDeg, r)) => Vertex(id, r) } } - /** * Compute the PageRank of a graph returning the pagerank of each vertex as an RDD */ @@ -61,8 +58,6 @@ object Analytics { maxIter).mapVertices { case Vertex(vid, data) => Vertex(vid, data._2) } } - - /** * Compute the connected component membership of each vertex * and return an RDD with the vertex value containing the @@ -79,7 +74,6 @@ object Analytics { gatherDirection = EdgeDirection.Both) } - /** * Compute the shortest path to a set of markers */ @@ -210,8 +204,6 @@ object Analytics { // vertex => vertex.id < maxUser).vertices // } - - def main(args: Array[String]) = { val host = args(0) val taskType = args(1) @@ -223,9 +215,9 @@ object Analytics { } } - // System.setProperty("spark.serializer", "spark.KryoSerializer") - // //System.setProperty("spark.shuffle.compress", "false") - // System.setProperty("spark.kryo.registrator", "spark.graphlab.AnalyticsKryoRegistrator") + System.setProperty("spark.serializer", "spark.KryoSerializer") + //System.setProperty("spark.shuffle.compress", "false") + System.setProperty("spark.kryo.registrator", "spark.graph.GraphKryoRegistrator") taskType match { case "pagerank" => { @@ -268,7 +260,7 @@ object Analytics { // val pr = if(isDynamic) Analytics.dynamicPagerank(graph, tol, numIter) // else Analytics.pagerank(graph, numIter) println("Total rank: " + pr.vertices.map{ case Vertex(id,r) => r }.reduce(_+_) ) - if(!outFname.isEmpty) { + if (!outFname.isEmpty) { println("Saving pageranks of pages to " + outFname) pr.vertices.map{case Vertex(id, r) => id + "\t" + r}.saveAsTextFile(outFname) } @@ -408,8 +400,5 @@ object Analytics { println("Invalid task type.") } } - } - - } diff --git a/graph/src/main/scala/spark/graph/Graph.scala b/graph/src/main/scala/spark/graph/Graph.scala index 766ef1520613c..1db92217ded9f 100644 --- a/graph/src/main/scala/spark/graph/Graph.scala +++ b/graph/src/main/scala/spark/graph/Graph.scala @@ -311,11 +311,17 @@ object Graph { /** * Load an edge list from file initializing the Graph RDD */ - def textFile[ED: ClassManifest](sc: SparkContext, fname: String, edgeParser: Array[String] => ED) = { + def textFile[ED: ClassManifest]( + sc: SparkContext, + path: String, + edgeParser: Array[String] => ED, + minEdgePartitions: Int = 1, + minVertexPartitions: Int = 1) + : Graph[Int, ED] = { // Parse the edge data table - val edges = sc.textFile(fname).flatMap { line => - if(!line.isEmpty && line(0) != '#') { + val edges = sc.textFile(path).flatMap { line => + if (!line.isEmpty && line(0) != '#') { val lineArray = line.split("\\s+") if(lineArray.length < 2) { println("Invalid line: " + line) @@ -326,13 +332,15 @@ object Graph { val tail = lineArray.drop(2) val edata = edgeParser(tail) Array(Edge(source.trim.toInt, target.trim.toInt, edata)) - } else { Array.empty[Edge[ED]] } + } else { + Array.empty[Edge[ED]] + } }.cache() val graph = fromEdges(edges) - println("Loaded graph:" + - "\n\t#edges: " + graph.numEdges + - "\n\t#vertices: " + graph.numVertices) + // println("Loaded graph:" + + // "\n\t#edges: " + graph.numEdges + + // "\n\t#vertices: " + graph.numVertices) graph } From 3728e1bc408a5666ee0bc0af208b296abdc92427 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 7 Apr 2013 15:05:46 +0800 Subject: [PATCH 032/531] Code to run bagel vs graph experiments. --- .../spark/bagel/examples/PageRankUtils.scala | 6 +- .../main/scala/spark/graph/Analytics.scala | 16 ++++- .../main/scala/spark/graph/BagelTest.scala | 71 +++++++++++++++++++ .../src/main/scala/spark/graph/GraphLab.scala | 47 ++++++------ project/SparkBuild.scala | 2 +- 5 files changed, 112 insertions(+), 30 deletions(-) create mode 100644 graph/src/main/scala/spark/graph/BagelTest.scala diff --git a/bagel/src/main/scala/spark/bagel/examples/PageRankUtils.scala b/bagel/src/main/scala/spark/bagel/examples/PageRankUtils.scala index b97d786ed4d38..df63000150fa9 100644 --- a/bagel/src/main/scala/spark/bagel/examples/PageRankUtils.scala +++ b/bagel/src/main/scala/spark/bagel/examples/PageRankUtils.scala @@ -13,16 +13,16 @@ import java.io.{InputStream, OutputStream, DataInputStream, DataOutputStream} import com.esotericsoftware.kryo._ class PageRankUtils extends Serializable { - def computeWithCombiner(numVertices: Long, epsilon: Double)( + def computeWithCombiner(numVertices: Long, epsilon: Double, terminateSteps: Int = 10)( self: PRVertex, messageSum: Option[Double], superstep: Int ): (PRVertex, Array[PRMessage]) = { val newValue = messageSum match { case Some(msgSum) if msgSum != 0 => - 0.15 / numVertices + 0.85 * msgSum + 0.15 + 0.85 * msgSum case _ => self.value } - val terminate = superstep >= 10 + val terminate = superstep >= terminateSteps val outbox: Array[PRMessage] = if (!terminate) diff --git a/graph/src/main/scala/spark/graph/Analytics.scala b/graph/src/main/scala/spark/graph/Analytics.scala index 8f2844f34ff9d..4a7449ba4f63f 100644 --- a/graph/src/main/scala/spark/graph/Analytics.scala +++ b/graph/src/main/scala/spark/graph/Analytics.scala @@ -10,15 +10,27 @@ object Analytics { /** * Compute the PageRank of a graph returning the pagerank of each vertex as an RDD */ + // def pagerank[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], numIter: Int) = { + // // Compute the out degree of each vertex + // val pagerankGraph = graph.updateVertices[Int, (Int, Float)](graph.outDegrees, + // (vertex, deg) => (deg.getOrElse(0), 1.0F) + // ) + // GraphLab.iterateGA[(Int, Float), ED, Float](pagerankGraph)( + // (me_id, edge) => edge.src.data._2 / edge.src.data._1, // gather + // (a: Float, b: Float) => a + b, // merge + // (vertex, a: Option[Float]) => (vertex.data._1, (0.15F + 0.85F * a.getOrElse(0F))), // apply + // numIter).mapVertices{ case Vertex(id, (outDeg, r)) => Vertex(id, r) } + // } def pagerank[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], numIter: Int) = { // Compute the out degree of each vertex val pagerankGraph = graph.updateVertices[Int, (Int, Float)](graph.outDegrees, (vertex, deg) => (deg.getOrElse(0), 1.0F) ) - GraphLab.iterateGA[(Int, Float), ED, Float](pagerankGraph)( + GraphLab.iterateGA2[(Int, Float), ED, Float](pagerankGraph)( (me_id, edge) => edge.src.data._2 / edge.src.data._1, // gather (a: Float, b: Float) => a + b, // merge - (vertex, a: Option[Float]) => (vertex.data._1, (0.15F + 0.85F * a.getOrElse(0F))), // apply + 0.0F, // default + (vertex, a: Float) => (vertex.data._1, (0.15F + 0.85F * a)), // apply numIter).mapVertices{ case Vertex(id, (outDeg, r)) => Vertex(id, r) } } diff --git a/graph/src/main/scala/spark/graph/BagelTest.scala b/graph/src/main/scala/spark/graph/BagelTest.scala new file mode 100644 index 0000000000000..eee53bd6f6a6b --- /dev/null +++ b/graph/src/main/scala/spark/graph/BagelTest.scala @@ -0,0 +1,71 @@ +package spark.graph + +import spark._ +import spark.SparkContext._ +import spark.bagel.Bagel +import spark.bagel.examples._ + + +object BagelTest { + + def main(args: Array[String]) { + val host = args(0) + val taskType = args(1) + val fname = args(2) + val options = args.drop(3).map { arg => + arg.dropWhile(_ == '-').split('=') match { + case Array(opt, v) => (opt -> v) + case _ => throw new IllegalArgumentException("Invalid argument: " + arg) + } + } + + System.setProperty("spark.serializer", "spark.KryoSerializer") + //System.setProperty("spark.shuffle.compress", "false") + System.setProperty("spark.kryo.registrator", "spark.bagel.examples.PRKryoRegistrator") + + var numIter = Int.MaxValue + var isDynamic = false + var tol:Float = 0.001F + var outFname = "" + var numVPart = 4 + var numEPart = 4 + + options.foreach{ + case ("numIter", v) => numIter = v.toInt + case ("dynamic", v) => isDynamic = v.toBoolean + case ("tol", v) => tol = v.toFloat + case ("output", v) => outFname = v + case ("numVPart", v) => numVPart = v.toInt + case ("numEPart", v) => numEPart = v.toInt + case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) + } + + val sc = new SparkContext(host, "PageRank(" + fname + ")") + val g = Graph.textFile(sc, fname, a => 1.0F).withPartitioner(numVPart, numEPart).cache() + val startTime = System.currentTimeMillis + + val numVertices = g.vertices.count() + + val vertices = g.collectNeighborIds(EdgeDirection.Out).map { case (vid, neighbors) => + (vid.toString, new PRVertex(1.0, neighbors.map(_.toString))) + } + + // Do the computation + val epsilon = 0.01 / numVertices + val messages = sc.parallelize(Array[(String, PRMessage)]()) + val utils = new PageRankUtils + val result = + Bagel.run( + sc, vertices, messages, combiner = new PRCombiner(), + numPartitions = numVPart)( + utils.computeWithCombiner(numVertices, epsilon, numIter)) + + println("Total rank: " + result.map{ case (id, r) => r.value }.reduce(_+_) ) + if (!outFname.isEmpty) { + println("Saving pageranks of pages to " + outFname) + result.map{ case (id, r) => id + "\t" + r.value }.saveAsTextFile(outFname) + } + println("Runtime: " + ((System.currentTimeMillis - startTime)/1000.0) + " seconds") + sc.stop() + } +} diff --git a/graph/src/main/scala/spark/graph/GraphLab.scala b/graph/src/main/scala/spark/graph/GraphLab.scala index b0efdadce92c7..4de453663d0f6 100644 --- a/graph/src/main/scala/spark/graph/GraphLab.scala +++ b/graph/src/main/scala/spark/graph/GraphLab.scala @@ -6,30 +6,29 @@ import spark.RDD object GraphLab { - // def iterateGA[VD: ClassManifest, ED: ClassManifest, A: ClassManifest]( - // rawGraph: Graph[VD, ED])( - // gather: (Vid, EdgeWithVertices[VD, ED]) => A, - // merge: (A, A) => A, - // default: A, - // apply: (Vertex[VD], A) => VD, - // numIter: Int, - // gatherDirection: EdgeDirection.EdgeDirection = EdgeDirection.In) : Graph[VD, ED] = { - - // var graph = rawGraph.cache() - - // var i = 0 - // while (i < numIter) { - - // val accUpdates: RDD[(Vid, A)] = - // graph.mapReduceNeighborhood(gather, merge, default, gatherDirection) - - // def applyFunc(v: Vertex[VD], update: Option[A]): VD = { apply(v, update.get) } - // graph = graph.updateVertices(accUpdates, applyFunc).cache() - - // i += 1 - // } - // graph - // } + def iterateGA2[VD: ClassManifest, ED: ClassManifest, A: ClassManifest](graph: Graph[VD, ED])( + gather: (Vid, EdgeWithVertices[VD, ED]) => A, + merge: (A, A) => A, + default: A, + apply: (Vertex[VD], A) => VD, + numIter: Int, + gatherDirection: EdgeDirection = EdgeDirection.In) : Graph[VD, ED] = { + + var g = graph.cache() + + var i = 0 + while (i < numIter) { + + val accUpdates: RDD[(Vid, A)] = + g.mapReduceNeighborhood(gather, merge, default, gatherDirection) + + def applyFunc(v: Vertex[VD], update: Option[A]): VD = { apply(v, update.get) } + g = g.updateVertices(accUpdates, applyFunc).cache() + + i += 1 + } + g + } def iterateGA[VD: ClassManifest, ED: ClassManifest, A: ClassManifest](graph: Graph[VD, ED])( gatherFunc: (Vid, EdgeWithVertices[VD, ED]) => A, diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 6e6c72517acad..56610e4385718 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -27,7 +27,7 @@ object SparkBuild extends Build { lazy val bagel = Project("bagel", file("bagel"), settings = bagelSettings) dependsOn (core) - lazy val graph = Project("graph", file("graph"), settings = graphSettings) dependsOn (core) + lazy val graph = Project("graph", file("graph"), settings = graphSettings) dependsOn (core, bagel) lazy val streaming = Project("streaming", file("streaming"), settings = streamingSettings) dependsOn (core) From 583a5858e7ad12760a8f4f96fb288c88aa7a9e84 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 7 Apr 2013 15:45:28 +0800 Subject: [PATCH 033/531] Added more logging to Analytics. --- graph/src/main/scala/spark/graph/Analytics.scala | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/graph/src/main/scala/spark/graph/Analytics.scala b/graph/src/main/scala/spark/graph/Analytics.scala index 4a7449ba4f63f..d18630cde0537 100644 --- a/graph/src/main/scala/spark/graph/Analytics.scala +++ b/graph/src/main/scala/spark/graph/Analytics.scala @@ -5,7 +5,7 @@ import spark.SparkContext._ // import breeze.linalg._ -object Analytics { +object Analytics extends Logging { /** * Compute the PageRank of a graph returning the pagerank of each vertex as an RDD @@ -265,18 +265,23 @@ object Analytics { println("======================================") val sc = new SparkContext(host, "PageRank(" + fname + ")") - val graph = Graph.textFile(sc, fname, a => 1.0F).withPartitioner(numVPart, numEPart) + + val graph = Graph.textFile(sc, fname, a => 1.0F).withPartitioner(numVPart, numEPart).cache() + val startTime = System.currentTimeMillis + logInfo("GRAPHX: starting tasks") + logInfo("GRAPHX: Number of vertices " + graph.vertices.count) + logInfo("GRAPHX: Number of edges " + graph.edges.count) val pr = Analytics.pagerank(graph, numIter) // val pr = if(isDynamic) Analytics.dynamicPagerank(graph, tol, numIter) // else Analytics.pagerank(graph, numIter) - println("Total rank: " + pr.vertices.map{ case Vertex(id,r) => r }.reduce(_+_) ) + logInfo("GRAPHX: Total rank: " + pr.vertices.map{ case Vertex(id,r) => r }.reduce(_+_) ) if (!outFname.isEmpty) { println("Saving pageranks of pages to " + outFname) pr.vertices.map{case Vertex(id, r) => id + "\t" + r}.saveAsTextFile(outFname) } - println("Runtime: " + ((System.currentTimeMillis - startTime)/1000.0) + " seconds") + logInfo("GRAPHX: Runtime: " + ((System.currentTimeMillis - startTime)/1000.0) + " seconds") sc.stop() } From 2635416cee1e53117edd47e73948fe1c352a7037 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Tue, 16 Apr 2013 11:27:56 -0700 Subject: [PATCH 034/531] switching from floats to doubles in pagerank and sssp --- .../main/scala/spark/graph/Analytics.scala | 64 +++++++++---------- 1 file changed, 32 insertions(+), 32 deletions(-) diff --git a/graph/src/main/scala/spark/graph/Analytics.scala b/graph/src/main/scala/spark/graph/Analytics.scala index 8f2844f34ff9d..809a7b120669a 100644 --- a/graph/src/main/scala/spark/graph/Analytics.scala +++ b/graph/src/main/scala/spark/graph/Analytics.scala @@ -12,13 +12,13 @@ object Analytics { */ def pagerank[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], numIter: Int) = { // Compute the out degree of each vertex - val pagerankGraph = graph.updateVertices[Int, (Int, Float)](graph.outDegrees, - (vertex, deg) => (deg.getOrElse(0), 1.0F) + val pagerankGraph = graph.updateVertices[Int, (Int, Double)](graph.outDegrees, + (vertex, deg) => (deg.getOrElse(0), 1.0) ) - GraphLab.iterateGA[(Int, Float), ED, Float](pagerankGraph)( + GraphLab.iterateGA[(Int, Double), ED, Double](pagerankGraph)( (me_id, edge) => edge.src.data._2 / edge.src.data._1, // gather - (a: Float, b: Float) => a + b, // merge - (vertex, a: Option[Float]) => (vertex.data._1, (0.15F + 0.85F * a.getOrElse(0F))), // apply + (a: Double, b: Double) => a + b, // merge + (vertex, a: Option[Double]) => (vertex.data._1, (0.15 + 0.85 * a.getOrElse(0.0))), // apply numIter).mapVertices{ case Vertex(id, (outDeg, r)) => Vertex(id, r) } } @@ -27,14 +27,14 @@ object Analytics { */ def pregelPagerank[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], numIter: Int) = { // Compute the out degree of each vertex - val pagerankGraph = graph.updateVertices[Int, (Int, Float)](graph.outDegrees, - (vertex, deg) => (deg.getOrElse(0), 1.0F) + val pagerankGraph = graph.updateVertices[Int, (Int, Double)](graph.outDegrees, + (vertex, deg) => (deg.getOrElse(0), 1.0) ) - Pregel.iterate[(Int, Float), ED, Float](pagerankGraph)( - (vertex, a: Float) => (vertex.data._1, (0.15F + 0.85F * a)), // apply + Pregel.iterate[(Int, Double), ED, Double](pagerankGraph)( + (vertex, a: Double) => (vertex.data._1, (0.15 + 0.85 * a)), // apply (me_id, edge) => Some(edge.src.data._2 / edge.src.data._1), // gather - (a: Float, b: Float) => a + b, // merge - 1.0F, + (a: Double, b: Double) => a + b, // merge + 1.0, numIter).mapVertices{ case Vertex(id, (outDeg, r)) => Vertex(id, r) } } @@ -42,18 +42,18 @@ object Analytics { * Compute the PageRank of a graph returning the pagerank of each vertex as an RDD */ def dynamicPagerank[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], - tol: Float, maxIter: Int = 10) = { + tol: Double, maxIter: Int = 10) = { // Compute the out degree of each vertex - val pagerankGraph = graph.updateVertices[Int, (Int, Float, Float)](graph.outDegrees, - (vertex, degIter) => (degIter.sum, 1.0F, 1.0F) + val pagerankGraph = graph.updateVertices[Int, (Int, Double, Double)](graph.outDegrees, + (vertex, degIter) => (degIter.sum, 1.0, 1.0) ) // Run PageRank GraphLab.iterateGAS(pagerankGraph)( (me_id, edge) => edge.src.data._2 / edge.src.data._1, // gather - (a: Float, b: Float) => a + b, - (vertex, a: Option[Float]) => - (vertex.data._1, (0.15F + 0.85F * a.getOrElse(0F)), vertex.data._2), // apply + (a: Double, b: Double) => a + b, + (vertex, a: Option[Double]) => + (vertex.data._1, (0.15 + 0.85 * a.getOrElse(0.0)), vertex.data._2), // apply (me_id, edge) => math.abs(edge.src.data._2 - edge.dst.data._1) > tol, // scatter maxIter).mapVertices { case Vertex(vid, data) => Vertex(vid, data._2) } } @@ -77,15 +77,15 @@ object Analytics { /** * Compute the shortest path to a set of markers */ - def shortestPath[VD: Manifest](graph: Graph[VD, Float], sources: List[Int], numIter: Int) = { + def shortestPath[VD: Manifest](graph: Graph[VD, Double], sources: List[Int], numIter: Int) = { val sourceSet = sources.toSet val spGraph = graph.mapVertices { - case Vertex(vid, _) => Vertex(vid, (if(sourceSet.contains(vid)) 0.0F else Float.MaxValue)) + case Vertex(vid, _) => Vertex(vid, (if(sourceSet.contains(vid)) 0.0 else Double.MaxValue)) } - GraphLab.iterateGA[Float, Float, Float](spGraph)( + GraphLab.iterateGA[Double, Double, Double](spGraph)( (me_id, edge) => edge.otherVertex(me_id).data + edge.data, // gather - (a: Float, b: Float) => math.min(a, b), // merge - (v, a: Option[Float]) => math.min(v.data, a.getOrElse(Float.MaxValue)), // apply + (a: Double, b: Double) => math.min(a, b), // merge + (v, a: Option[Double]) => math.min(v.data, a.getOrElse(Double.MaxValue)), // apply numIter, gatherDirection = EdgeDirection.In) } @@ -123,12 +123,12 @@ object Analytics { // /** // * Compute the shortest path to a set of markers // */ - // def dynamicShortestPath[VD: Manifest, ED: Manifest](graph: Graph[VD, Float], + // def dynamicShortestPath[VD: Manifest, ED: Manifest](graph: Graph[VD, Double], // sources: List[Int], numIter: Int) = { // val sourceSet = sources.toSet // val vertices = graph.vertices.mapPartitions( // iter => iter.map { - // case (vid, _) => (vid, (if(sourceSet.contains(vid)) 0.0F else Float.MaxValue) ) + // case (vid, _) => (vid, (if(sourceSet.contains(vid)) 0.0F else Double.MaxValue) ) // }); // val edges = graph.edges // .mapValues(v => None) @@ -137,9 +137,9 @@ object Analytics { // val niterations = Int.MaxValue // spGraph.iterateDynamic( // (me_id, edge) => edge.otherVertex(me_id).data + edge.data, // gather - // (a: Float, b: Float) => math.min(a, b), // merge - // Float.MaxValue, - // (v, a: Float) => math.min(v.data, a), // apply + // (a: Double, b: Double) => math.min(a, b), // merge + // Double.MaxValue, + // (v, a: Double) => math.min(v.data, a), // apply // (me_id, edge) => edge.vertex(me_id).data + edge.data < edge.otherVertex(me_id).data, // scatter // numIter, // gatherEdges = EdgeDirection.In, @@ -224,7 +224,7 @@ object Analytics { var numIter = Int.MaxValue var isDynamic = false - var tol:Float = 0.001F + var tol:Double = 0.001 var outFname = "" var numVPart = 4 var numEPart = 4 @@ -232,7 +232,7 @@ object Analytics { options.foreach{ case ("numIter", v) => numIter = v.toInt case ("dynamic", v) => isDynamic = v.toBoolean - case ("tol", v) => tol = v.toFloat + case ("tol", v) => tol = v.toDouble case ("output", v) => outFname = v case ("numVPart", v) => numVPart = v.toInt case ("numEPart", v) => numEPart = v.toInt @@ -253,7 +253,7 @@ object Analytics { println("======================================") val sc = new SparkContext(host, "PageRank(" + fname + ")") - val graph = Graph.textFile(sc, fname, a => 1.0F).withPartitioner(numVPart, numEPart) + val graph = Graph.textFile(sc, fname, a => 1.0).withPartitioner(numVPart, numEPart) val startTime = System.currentTimeMillis val pr = Analytics.pagerank(graph, numIter) @@ -292,7 +292,7 @@ object Analytics { println("======================================") val sc = new SparkContext(host, "ConnectedComponents(" + fname + ")") - val graph = Graph.textFile(sc, fname, a => 1.0F) + val graph = Graph.textFile(sc, fname, a => 1.0) val cc = Analytics.connectedComponents(graph, numIter) // val cc = if(isDynamic) Analytics.dynamicConnectedComponents(graph, numIter) // else Analytics.connectedComponents(graph, numIter) @@ -335,7 +335,7 @@ object Analytics { println("======================================") val sc = new SparkContext(host, "ShortestPath(" + fname + ")") - val graph = Graph.textFile(sc, fname, a => (if(a.isEmpty) 1.0F else a(0).toFloat ) ) + val graph = Graph.textFile(sc, fname, a => (if(a.isEmpty) 1.0 else a(0).toDouble ) ) val sp = Analytics.shortestPath(graph, sources, numIter) // val cc = if(isDynamic) Analytics.dynamicShortestPath(graph, sources, numIter) // else Analytics.shortestPath(graph, sources, numIter) From 70ba4d1740e4816aaf5a6134db613b34f03c2f75 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 5 May 2013 17:12:30 -0700 Subject: [PATCH 035/531] Refactored the Graph API for discussion. --- .../main/scala/spark/graph/Analytics.scala | 788 +++++++++--------- graph/src/main/scala/spark/graph/Edge.scala | 7 + .../scala/spark/graph/EdgeWithVertices.scala | 17 + graph/src/main/scala/spark/graph/Graph.scala | 488 +---------- .../src/main/scala/spark/graph/GraphLab.scala | 224 ++--- .../main/scala/spark/graph/GraphLoader.scala | 54 ++ .../src/main/scala/spark/graph/GraphOps.scala | 30 + graph/src/main/scala/spark/graph/Pregel.scala | 33 +- graph/src/main/scala/spark/graph/Timer.scala | 14 - graph/src/main/scala/spark/graph/Vertex.scala | 11 + .../spark/graph/impl/EdgePartition.scala | 49 ++ .../{ => impl}/EdgeWithVerticesRDD.scala | 4 +- .../scala/spark/graph/impl/GraphImpl.scala | 343 ++++++++ .../spark/graph/{ => perf}/BagelTest.scala | 5 +- .../scala/spark/graph/perf/SparkTest.scala | 72 ++ 15 files changed, 1131 insertions(+), 1008 deletions(-) create mode 100644 graph/src/main/scala/spark/graph/Edge.scala create mode 100644 graph/src/main/scala/spark/graph/EdgeWithVertices.scala create mode 100644 graph/src/main/scala/spark/graph/GraphLoader.scala create mode 100644 graph/src/main/scala/spark/graph/GraphOps.scala delete mode 100644 graph/src/main/scala/spark/graph/Timer.scala create mode 100644 graph/src/main/scala/spark/graph/Vertex.scala create mode 100644 graph/src/main/scala/spark/graph/impl/EdgePartition.scala rename graph/src/main/scala/spark/graph/{ => impl}/EdgeWithVerticesRDD.scala (97%) create mode 100644 graph/src/main/scala/spark/graph/impl/GraphImpl.scala rename graph/src/main/scala/spark/graph/{ => perf}/BagelTest.scala (93%) create mode 100644 graph/src/main/scala/spark/graph/perf/SparkTest.scala diff --git a/graph/src/main/scala/spark/graph/Analytics.scala b/graph/src/main/scala/spark/graph/Analytics.scala index d18630cde0537..e6ad2d05cfaa3 100644 --- a/graph/src/main/scala/spark/graph/Analytics.scala +++ b/graph/src/main/scala/spark/graph/Analytics.scala @@ -7,32 +7,32 @@ import spark.SparkContext._ object Analytics extends Logging { - /** - * Compute the PageRank of a graph returning the pagerank of each vertex as an RDD - */ - // def pagerank[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], numIter: Int) = { - // // Compute the out degree of each vertex - // val pagerankGraph = graph.updateVertices[Int, (Int, Float)](graph.outDegrees, - // (vertex, deg) => (deg.getOrElse(0), 1.0F) - // ) - // GraphLab.iterateGA[(Int, Float), ED, Float](pagerankGraph)( - // (me_id, edge) => edge.src.data._2 / edge.src.data._1, // gather - // (a: Float, b: Float) => a + b, // merge - // (vertex, a: Option[Float]) => (vertex.data._1, (0.15F + 0.85F * a.getOrElse(0F))), // apply - // numIter).mapVertices{ case Vertex(id, (outDeg, r)) => Vertex(id, r) } - // } - def pagerank[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], numIter: Int) = { - // Compute the out degree of each vertex - val pagerankGraph = graph.updateVertices[Int, (Int, Float)](graph.outDegrees, - (vertex, deg) => (deg.getOrElse(0), 1.0F) - ) - GraphLab.iterateGA2[(Int, Float), ED, Float](pagerankGraph)( - (me_id, edge) => edge.src.data._2 / edge.src.data._1, // gather - (a: Float, b: Float) => a + b, // merge - 0.0F, // default - (vertex, a: Float) => (vertex.data._1, (0.15F + 0.85F * a)), // apply - numIter).mapVertices{ case Vertex(id, (outDeg, r)) => Vertex(id, r) } - } +// /** +// * Compute the PageRank of a graph returning the pagerank of each vertex as an RDD +// */ +// // def pagerank[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], numIter: Int) = { +// // // Compute the out degree of each vertex +// // val pagerankGraph = graph.updateVertices[Int, (Int, Float)](graph.outDegrees, +// // (vertex, deg) => (deg.getOrElse(0), 1.0F) +// // ) +// // GraphLab.iterateGA[(Int, Float), ED, Float](pagerankGraph)( +// // (me_id, edge) => edge.src.data._2 / edge.src.data._1, // gather +// // (a: Float, b: Float) => a + b, // merge +// // (vertex, a: Option[Float]) => (vertex.data._1, (0.15F + 0.85F * a.getOrElse(0F))), // apply +// // numIter).mapVertices{ case Vertex(id, (outDeg, r)) => Vertex(id, r) } +// // } +// def pagerank[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], numIter: Int) = { +// // Compute the out degree of each vertex +// val pagerankGraph = graph.updateVertices[Int, (Int, Float)](graph.outDegrees, +// (vertex, deg) => (deg.getOrElse(0), 1.0F) +// ) +// GraphLab.iterateGA2[(Int, Float), ED, Float](pagerankGraph)( +// (me_id, edge) => edge.src.data._2 / edge.src.data._1, // gather +// (a: Float, b: Float) => a + b, // merge +// 0.0F, // default +// (vertex, a: Float) => (vertex.data._1, (0.15F + 0.85F * a)), // apply +// numIter).mapVertices{ case Vertex(id, (outDeg, r)) => Vertex(id, r) } +// } /** * Compute the PageRank of a graph returning the pagerank of each vertex as an RDD @@ -50,372 +50,372 @@ object Analytics extends Logging { numIter).mapVertices{ case Vertex(id, (outDeg, r)) => Vertex(id, r) } } - /** - * Compute the PageRank of a graph returning the pagerank of each vertex as an RDD - */ - def dynamicPagerank[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], - tol: Float, maxIter: Int = 10) = { - // Compute the out degree of each vertex - val pagerankGraph = graph.updateVertices[Int, (Int, Float, Float)](graph.outDegrees, - (vertex, degIter) => (degIter.sum, 1.0F, 1.0F) - ) - - // Run PageRank - GraphLab.iterateGAS(pagerankGraph)( - (me_id, edge) => edge.src.data._2 / edge.src.data._1, // gather - (a: Float, b: Float) => a + b, - (vertex, a: Option[Float]) => - (vertex.data._1, (0.15F + 0.85F * a.getOrElse(0F)), vertex.data._2), // apply - (me_id, edge) => math.abs(edge.src.data._2 - edge.dst.data._1) > tol, // scatter - maxIter).mapVertices { case Vertex(vid, data) => Vertex(vid, data._2) } - } - - /** - * Compute the connected component membership of each vertex - * and return an RDD with the vertex value containing the - * lowest vertex id in the connected component containing - * that vertex. - */ - def connectedComponents[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], numIter: Int) = { - val ccGraph = graph.mapVertices { case Vertex(vid, _) => Vertex(vid, vid) } - GraphLab.iterateGA[Int, ED, Int](ccGraph)( - (me_id, edge) => edge.otherVertex(me_id).data, // gather - (a: Int, b: Int) => math.min(a, b), // merge - (v, a: Option[Int]) => math.min(v.data, a.getOrElse(Integer.MAX_VALUE)), // apply - numIter, - gatherDirection = EdgeDirection.Both) - } - - /** - * Compute the shortest path to a set of markers - */ - def shortestPath[VD: Manifest](graph: Graph[VD, Float], sources: List[Int], numIter: Int) = { - val sourceSet = sources.toSet - val spGraph = graph.mapVertices { - case Vertex(vid, _) => Vertex(vid, (if(sourceSet.contains(vid)) 0.0F else Float.MaxValue)) - } - GraphLab.iterateGA[Float, Float, Float](spGraph)( - (me_id, edge) => edge.otherVertex(me_id).data + edge.data, // gather - (a: Float, b: Float) => math.min(a, b), // merge - (v, a: Option[Float]) => math.min(v.data, a.getOrElse(Float.MaxValue)), // apply - numIter, - gatherDirection = EdgeDirection.In) - } - - // /** - // * Compute the connected component membership of each vertex - // * and return an RDD with the vertex value containing the - // * lowest vertex id in the connected component containing - // * that vertex. - // */ - // def dynamicConnectedComponents[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], - // numIter: Int = Int.MaxValue) = { - - // val vertices = graph.vertices.mapPartitions(iter => iter.map { case (vid, _) => (vid, vid) }) - // val edges = graph.edges // .mapValues(v => None) - // val ccGraph = new Graph(vertices, edges) - - // ccGraph.iterateDynamic( - // (me_id, edge) => edge.otherVertex(me_id).data, // gather - // (a: Int, b: Int) => math.min(a, b), // merge - // Integer.MAX_VALUE, - // (v, a: Int) => math.min(v.data, a), // apply - // (me_id, edge) => edge.otherVertex(me_id).data > edge.vertex(me_id).data, // scatter - // numIter, - // gatherEdges = EdgeDirection.Both, - // scatterEdges = EdgeDirection.Both).vertices - // // - // // graph_ret.vertices.collect.foreach(println) - // // graph_ret.edges.take(10).foreach(println) - // } - - - - - // /** - // * Compute the shortest path to a set of markers - // */ - // def dynamicShortestPath[VD: Manifest, ED: Manifest](graph: Graph[VD, Float], - // sources: List[Int], numIter: Int) = { - // val sourceSet = sources.toSet - // val vertices = graph.vertices.mapPartitions( - // iter => iter.map { - // case (vid, _) => (vid, (if(sourceSet.contains(vid)) 0.0F else Float.MaxValue) ) - // }); - - // val edges = graph.edges // .mapValues(v => None) - // val spGraph = new Graph(vertices, edges) - - // val niterations = Int.MaxValue - // spGraph.iterateDynamic( - // (me_id, edge) => edge.otherVertex(me_id).data + edge.data, // gather - // (a: Float, b: Float) => math.min(a, b), // merge - // Float.MaxValue, - // (v, a: Float) => math.min(v.data, a), // apply - // (me_id, edge) => edge.vertex(me_id).data + edge.data < edge.otherVertex(me_id).data, // scatter - // numIter, - // gatherEdges = EdgeDirection.In, - // scatterEdges = EdgeDirection.Out).vertices - // } - - - - - // /** - // * - // */ - // def alternatingLeastSquares[VD: ClassManifest, ED: ClassManifest](graph: Graph[VD, Double], - // latentK: Int, lambda: Double, numIter: Int) = { - // val vertices = graph.vertices.mapPartitions( _.map { - // case (vid, _) => (vid, Array.fill(latentK){ scala.util.Random.nextDouble() } ) - // }).cache - // val maxUser = graph.edges.map(_._1).reduce(math.max(_,_)) - // val edges = graph.edges // .mapValues(v => None) - // val alsGraph = new Graph(vertices, edges) - // alsGraph.numVPart = graph.numVPart - // alsGraph.numEPart = graph.numEPart - - // val niterations = Int.MaxValue - // alsGraph.iterateDynamic[(Array[Double], Array[Double])]( - // (me_id, edge) => { // gather - // val X = edge.otherVertex(me_id).data - // val y = edge.data - // val Xy = X.map(_ * y) - // val XtX = (for(i <- 0 until latentK; j <- i until latentK) yield(X(i) * X(j))).toArray - // (Xy, XtX) - // }, - // (a, b) => { - // // The difference between the while loop and the zip is a FACTOR OF TWO in overall - // // runtime - // var i = 0 - // while(i < a._1.length) { a._1(i) += b._1(i); i += 1 } - // i = 0 - // while(i < a._2.length) { a._2(i) += b._2(i); i += 1 } - // a - // // (a._1.zip(b._1).map{ case (q,r) => q+r }, a._2.zip(b._2).map{ case (q,r) => q+r }) - // }, - // (Array.empty[Double], Array.empty[Double]), // default value is empty - // (vertex, accum) => { // apply - // val XyArray = accum._1 - // val XtXArray = accum._2 - // if(XyArray.isEmpty) vertex.data // no neighbors - // else { - // val XtX = DenseMatrix.tabulate(latentK,latentK){ (i,j) => - // (if(i < j) XtXArray(i + (j+1)*j/2) else XtXArray(i + (j+1)*j/2)) + - // (if(i == j) lambda else 1.0F) //regularization - // } - // val Xy = DenseMatrix.create(latentK,1,XyArray) - // val w = XtX \ Xy - // w.data - // } - // }, - // (me_id, edge) => true, - // numIter, - // gatherEdges = EdgeDirection.Both, - // scatterEdges = EdgeDirection.Both, - // vertex => vertex.id < maxUser).vertices - // } - - def main(args: Array[String]) = { - val host = args(0) - val taskType = args(1) - val fname = args(2) - val options = args.drop(3).map { arg => - arg.dropWhile(_ == '-').split('=') match { - case Array(opt, v) => (opt -> v) - case _ => throw new IllegalArgumentException("Invalid argument: " + arg) - } - } - - System.setProperty("spark.serializer", "spark.KryoSerializer") - //System.setProperty("spark.shuffle.compress", "false") - System.setProperty("spark.kryo.registrator", "spark.graph.GraphKryoRegistrator") - - taskType match { - case "pagerank" => { - - var numIter = Int.MaxValue - var isDynamic = false - var tol:Float = 0.001F - var outFname = "" - var numVPart = 4 - var numEPart = 4 - - options.foreach{ - case ("numIter", v) => numIter = v.toInt - case ("dynamic", v) => isDynamic = v.toBoolean - case ("tol", v) => tol = v.toFloat - case ("output", v) => outFname = v - case ("numVPart", v) => numVPart = v.toInt - case ("numEPart", v) => numEPart = v.toInt - case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) - } - - if(!isDynamic && numIter == Int.MaxValue) { - println("Set number of iterations!") - sys.exit(1) - } - println("======================================") - println("| PageRank |") - println("--------------------------------------") - println(" Using parameters:") - println(" \tDynamic: " + isDynamic) - if(isDynamic) println(" \t |-> Tolerance: " + tol) - println(" \tNumIter: " + numIter) - println("======================================") - - val sc = new SparkContext(host, "PageRank(" + fname + ")") - - val graph = Graph.textFile(sc, fname, a => 1.0F).withPartitioner(numVPart, numEPart).cache() - - val startTime = System.currentTimeMillis - logInfo("GRAPHX: starting tasks") - logInfo("GRAPHX: Number of vertices " + graph.vertices.count) - logInfo("GRAPHX: Number of edges " + graph.edges.count) - - val pr = Analytics.pagerank(graph, numIter) - // val pr = if(isDynamic) Analytics.dynamicPagerank(graph, tol, numIter) - // else Analytics.pagerank(graph, numIter) - logInfo("GRAPHX: Total rank: " + pr.vertices.map{ case Vertex(id,r) => r }.reduce(_+_) ) - if (!outFname.isEmpty) { - println("Saving pageranks of pages to " + outFname) - pr.vertices.map{case Vertex(id, r) => id + "\t" + r}.saveAsTextFile(outFname) - } - logInfo("GRAPHX: Runtime: " + ((System.currentTimeMillis - startTime)/1000.0) + " seconds") - sc.stop() - } - - case "cc" => { - - var numIter = Int.MaxValue - var isDynamic = false - - options.foreach{ - case ("numIter", v) => numIter = v.toInt - case ("dynamic", v) => isDynamic = v.toBoolean - case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) - } - - 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 + ")") - val graph = Graph.textFile(sc, fname, a => 1.0F) - val cc = Analytics.connectedComponents(graph, numIter) - // val cc = if(isDynamic) Analytics.dynamicConnectedComponents(graph, numIter) - // else Analytics.connectedComponents(graph, numIter) - println("Components: " + cc.vertices.map(_.data).distinct()) - - sc.stop() - } - - case "shortestpath" => { - - var numIter = Int.MaxValue - var isDynamic = true - var sources: List[Int] = List.empty - - options.foreach{ - case ("numIter", v) => numIter = v.toInt - case ("dynamic", v) => isDynamic = v.toBoolean - case ("source", v) => sources ++= List(v.toInt) - case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) - } - - - if(!isDynamic && numIter == Int.MaxValue) { - println("Set number of iterations!") - sys.exit(1) - } - - if(sources.isEmpty) { - println("No sources provided!") - sys.exit(1) - } - - println("======================================") - println("| Shortest Path |") - println("--------------------------------------") - println(" Using parameters:") - println(" \tDynamic: " + isDynamic) - println(" \tNumIter: " + numIter) - println(" \tSources: [" + sources.mkString(", ") + "]") - println("======================================") - - val sc = new SparkContext(host, "ShortestPath(" + fname + ")") - val graph = Graph.textFile(sc, fname, a => (if(a.isEmpty) 1.0F else a(0).toFloat ) ) - val sp = Analytics.shortestPath(graph, sources, numIter) - // val cc = if(isDynamic) Analytics.dynamicShortestPath(graph, sources, numIter) - // else Analytics.shortestPath(graph, sources, numIter) - println("Longest Path: " + sp.vertices.map(_.data).reduce(math.max(_,_))) - - sc.stop() - } - - - // case "als" => { - - // var numIter = 5 - // var lambda = 0.01 - // var latentK = 10 - // var usersFname = "usersFactors.tsv" - // var moviesFname = "moviesFname.tsv" - // var numVPart = 4 - // var numEPart = 4 - - // options.foreach{ - // case ("numIter", v) => numIter = v.toInt - // case ("lambda", v) => lambda = v.toDouble - // case ("latentK", v) => latentK = v.toInt - // case ("usersFname", v) => usersFname = v - // case ("moviesFname", v) => moviesFname = v - // case ("numVPart", v) => numVPart = v.toInt - // case ("numEPart", v) => numEPart = v.toInt - // case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) - // } - - // println("======================================") - // println("| Alternating Least Squares |") - // println("--------------------------------------") - // println(" Using parameters:") - // println(" \tNumIter: " + numIter) - // println(" \tLambda: " + lambda) - // println(" \tLatentK: " + latentK) - // println(" \tusersFname: " + usersFname) - // println(" \tmoviesFname: " + moviesFname) - // println("======================================") - - // val sc = new SparkContext(host, "ALS(" + fname + ")") - // val graph = Graph.textFile(sc, fname, a => a(0).toDouble ) - // graph.numVPart = numVPart - // graph.numEPart = numEPart - - // val maxUser = graph.edges.map(_._1).reduce(math.max(_,_)) - // val minMovie = graph.edges.map(_._2).reduce(math.min(_,_)) - // assert(maxUser < minMovie) - - // val factors = Analytics.alternatingLeastSquares(graph, latentK, lambda, numIter).cache - // factors.filter(_._1 <= maxUser).map(r => r._1 + "\t" + r._2.mkString("\t")) - // .saveAsTextFile(usersFname) - // factors.filter(_._1 >= minMovie).map(r => r._1 + "\t" + r._2.mkString("\t")) - // .saveAsTextFile(moviesFname) - - // sc.stop() - // } - - - case _ => { - println("Invalid task type.") - } - } - } +// /** +// * Compute the PageRank of a graph returning the pagerank of each vertex as an RDD +// */ +// def dynamicPagerank[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], +// tol: Float, maxIter: Int = 10) = { +// // Compute the out degree of each vertex +// val pagerankGraph = graph.updateVertices[Int, (Int, Float, Float)](graph.outDegrees, +// (vertex, degIter) => (degIter.sum, 1.0F, 1.0F) +// ) + +// // Run PageRank +// GraphLab.iterateGAS(pagerankGraph)( +// (me_id, edge) => edge.src.data._2 / edge.src.data._1, // gather +// (a: Float, b: Float) => a + b, +// (vertex, a: Option[Float]) => +// (vertex.data._1, (0.15F + 0.85F * a.getOrElse(0F)), vertex.data._2), // apply +// (me_id, edge) => math.abs(edge.src.data._2 - edge.dst.data._1) > tol, // scatter +// maxIter).mapVertices { case Vertex(vid, data) => Vertex(vid, data._2) } +// } + +// /** +// * Compute the connected component membership of each vertex +// * and return an RDD with the vertex value containing the +// * lowest vertex id in the connected component containing +// * that vertex. +// */ +// def connectedComponents[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], numIter: Int) = { +// val ccGraph = graph.mapVertices { case Vertex(vid, _) => Vertex(vid, vid) } +// GraphLab.iterateGA[Int, ED, Int](ccGraph)( +// (me_id, edge) => edge.otherVertex(me_id).data, // gather +// (a: Int, b: Int) => math.min(a, b), // merge +// (v, a: Option[Int]) => math.min(v.data, a.getOrElse(Integer.MAX_VALUE)), // apply +// numIter, +// gatherDirection = EdgeDirection.Both) +// } + +// /** +// * Compute the shortest path to a set of markers +// */ +// def shortestPath[VD: Manifest](graph: Graph[VD, Float], sources: List[Int], numIter: Int) = { +// val sourceSet = sources.toSet +// val spGraph = graph.mapVertices { +// case Vertex(vid, _) => Vertex(vid, (if(sourceSet.contains(vid)) 0.0F else Float.MaxValue)) +// } +// GraphLab.iterateGA[Float, Float, Float](spGraph)( +// (me_id, edge) => edge.otherVertex(me_id).data + edge.data, // gather +// (a: Float, b: Float) => math.min(a, b), // merge +// (v, a: Option[Float]) => math.min(v.data, a.getOrElse(Float.MaxValue)), // apply +// numIter, +// gatherDirection = EdgeDirection.In) +// } + +// // /** +// // * Compute the connected component membership of each vertex +// // * and return an RDD with the vertex value containing the +// // * lowest vertex id in the connected component containing +// // * that vertex. +// // */ +// // def dynamicConnectedComponents[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], +// // numIter: Int = Int.MaxValue) = { + +// // val vertices = graph.vertices.mapPartitions(iter => iter.map { case (vid, _) => (vid, vid) }) +// // val edges = graph.edges // .mapValues(v => None) +// // val ccGraph = new Graph(vertices, edges) + +// // ccGraph.iterateDynamic( +// // (me_id, edge) => edge.otherVertex(me_id).data, // gather +// // (a: Int, b: Int) => math.min(a, b), // merge +// // Integer.MAX_VALUE, +// // (v, a: Int) => math.min(v.data, a), // apply +// // (me_id, edge) => edge.otherVertex(me_id).data > edge.vertex(me_id).data, // scatter +// // numIter, +// // gatherEdges = EdgeDirection.Both, +// // scatterEdges = EdgeDirection.Both).vertices +// // // +// // // graph_ret.vertices.collect.foreach(println) +// // // graph_ret.edges.take(10).foreach(println) +// // } + + + + +// // /** +// // * Compute the shortest path to a set of markers +// // */ +// // def dynamicShortestPath[VD: Manifest, ED: Manifest](graph: Graph[VD, Float], +// // sources: List[Int], numIter: Int) = { +// // val sourceSet = sources.toSet +// // val vertices = graph.vertices.mapPartitions( +// // iter => iter.map { +// // case (vid, _) => (vid, (if(sourceSet.contains(vid)) 0.0F else Float.MaxValue) ) +// // }); + +// // val edges = graph.edges // .mapValues(v => None) +// // val spGraph = new Graph(vertices, edges) + +// // val niterations = Int.MaxValue +// // spGraph.iterateDynamic( +// // (me_id, edge) => edge.otherVertex(me_id).data + edge.data, // gather +// // (a: Float, b: Float) => math.min(a, b), // merge +// // Float.MaxValue, +// // (v, a: Float) => math.min(v.data, a), // apply +// // (me_id, edge) => edge.vertex(me_id).data + edge.data < edge.otherVertex(me_id).data, // scatter +// // numIter, +// // gatherEdges = EdgeDirection.In, +// // scatterEdges = EdgeDirection.Out).vertices +// // } + + + + +// // /** +// // * +// // */ +// // def alternatingLeastSquares[VD: ClassManifest, ED: ClassManifest](graph: Graph[VD, Double], +// // latentK: Int, lambda: Double, numIter: Int) = { +// // val vertices = graph.vertices.mapPartitions( _.map { +// // case (vid, _) => (vid, Array.fill(latentK){ scala.util.Random.nextDouble() } ) +// // }).cache +// // val maxUser = graph.edges.map(_._1).reduce(math.max(_,_)) +// // val edges = graph.edges // .mapValues(v => None) +// // val alsGraph = new Graph(vertices, edges) +// // alsGraph.numVPart = graph.numVPart +// // alsGraph.numEPart = graph.numEPart + +// // val niterations = Int.MaxValue +// // alsGraph.iterateDynamic[(Array[Double], Array[Double])]( +// // (me_id, edge) => { // gather +// // val X = edge.otherVertex(me_id).data +// // val y = edge.data +// // val Xy = X.map(_ * y) +// // val XtX = (for(i <- 0 until latentK; j <- i until latentK) yield(X(i) * X(j))).toArray +// // (Xy, XtX) +// // }, +// // (a, b) => { +// // // The difference between the while loop and the zip is a FACTOR OF TWO in overall +// // // runtime +// // var i = 0 +// // while(i < a._1.length) { a._1(i) += b._1(i); i += 1 } +// // i = 0 +// // while(i < a._2.length) { a._2(i) += b._2(i); i += 1 } +// // a +// // // (a._1.zip(b._1).map{ case (q,r) => q+r }, a._2.zip(b._2).map{ case (q,r) => q+r }) +// // }, +// // (Array.empty[Double], Array.empty[Double]), // default value is empty +// // (vertex, accum) => { // apply +// // val XyArray = accum._1 +// // val XtXArray = accum._2 +// // if(XyArray.isEmpty) vertex.data // no neighbors +// // else { +// // val XtX = DenseMatrix.tabulate(latentK,latentK){ (i,j) => +// // (if(i < j) XtXArray(i + (j+1)*j/2) else XtXArray(i + (j+1)*j/2)) + +// // (if(i == j) lambda else 1.0F) //regularization +// // } +// // val Xy = DenseMatrix.create(latentK,1,XyArray) +// // val w = XtX \ Xy +// // w.data +// // } +// // }, +// // (me_id, edge) => true, +// // numIter, +// // gatherEdges = EdgeDirection.Both, +// // scatterEdges = EdgeDirection.Both, +// // vertex => vertex.id < maxUser).vertices +// // } + +// def main(args: Array[String]) = { +// val host = args(0) +// val taskType = args(1) +// val fname = args(2) +// val options = args.drop(3).map { arg => +// arg.dropWhile(_ == '-').split('=') match { +// case Array(opt, v) => (opt -> v) +// case _ => throw new IllegalArgumentException("Invalid argument: " + arg) +// } +// } + +// System.setProperty("spark.serializer", "spark.KryoSerializer") +// //System.setProperty("spark.shuffle.compress", "false") +// System.setProperty("spark.kryo.registrator", "spark.graph.GraphKryoRegistrator") + +// taskType match { +// case "pagerank" => { + +// var numIter = Int.MaxValue +// var isDynamic = false +// var tol:Float = 0.001F +// var outFname = "" +// var numVPart = 4 +// var numEPart = 4 + +// options.foreach{ +// case ("numIter", v) => numIter = v.toInt +// case ("dynamic", v) => isDynamic = v.toBoolean +// case ("tol", v) => tol = v.toFloat +// case ("output", v) => outFname = v +// case ("numVPart", v) => numVPart = v.toInt +// case ("numEPart", v) => numEPart = v.toInt +// case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) +// } + +// if(!isDynamic && numIter == Int.MaxValue) { +// println("Set number of iterations!") +// sys.exit(1) +// } +// println("======================================") +// println("| PageRank |") +// println("--------------------------------------") +// println(" Using parameters:") +// println(" \tDynamic: " + isDynamic) +// if(isDynamic) println(" \t |-> Tolerance: " + tol) +// println(" \tNumIter: " + numIter) +// println("======================================") + +// val sc = new SparkContext(host, "PageRank(" + fname + ")") + +// val graph = Graph.textFile(sc, fname, a => 1.0F).withPartitioner(numVPart, numEPart).cache() + +// val startTime = System.currentTimeMillis +// logInfo("GRAPHX: starting tasks") +// logInfo("GRAPHX: Number of vertices " + graph.vertices.count) +// logInfo("GRAPHX: Number of edges " + graph.edges.count) + +// val pr = Analytics.pagerank(graph, numIter) +// // val pr = if(isDynamic) Analytics.dynamicPagerank(graph, tol, numIter) +// // else Analytics.pagerank(graph, numIter) +// logInfo("GRAPHX: Total rank: " + pr.vertices.map{ case Vertex(id,r) => r }.reduce(_+_) ) +// if (!outFname.isEmpty) { +// println("Saving pageranks of pages to " + outFname) +// pr.vertices.map{case Vertex(id, r) => id + "\t" + r}.saveAsTextFile(outFname) +// } +// logInfo("GRAPHX: Runtime: " + ((System.currentTimeMillis - startTime)/1000.0) + " seconds") +// sc.stop() +// } + +// case "cc" => { + +// var numIter = Int.MaxValue +// var isDynamic = false + +// options.foreach{ +// case ("numIter", v) => numIter = v.toInt +// case ("dynamic", v) => isDynamic = v.toBoolean +// case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) +// } + +// 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 + ")") +// val graph = Graph.textFile(sc, fname, a => 1.0F) +// val cc = Analytics.connectedComponents(graph, numIter) +// // val cc = if(isDynamic) Analytics.dynamicConnectedComponents(graph, numIter) +// // else Analytics.connectedComponents(graph, numIter) +// println("Components: " + cc.vertices.map(_.data).distinct()) + +// sc.stop() +// } + +// case "shortestpath" => { + +// var numIter = Int.MaxValue +// var isDynamic = true +// var sources: List[Int] = List.empty + +// options.foreach{ +// case ("numIter", v) => numIter = v.toInt +// case ("dynamic", v) => isDynamic = v.toBoolean +// case ("source", v) => sources ++= List(v.toInt) +// case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) +// } + + +// if(!isDynamic && numIter == Int.MaxValue) { +// println("Set number of iterations!") +// sys.exit(1) +// } + +// if(sources.isEmpty) { +// println("No sources provided!") +// sys.exit(1) +// } + +// println("======================================") +// println("| Shortest Path |") +// println("--------------------------------------") +// println(" Using parameters:") +// println(" \tDynamic: " + isDynamic) +// println(" \tNumIter: " + numIter) +// println(" \tSources: [" + sources.mkString(", ") + "]") +// println("======================================") + +// val sc = new SparkContext(host, "ShortestPath(" + fname + ")") +// val graph = Graph.textFile(sc, fname, a => (if(a.isEmpty) 1.0F else a(0).toFloat ) ) +// val sp = Analytics.shortestPath(graph, sources, numIter) +// // val cc = if(isDynamic) Analytics.dynamicShortestPath(graph, sources, numIter) +// // else Analytics.shortestPath(graph, sources, numIter) +// println("Longest Path: " + sp.vertices.map(_.data).reduce(math.max(_,_))) + +// sc.stop() +// } + + +// // case "als" => { + +// // var numIter = 5 +// // var lambda = 0.01 +// // var latentK = 10 +// // var usersFname = "usersFactors.tsv" +// // var moviesFname = "moviesFname.tsv" +// // var numVPart = 4 +// // var numEPart = 4 + +// // options.foreach{ +// // case ("numIter", v) => numIter = v.toInt +// // case ("lambda", v) => lambda = v.toDouble +// // case ("latentK", v) => latentK = v.toInt +// // case ("usersFname", v) => usersFname = v +// // case ("moviesFname", v) => moviesFname = v +// // case ("numVPart", v) => numVPart = v.toInt +// // case ("numEPart", v) => numEPart = v.toInt +// // case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) +// // } + +// // println("======================================") +// // println("| Alternating Least Squares |") +// // println("--------------------------------------") +// // println(" Using parameters:") +// // println(" \tNumIter: " + numIter) +// // println(" \tLambda: " + lambda) +// // println(" \tLatentK: " + latentK) +// // println(" \tusersFname: " + usersFname) +// // println(" \tmoviesFname: " + moviesFname) +// // println("======================================") + +// // val sc = new SparkContext(host, "ALS(" + fname + ")") +// // val graph = Graph.textFile(sc, fname, a => a(0).toDouble ) +// // graph.numVPart = numVPart +// // graph.numEPart = numEPart + +// // val maxUser = graph.edges.map(_._1).reduce(math.max(_,_)) +// // val minMovie = graph.edges.map(_._2).reduce(math.min(_,_)) +// // assert(maxUser < minMovie) + +// // val factors = Analytics.alternatingLeastSquares(graph, latentK, lambda, numIter).cache +// // factors.filter(_._1 <= maxUser).map(r => r._1 + "\t" + r._2.mkString("\t")) +// // .saveAsTextFile(usersFname) +// // factors.filter(_._1 >= minMovie).map(r => r._1 + "\t" + r._2.mkString("\t")) +// // .saveAsTextFile(moviesFname) + +// // sc.stop() +// // } + + +// case _ => { +// println("Invalid task type.") +// } +// } +// } } diff --git a/graph/src/main/scala/spark/graph/Edge.scala b/graph/src/main/scala/spark/graph/Edge.scala new file mode 100644 index 0000000000000..8f022e812a500 --- /dev/null +++ b/graph/src/main/scala/spark/graph/Edge.scala @@ -0,0 +1,7 @@ +package spark.graph + + +case class Edge[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED] ( + var src: Vid = 0, + var dst: Vid = 0, + var data: ED = nullValue[ED]) diff --git a/graph/src/main/scala/spark/graph/EdgeWithVertices.scala b/graph/src/main/scala/spark/graph/EdgeWithVertices.scala new file mode 100644 index 0000000000000..a731f73709cf6 --- /dev/null +++ b/graph/src/main/scala/spark/graph/EdgeWithVertices.scala @@ -0,0 +1,17 @@ +package spark.graph + + +class EdgeWithVertices[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) VD, + @specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED] { + var src: Vertex[VD] = _ + var dst: Vertex[VD] = _ + var data: ED = _ + + def otherVertex(vid: Vid): Vertex[VD] = if (src.id == vid) dst else src + + def vertex(vid: Vid): Vertex[VD] = if (src.id == vid) src else dst + + def relativeDirection(vid: Vid): EdgeDirection = { + if (vid == src.id) EdgeDirection.Out else EdgeDirection.In + } +} diff --git a/graph/src/main/scala/spark/graph/Graph.scala b/graph/src/main/scala/spark/graph/Graph.scala index 1db92217ded9f..04b8c840fd28d 100644 --- a/graph/src/main/scala/spark/graph/Graph.scala +++ b/graph/src/main/scala/spark/graph/Graph.scala @@ -1,483 +1,57 @@ package spark.graph -import scala.collection.JavaConversions._ -import scala.collection.mutable.ArrayBuffer +import spark.RDD -import it.unimi.dsi.fastutil.ints.IntArrayList -import spark.{ClosureCleaner, HashPartitioner, Partitioner, RDD} -import spark.SparkContext -import spark.SparkContext._ -import spark.graph.Graph.EdgePartition -import spark.storage.StorageLevel +abstract class Graph[VD: ClassManifest, ED: ClassManifest] { + def vertices(): RDD[Vertex[VD]] -case class Vertex[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) VD] ( - var id: Vid = 0, - var data: VD = nullValue[VD]) { + def edges(): RDD[Edge[ED]] - def this(tuple: Tuple2[Vid, VD]) = this(tuple._1, tuple._2) + def edgesWithVertices(): RDD[EdgeWithVertices[VD, ED]] - def tuple = (id, data) -} - - -case class Edge[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED] ( - var src: Vid = 0, - var dst: Vid = 0, - var data: ED = nullValue[ED]) - - -class EdgeWithVertices[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) VD, - @specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED] { - var src: Vertex[VD] = _ - var dst: Vertex[VD] = _ - var data: ED = _ - - def otherVertex(vid: Vid): Vertex[VD] = if (src.id == vid) dst else src - - def vertex(vid: Vid): Vertex[VD] = if (src.id == vid) src else dst - - def relativeDirection(vid: Vid): EdgeDirection = { - if (vid == src.id) EdgeDirection.Out else EdgeDirection.In - } -} - - -/** - * A Graph RDD that supports computation on graphs. - */ -class Graph[VD: ClassManifest, ED: ClassManifest] protected ( - val numVertexPartitions: Int, - val numEdgePartitions: Int, - _rawVertices: RDD[Vertex[VD]], - _rawEdges: RDD[Edge[ED]], - _rawVTable: RDD[(Vid, (VD, Array[Pid]))], - _rawETable: RDD[(Pid, EdgePartition[ED])]) { - - def this(vertices: RDD[Vertex[VD]], edges: RDD[Edge[ED]]) = { - this(vertices.partitions.size, edges.partitions.size, vertices, edges, null, null) - } - - def withPartitioner(numVertexPartitions: Int, numEdgePartitions: Int): Graph[VD, ED] = { - if (_cached) { - (new Graph(numVertexPartitions, numEdgePartitions, null, null, _rawVTable, _rawETable)) - .cache() - } else { - new Graph(numVertexPartitions, numEdgePartitions, _rawVertices, _rawEdges, null, null) - } - } - - def withVertexPartitioner(numVertexPartitions: Int) = { - withPartitioner(numVertexPartitions, numEdgePartitions) - } - - def withEdgePartitioner(numEdgePartitions: Int) = { - withPartitioner(numVertexPartitions, numEdgePartitions) - } + def cache(): Graph[VD, ED] - protected var _cached = false + def mapVertices[VD2: ClassManifest](f: Vertex[VD] => VD2): Graph[VD2, ED] - def cache(): Graph[VD, ED] = { - eTable.cache() - vTable.cache() - _cached = true - this - } - - /** Return a RDD of vertices. */ - def vertices: RDD[Vertex[VD]] = { - if (!_cached && _rawVertices != null) { - _rawVertices - } else { - vTable.map { case(vid, (data, pids)) => new Vertex(vid, data) } - } - } - - /** Return a RDD of edges. */ - def edges: RDD[Edge[ED]] = { - if (!_cached && _rawEdges != null) { - _rawEdges - } else { - eTable.mapPartitions { iter => iter.next._2.iterator } - } - } - - /** Return a RDD that brings edges with its source and destination vertices together. */ - def edgesWithVertices: RDD[EdgeWithVertices[VD, ED]] = { - (new EdgeWithVerticesRDD(vTableReplicated, eTable)).mapPartitions { part => part.next._2 } - } - - lazy val numEdges: Long = edges.count() - - lazy val numVertices: Long = vertices.count() - - lazy val inDegrees = mapReduceNeighborhood[Vid]((vid, edge) => 1, _+_, 0, EdgeDirection.In) - - lazy val outDegrees = mapReduceNeighborhood[Vid]((vid,edge) => 1, _+_, 0, EdgeDirection.Out) - - lazy val degrees = mapReduceNeighborhood[Vid]((vid,edge) => 1, _+_, 0, EdgeDirection.Both) + def mapEdges[ED2: ClassManifest](f: Edge[ED] => ED2): Graph[VD, ED2] /** Return a new graph with its edge directions reversed. */ - lazy val reverse: Graph[VD,ED] = { - newGraph(vertices, edges.map{ case Edge(s, t, e) => Edge(t, s, e) }) - } - - def collectNeighborIds(edgeDirection: EdgeDirection) : RDD[(Vid, Array[Vid])] = { - mapReduceNeighborhood[Array[Vid]]( - (vid, edge) => Array(edge.otherVertex(vid).id), - (a, b) => a ++ b, - Array.empty[Vid], - edgeDirection) - } - - def mapVertices[VD2: ClassManifest](f: Vertex[VD] => Vertex[VD2]): Graph[VD2, ED] = { - newGraph(vertices.map(f), edges) - } - - def mapEdges[ED2: ClassManifest](f: Edge[ED] => Edge[ED2]): Graph[VD, ED2] = { - newGraph(vertices, edges.map(f)) - } - - ////////////////////////////////////////////////////////////////////////////////////////////////// - // Lower level transformation methods - ////////////////////////////////////////////////////////////////////////////////////////////////// - - def mapReduceNeighborhood[VD2: ClassManifest]( - mapFunc: (Vid, EdgeWithVertices[VD, ED]) => VD2, - reduceFunc: (VD2, VD2) => VD2, - default: VD2, - gatherDirection: EdgeDirection): RDD[(Vid, VD2)] = { - - ClosureCleaner.clean(mapFunc) - ClosureCleaner.clean(reduceFunc) - - val newVTable = vTableReplicated.mapPartitions({ part => - part.map { v => (v._1, MutableTuple2(v._2, default)) } - }, preservesPartitioning = true) + def reverse: Graph[VD, ED] - (new EdgeWithVerticesRDD[MutableTuple2[VD, VD2], ED](newVTable, eTable)) - .mapPartitions { part => - val (vmap, edges) = part.next() - val edgeSansAcc = new EdgeWithVertices[VD, ED]() - edgeSansAcc.src = new Vertex[VD] - edgeSansAcc.dst = new Vertex[VD] - edges.foreach { e: EdgeWithVertices[MutableTuple2[VD, VD2], ED] => - edgeSansAcc.data = e.data - edgeSansAcc.src.data = e.src.data._1 - edgeSansAcc.dst.data = e.dst.data._1 - edgeSansAcc.src.id = e.src.id - edgeSansAcc.dst.id = e.dst.id - if (gatherDirection == EdgeDirection.In || gatherDirection == EdgeDirection.Both) { - e.dst.data._2 = reduceFunc(e.dst.data._2, mapFunc(edgeSansAcc.dst.id, edgeSansAcc)) - } - if (gatherDirection == EdgeDirection.Out || gatherDirection == EdgeDirection.Both) { - e.src.data._2 = reduceFunc(e.src.data._2, mapFunc(edgeSansAcc.src.id, edgeSansAcc)) - } - } - vmap.int2ObjectEntrySet().fastIterator().map{ entry => - (entry.getIntKey(), entry.getValue()._2) - } - } - .combineByKey((v: VD2) => v, reduceFunc, null, vertexPartitioner, false) - } + def aggregateNeighbors[VD2: ClassManifest]( + mapFunc: (Vid, EdgeWithVertices[VD, ED]) => Option[VD2], + reduceFunc: (VD2, VD2) => VD2, + gatherDirection: EdgeDirection) + : RDD[(Vid, VD2)] - /** - * Same as mapReduceNeighborhood but map function can return none and there is no default value. - * As a consequence, the resulting table may be much smaller than the set of vertices. - */ - def flatMapReduceNeighborhood[VD2: ClassManifest]( - mapFunc: (Vid, EdgeWithVertices[VD, ED]) => Option[VD2], - reduceFunc: (VD2, VD2) => VD2, - gatherDirection: EdgeDirection): RDD[(Vid, VD2)] = { - - ClosureCleaner.clean(mapFunc) - ClosureCleaner.clean(reduceFunc) - - val newVTable = vTableReplicated.mapPartitions({ part => - part.map { v => (v._1, MutableTuple2(v._2, Option.empty[VD2])) } - }, preservesPartitioning = true) - - (new EdgeWithVerticesRDD[MutableTuple2[VD, Option[VD2]], ED](newVTable, eTable)) - .mapPartitions { part => - val (vmap, edges) = part.next() - val edgeSansAcc = new EdgeWithVertices[VD, ED]() - edgeSansAcc.src = new Vertex[VD] - edgeSansAcc.dst = new Vertex[VD] - edges.foreach { e: EdgeWithVertices[MutableTuple2[VD, Option[VD2]], ED] => - edgeSansAcc.data = e.data - edgeSansAcc.src.data = e.src.data._1 - edgeSansAcc.dst.data = e.dst.data._1 - edgeSansAcc.src.id = e.src.id - edgeSansAcc.dst.id = e.dst.id - if (gatherDirection == EdgeDirection.In || gatherDirection == EdgeDirection.Both) { - e.dst.data._2 = - if (e.dst.data._2.isEmpty) { - mapFunc(edgeSansAcc.dst.id, edgeSansAcc) - } else { - val tmp = mapFunc(edgeSansAcc.dst.id, edgeSansAcc) - if (!tmp.isEmpty) Some(reduceFunc(e.dst.data._2.get, tmp.get)) else e.dst.data._2 - } - } - if (gatherDirection == EdgeDirection.Out || gatherDirection == EdgeDirection.Both) { - e.dst.data._2 = - if (e.dst.data._2.isEmpty) { - mapFunc(edgeSansAcc.src.id, edgeSansAcc) - } else { - val tmp = mapFunc(edgeSansAcc.src.id, edgeSansAcc) - if (!tmp.isEmpty) Some(reduceFunc(e.src.data._2.get, tmp.get)) else e.src.data._2 - } - } - } - vmap.int2ObjectEntrySet().fastIterator().filter(!_.getValue()._2.isEmpty).map{ entry => - (entry.getIntKey(), entry.getValue()._2) - } - } - .map{ case (vid, aOpt) => (vid, aOpt.get) } - .combineByKey((v: VD2) => v, reduceFunc, null, vertexPartitioner, false) - } + def aggregateNeighbors[VD2: ClassManifest]( + mapFunc: (Vid, EdgeWithVertices[VD, ED]) => Option[VD2], + reduceFunc: (VD2, VD2) => VD2, + default: VD2, // Should this be a function or a value? + gatherDirection: EdgeDirection) + : RDD[(Vid, VD2)] def updateVertices[U: ClassManifest, VD2: ClassManifest]( updates: RDD[(Vid, U)], updateFunc: (Vertex[VD], Option[U]) => VD2) - : Graph[VD2, ED] = { - - ClosureCleaner.clean(updateFunc) - - val newVTable = vTable.leftOuterJoin(updates).mapPartitions({ iter => - iter.map { case (vid, ((vdata, pids), update)) => - val newVdata = updateFunc(Vertex(vid, vdata), update) - (vid, (newVdata, pids)) - } - }, preservesPartitioning = true).cache() - - new Graph(newVTable.partitions.size, eTable.partitions.size, null, null, newVTable, eTable) - } - - // def mapPartitions[U: ClassManifest]( - // f: (VertexHashMap[VD], Iterator[EdgeWithVertices[VD, ED]]) => Iterator[U], - // preservesPartitioning: Boolean = false): RDD[U] = { - // (new EdgeWithVerticesRDD(vTable, eTable)).mapPartitions({ part => - // val (vmap, iter) = part.next() - // f(vmap, iter) - // }, preservesPartitioning) - // } - - ////////////////////////////////////////////////////////////////////////////////////////////////// - // Internals hidden from callers - ////////////////////////////////////////////////////////////////////////////////////////////////// + : Graph[VD2, ED] - // TODO: Support non-hash partitioning schemes. - protected val vertexPartitioner = new HashPartitioner(numVertexPartitions) - protected val edgePartitioner = new HashPartitioner(numEdgePartitions) - - /** Create a new graph but keep the current partitioning scheme. */ - protected def newGraph[VD2: ClassManifest, ED2: ClassManifest]( - vertices: RDD[Vertex[VD2]], edges: RDD[Edge[ED2]]): Graph[VD2, ED2] = { - (new Graph[VD2, ED2](vertices, edges)).withPartitioner(numVertexPartitions, numEdgePartitions) - } - - protected lazy val eTable: RDD[(Pid, EdgePartition[ED])] = { - if (_rawETable == null) { - Graph.createETable(_rawEdges, numEdgePartitions) - } else { - _rawETable - } - } - - protected lazy val vTable: RDD[(Vid, (VD, Array[Pid]))] = { - if (_rawVTable == null) { - Graph.createVTable(_rawVertices, eTable, numVertexPartitions) - } else { - _rawVTable - } - } + // This one can be used to skip records when we can do in-place update. + // Annoying that we can't rename it ... + def updateVertices2[U: ClassManifest]( + updates: RDD[(Vid, U)], + updateFunc: (Vertex[VD], U) => VD) + : Graph[VD, ED] - protected lazy val vTableReplicated: RDD[(Vid, VD)] = { - // Join vid2pid and vTable, generate a shuffle dependency on the joined result, and get - // the shuffle id so we can use it on the slave. - vTable - .flatMap { case (vid, (vdata, pids)) => pids.iterator.map { pid => (pid, (vid, vdata)) } } - .partitionBy(edgePartitioner) - .mapPartitions( - { part => part.map { case(pid, (vid, vdata)) => (vid, vdata) } }, - preservesPartitioning = true) - } + // 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) } object Graph { - /** - * Load an edge list from file initializing the Graph RDD - */ - def textFile[ED: ClassManifest]( - sc: SparkContext, - path: String, - edgeParser: Array[String] => ED, - minEdgePartitions: Int = 1, - minVertexPartitions: Int = 1) - : Graph[Int, ED] = { - - // Parse the edge data table - val edges = sc.textFile(path).flatMap { line => - if (!line.isEmpty && line(0) != '#') { - val lineArray = line.split("\\s+") - if(lineArray.length < 2) { - println("Invalid line: " + line) - assert(false) - } - val source = lineArray(0) - val target = lineArray(1) - val tail = lineArray.drop(2) - val edata = edgeParser(tail) - Array(Edge(source.trim.toInt, target.trim.toInt, edata)) - } else { - Array.empty[Edge[ED]] - } - }.cache() - - val graph = fromEdges(edges) - // println("Loaded graph:" + - // "\n\t#edges: " + graph.numEdges + - // "\n\t#vertices: " + graph.numVertices) - - graph - } - - def fromEdges[ED: ClassManifest](edges: RDD[Edge[ED]]): Graph[Int, ED] = { - val vertices = edges.flatMap { edge => List((edge.src, 1), (edge.dst, 1)) } - .reduceByKey(_ + _) - .map{ case (vid, degree) => Vertex(vid, degree) } - (new Graph[Int, ED](vertices, edges)) - } - - /** - * Make k-cycles - */ - def kCycles(sc: SparkContext, numCycles: Int = 3, size: Int = 3) = { - // Construct the edges - val edges = sc.parallelize(for (i <- 0 until numCycles; j <- 0 until size) yield { - val offset = i * numCycles - val source = offset + j - val target = offset + ((j + 1) % size) - Edge(source, target, i * numCycles + j) - }) - // Change vertex data to be the lowest vertex id of the vertex in that cycle - val graph = fromEdges(edges).mapVertices{ - case Vertex(id, degree) => Vertex(id, (id/numCycles) * numCycles) - } - graph - } - - /** - * Make a regular grid graph - **/ - def grid(sc: SparkContext, numRows: Int = 5, numCols: Int = 5) = { - def coord(vid: Int) = (vid % numRows, vid / numRows) - val vertices = sc.parallelize( 0 until (numRows * numCols) ).map( - vid => Vertex(vid, coord(vid))) - def index(r: Int, c:Int) = (r + c * numRows) - val edges = vertices.flatMap{ case Vertex(vid, (r,c)) => - (if(r+1 < numRows) List(Edge(vid, index(r+1,c), 1.0F)) else List.empty) ++ - (if(c+1 < numCols) List(Edge(vid, index(r,c+1), 1.0F)) else List.empty) - } - new Graph(vertices, edges) - } - - - /** - * A partition of edges in 3 large columnar arrays. - */ - private[graph] - class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED: ClassManifest] - { - val srcIds: IntArrayList = new IntArrayList - val dstIds: IntArrayList = new IntArrayList - // TODO: Specialize data. - val data: ArrayBuffer[ED] = new ArrayBuffer[ED] - - /** Add a new edge to the partition. */ - def add(src: Vid, dst: Vid, d: ED) { - srcIds.add(src) - dstIds.add(dst) - data += d - } - - def trim() { - srcIds.trim() - dstIds.trim() - } - - def size: Int = srcIds.size - - def iterator = new Iterator[Edge[ED]] { - private var edge = new Edge[ED] - private var pos = 0 - - override def hasNext: Boolean = pos < EdgePartition.this.size - - override def next(): Edge[ED] = { - edge.src = srcIds.get(pos) - edge.dst = dstIds.get(pos) - edge.data = data(pos) - pos += 1 - edge - } - } - } - - /** - * Create the edge table RDD, which is much more efficient for Java heap storage than the - * normal edges data structure (RDD[(Vid, Vid, ED)]). - * - * The edge table contains multiple partitions, and each partition contains only one RDD - * key-value pair: the key is the partition id, and the value is an EdgePartition object - * containing all the edges in a partition. - */ - protected def createETable[ED: ClassManifest](edges: RDD[Edge[ED]], numPartitions: Int) - : RDD[(Pid, EdgePartition[ED])] = { - edges - .map { e => - // Random partitioning based on the source vertex id. - (math.abs(e.src) % numPartitions, (e.src, e.dst, e.data)) - } - .partitionBy(new HashPartitioner(numPartitions)) - .mapPartitionsWithIndex({ (pid, iter) => - val edgePartition = new Graph.EdgePartition[ED] - iter.foreach { case (_, (src, dst, data)) => edgePartition.add(src, dst, data) } - Iterator((pid, edgePartition)) - }, preservesPartitioning = true) - } - - protected def createVTable[VD: ClassManifest, ED: ClassManifest]( - vertices: RDD[Vertex[VD]], - eTable: RDD[(Pid, EdgePartition[ED])], - numPartitions: Int) - : RDD[(Vid, (VD, Array[Pid]))] = { - val partitioner = new HashPartitioner(numPartitions) - - // A key-value RDD. The key is a vertex id, and the value is a list of - // partitions that contains edges referencing the vertex. - val vid2pid : RDD[(Int, Seq[Pid])] = eTable.mapPartitions { iter => - val (pid, edgePartition) = iter.next() - val vSet = new it.unimi.dsi.fastutil.ints.IntOpenHashSet - var i = 0 - while (i < edgePartition.srcIds.size) { - vSet.add(edgePartition.srcIds.getInt(i)) - vSet.add(edgePartition.dstIds.getInt(i)) - i += 1 - } - vSet.iterator.map { vid => (vid.intValue, pid) } - }.groupByKey(partitioner) - - vertices - .map { v => (v.id, v.data) } - .partitionBy(partitioner) - .leftOuterJoin(vid2pid) - .mapValues { - case (vdata, None) => (vdata, Array.empty[Pid]) - case (vdata, Some(pids)) => (vdata, pids.toArray) - } - } + implicit def graphToGraphOps[VD: ClassManifest, ED: ClassManifest](g: Graph[VD, ED]) = g.ops } diff --git a/graph/src/main/scala/spark/graph/GraphLab.scala b/graph/src/main/scala/spark/graph/GraphLab.scala index 4de453663d0f6..9c157b9361dac 100644 --- a/graph/src/main/scala/spark/graph/GraphLab.scala +++ b/graph/src/main/scala/spark/graph/GraphLab.scala @@ -6,116 +6,116 @@ import spark.RDD object GraphLab { - def iterateGA2[VD: ClassManifest, ED: ClassManifest, A: ClassManifest](graph: Graph[VD, ED])( - gather: (Vid, EdgeWithVertices[VD, ED]) => A, - merge: (A, A) => A, - default: A, - apply: (Vertex[VD], A) => VD, - numIter: Int, - gatherDirection: EdgeDirection = EdgeDirection.In) : Graph[VD, ED] = { - - var g = graph.cache() - - var i = 0 - while (i < numIter) { - - val accUpdates: RDD[(Vid, A)] = - g.mapReduceNeighborhood(gather, merge, default, gatherDirection) - - def applyFunc(v: Vertex[VD], update: Option[A]): VD = { apply(v, update.get) } - g = g.updateVertices(accUpdates, applyFunc).cache() - - i += 1 - } - g - } - - def iterateGA[VD: ClassManifest, ED: ClassManifest, A: ClassManifest](graph: Graph[VD, ED])( - gatherFunc: (Vid, EdgeWithVertices[VD, ED]) => A, - mergeFunc: (A, A) => A, - applyFunc: (Vertex[VD], Option[A]) => VD, - numIter: Int, - gatherDirection: EdgeDirection = EdgeDirection.In) : Graph[VD, ED] = { - - var g = graph.cache() - - def someGather(vid: Vid, edge: EdgeWithVertices[VD, ED]) = Some(gatherFunc(vid, edge)) - - var i = 0 - while (i < numIter) { - - val accUpdates: RDD[(Vid, A)] = - g.flatMapReduceNeighborhood(someGather, mergeFunc, gatherDirection) - - g = g.updateVertices(accUpdates, applyFunc).cache() - - i += 1 - } - g - } - - def iterateGAS[VD: ClassManifest, ED: ClassManifest, A: ClassManifest](graph: Graph[VD, ED])( - gatherFunc: (Vid, EdgeWithVertices[VD, ED]) => A, - mergeFunc: (A, A) => A, - applyFunc: (Vertex[VD], Option[A]) => VD, - scatterFunc: (Vid, EdgeWithVertices[VD, ED]) => Boolean, - numIter: Int, - gatherDirection: EdgeDirection = EdgeDirection.In, - scatterDirection: EdgeDirection = EdgeDirection.Out) : Graph[VD, ED] = { - - var g = graph.mapVertices{ case Vertex(id,data) => Vertex(id, (true, data)) }.cache() - - def gather(vid: Vid, e: EdgeWithVertices[(Boolean, VD), ED]) = { - if(e.vertex(vid).data._1) { - val edge = new EdgeWithVertices[VD,ED] - edge.src = Vertex(e.src.id, e.src.data._2) - edge.dst = Vertex(e.dst.id, e.dst.data._2) - Some(gatherFunc(vid, edge)) - } else { - None - } - } - - def apply(v: Vertex[(Boolean, VD)], accum: Option[A]) = { - if(v.data._1) (true, applyFunc(Vertex(v.id, v.data._2), accum)) - else (false, v.data._2) - } - - def scatter(rawVid: Vid, e: EdgeWithVertices[(Boolean, VD),ED]) = { - val vid = e.otherVertex(rawVid).id - if(e.vertex(vid).data._1) { - val edge = new EdgeWithVertices[VD,ED] - edge.src = Vertex(e.src.id, e.src.data._2) - edge.dst = Vertex(e.dst.id, e.dst.data._2) - Some(scatterFunc(vid, edge)) - } else { - None - } - } - - def applyActive(v: Vertex[(Boolean, VD)], accum: Option[Boolean]) = - (accum.getOrElse(false), v.data._2) - - var i = 0 - var numActive = g.numVertices - while (i < numIter && numActive > 0) { - - val accUpdates: RDD[(Vid, A)] = - g.flatMapReduceNeighborhood(gather, mergeFunc, gatherDirection) - - g = g.updateVertices(accUpdates, apply).cache() - - // Scatter is basically a gather in the opposite direction so we reverse the edge direction - val activeVertices: RDD[(Vid, Boolean)] = - g.flatMapReduceNeighborhood(scatter, _ || _, scatterDirection.reverse) - - g = g.updateVertices(activeVertices, applyActive).cache() - - numActive = g.vertices.map(v => if (v.data._1) 1 else 0).reduce( _ + _ ) - println("Number active vertices: " + numActive) - i += 1 - } - - g.mapVertices(v => Vertex(v.id, v.data._2)) - } + // def iterateGA2[VD: ClassManifest, ED: ClassManifest, A: ClassManifest](graph: Graph[VD, ED])( + // gather: (Vid, EdgeWithVertices[VD, ED]) => A, + // merge: (A, A) => A, + // default: A, + // apply: (Vertex[VD], A) => VD, + // numIter: Int, + // gatherDirection: EdgeDirection = EdgeDirection.In) : Graph[VD, ED] = { + + // var g = graph.cache() + + // var i = 0 + // while (i < numIter) { + + // val accUpdates: RDD[(Vid, A)] = + // g.aggregateNeighbors(gather, merge, default, gatherDirection) + + // def applyFunc(v: Vertex[VD], update: Option[A]): VD = { apply(v, update.get) } + // g = g.updateVertices(accUpdates, applyFunc).cache() + + // i += 1 + // } + // g + // } + + // def iterateGA[VD: ClassManifest, ED: ClassManifest, A: ClassManifest](graph: Graph[VD, ED])( + // gatherFunc: (Vid, EdgeWithVertices[VD, ED]) => A, + // mergeFunc: (A, A) => A, + // applyFunc: (Vertex[VD], Option[A]) => VD, + // numIter: Int, + // gatherDirection: EdgeDirection = EdgeDirection.In) : Graph[VD, ED] = { + + // var g = graph.cache() + + // def someGather(vid: Vid, edge: EdgeWithVertices[VD, ED]) = Some(gatherFunc(vid, edge)) + + // var i = 0 + // while (i < numIter) { + + // val accUpdates: RDD[(Vid, A)] = + // g.flatMapReduceNeighborhood(someGather, mergeFunc, gatherDirection) + + // g = g.updateVertices(accUpdates, applyFunc).cache() + + // i += 1 + // } + // g + // } + + // def iterateGAS[VD: ClassManifest, ED: ClassManifest, A: ClassManifest](graph: Graph[VD, ED])( + // gatherFunc: (Vid, EdgeWithVertices[VD, ED]) => A, + // mergeFunc: (A, A) => A, + // applyFunc: (Vertex[VD], Option[A]) => VD, + // scatterFunc: (Vid, EdgeWithVertices[VD, ED]) => Boolean, + // numIter: Int, + // gatherDirection: EdgeDirection = EdgeDirection.In, + // scatterDirection: EdgeDirection = EdgeDirection.Out) : Graph[VD, ED] = { + + // var g = graph.mapVertices{ case Vertex(id,data) => Vertex(id, (true, data)) }.cache() + + // def gather(vid: Vid, e: EdgeWithVertices[(Boolean, VD), ED]) = { + // if(e.vertex(vid).data._1) { + // val edge = new EdgeWithVertices[VD,ED] + // edge.src = Vertex(e.src.id, e.src.data._2) + // edge.dst = Vertex(e.dst.id, e.dst.data._2) + // Some(gatherFunc(vid, edge)) + // } else { + // None + // } + // } + + // def apply(v: Vertex[(Boolean, VD)], accum: Option[A]) = { + // if(v.data._1) (true, applyFunc(Vertex(v.id, v.data._2), accum)) + // else (false, v.data._2) + // } + + // def scatter(rawVid: Vid, e: EdgeWithVertices[(Boolean, VD),ED]) = { + // val vid = e.otherVertex(rawVid).id + // if(e.vertex(vid).data._1) { + // val edge = new EdgeWithVertices[VD,ED] + // edge.src = Vertex(e.src.id, e.src.data._2) + // edge.dst = Vertex(e.dst.id, e.dst.data._2) + // Some(scatterFunc(vid, edge)) + // } else { + // None + // } + // } + + // def applyActive(v: Vertex[(Boolean, VD)], accum: Option[Boolean]) = + // (accum.getOrElse(false), v.data._2) + + // var i = 0 + // var numActive = g.numVertices + // while (i < numIter && numActive > 0) { + + // val accUpdates: RDD[(Vid, A)] = + // g.flatMapReduceNeighborhood(gather, mergeFunc, gatherDirection) + + // g = g.updateVertices(accUpdates, apply).cache() + + // // Scatter is basically a gather in the opposite direction so we reverse the edge direction + // val activeVertices: RDD[(Vid, Boolean)] = + // g.flatMapReduceNeighborhood(scatter, _ || _, scatterDirection.reverse) + + // g = g.updateVertices(activeVertices, applyActive).cache() + + // numActive = g.vertices.map(v => if (v.data._1) 1 else 0).reduce( _ + _ ) + // println("Number active vertices: " + numActive) + // i += 1 + // } + + // g.mapVertices(v => Vertex(v.id, v.data._2)) + // } } diff --git a/graph/src/main/scala/spark/graph/GraphLoader.scala b/graph/src/main/scala/spark/graph/GraphLoader.scala new file mode 100644 index 0000000000000..7e1a05441364e --- /dev/null +++ b/graph/src/main/scala/spark/graph/GraphLoader.scala @@ -0,0 +1,54 @@ +package spark.graph + +import spark.RDD +import spark.SparkContext +import spark.SparkContext._ +import spark.graph.impl.GraphImpl + + +object GraphLoader { + + /** + * Load an edge list from file initializing the Graph RDD + */ + def textFile[ED: ClassManifest]( + sc: SparkContext, + path: String, + edgeParser: Array[String] => ED, + minEdgePartitions: Int = 1, + minVertexPartitions: Int = 1) + : GraphImpl[Int, ED] = { + + // Parse the edge data table + val edges = sc.textFile(path).flatMap { line => + if (!line.isEmpty && line(0) != '#') { + val lineArray = line.split("\\s+") + if(lineArray.length < 2) { + println("Invalid line: " + line) + assert(false) + } + val source = lineArray(0) + val target = lineArray(1) + val tail = lineArray.drop(2) + val edata = edgeParser(tail) + Array(Edge(source.trim.toInt, target.trim.toInt, edata)) + } else { + Array.empty[Edge[ED]] + } + }.cache() + + val graph = fromEdges(edges) + // println("Loaded graph:" + + // "\n\t#edges: " + graph.numEdges + + // "\n\t#vertices: " + graph.numVertices) + + graph + } + + def fromEdges[ED: ClassManifest](edges: RDD[Edge[ED]]): GraphImpl[Int, ED] = { + val vertices = edges.flatMap { edge => List((edge.src, 1), (edge.dst, 1)) } + .reduceByKey(_ + _) + .map{ case (vid, degree) => Vertex(vid, degree) } + new GraphImpl[Int, ED](vertices, edges) + } +} diff --git a/graph/src/main/scala/spark/graph/GraphOps.scala b/graph/src/main/scala/spark/graph/GraphOps.scala new file mode 100644 index 0000000000000..4fba8d19768e5 --- /dev/null +++ b/graph/src/main/scala/spark/graph/GraphOps.scala @@ -0,0 +1,30 @@ +package spark.graph + +import spark.RDD + + +class GraphOps[VD: ClassManifest, ED: ClassManifest](g: Graph[VD, ED]) { + + lazy val numEdges: Long = g.edges.count() + + lazy val numVertices: Long = g.vertices.count() + + lazy val inDegrees: RDD[(Vid, Int)] = { + g.aggregateNeighbors((vid, edge) => Some(1), _+_, EdgeDirection.In) + } + + lazy val outDegrees: RDD[(Vid, Int)] = { + g.aggregateNeighbors((vid,edge) => Some(1), _+_, EdgeDirection.Out) + } + + lazy val degrees: RDD[(Vid, Int)] = { + g.aggregateNeighbors((vid,edge) => Some(1), _+_, EdgeDirection.Both) + } + + def collectNeighborIds(edgeDirection: EdgeDirection) : RDD[(Vid, Array[Vid])] = { + g.aggregateNeighbors( + (vid, edge) => Some(Array(edge.otherVertex(vid).id)), + (a, b) => a ++ b, + edgeDirection) + } +} diff --git a/graph/src/main/scala/spark/graph/Pregel.scala b/graph/src/main/scala/spark/graph/Pregel.scala index 4bd881063495b..7f8849e4426c7 100644 --- a/graph/src/main/scala/spark/graph/Pregel.scala +++ b/graph/src/main/scala/spark/graph/Pregel.scala @@ -16,41 +16,20 @@ object Pregel { var g = graph.cache var i = 0 - def reverseGather(vid: Vid, edge: EdgeWithVertices[VD,ED]) = - sendMsg(edge.otherVertex(vid).id, edge) + def mapF(vid: Vid, edge: EdgeWithVertices[VD,ED]) = sendMsg(edge.otherVertex(vid).id, edge) + + def runProg(v: Vertex[VD], msg: Option[A]): VD = { + if (msg.isEmpty) v.data else vprog(v, msg.get) + } var msgs: RDD[(Vid, A)] = g.vertices.map{ v => (v.id, initialMsg) } while (i < numIter) { - - def runProg(v: Vertex[VD], msg: Option[A]): VD = if(msg.isEmpty) v.data else vprog(v, msg.get) - g = g.updateVertices(msgs, runProg).cache() - - msgs = g.flatMapReduceNeighborhood(reverseGather, mergeMsg, EdgeDirection.In) - + msgs = g.aggregateNeighbors(mapF, mergeMsg, EdgeDirection.In) i += 1 } g } - - def iterateOriginal[VD: ClassManifest, ED: ClassManifest, A: ClassManifest]( - rawGraph: Graph[VD, ED])( - vprog: ( Vertex[VD], A, Seq[Vid]) => Seq[(Vid, A)], - mergeMsg: (A, A) => A, - numIter: Int) : Graph[VD, ED] = { - - var graph = rawGraph.cache - var i = 0 - - val outNbrIds : RDD[(Vid, Array[Vid])] = graph.collectNeighborIds(EdgeDirection.Out) - - /// Todo implement - /// vprog takes the vertex, the message (A), and list of out neighbor ids - - graph - - } - } diff --git a/graph/src/main/scala/spark/graph/Timer.scala b/graph/src/main/scala/spark/graph/Timer.scala deleted file mode 100644 index 5ed4d70e1b967..0000000000000 --- a/graph/src/main/scala/spark/graph/Timer.scala +++ /dev/null @@ -1,14 +0,0 @@ -package spark.graph - - -class Timer { - - var lastTime = System.currentTimeMillis - - def tic = { - val currentTime = System.currentTimeMillis - val elapsedTime = (currentTime - lastTime)/1000.0 - lastTime = currentTime - elapsedTime - } -} diff --git a/graph/src/main/scala/spark/graph/Vertex.scala b/graph/src/main/scala/spark/graph/Vertex.scala new file mode 100644 index 0000000000000..543cc8e942778 --- /dev/null +++ b/graph/src/main/scala/spark/graph/Vertex.scala @@ -0,0 +1,11 @@ +package spark.graph + + +case class Vertex[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) VD] ( + var id: Vid = 0, + var data: VD = nullValue[VD]) { + + def this(tuple: Tuple2[Vid, VD]) = this(tuple._1, tuple._2) + + def tuple = (id, data) +} diff --git a/graph/src/main/scala/spark/graph/impl/EdgePartition.scala b/graph/src/main/scala/spark/graph/impl/EdgePartition.scala new file mode 100644 index 0000000000000..e5ed2db0f2947 --- /dev/null +++ b/graph/src/main/scala/spark/graph/impl/EdgePartition.scala @@ -0,0 +1,49 @@ +package spark.graph.impl + +import scala.collection.mutable.ArrayBuffer + +import it.unimi.dsi.fastutil.ints.IntArrayList + +import spark.graph._ + + +/** + * A partition of edges in 3 large columnar arrays. + */ +private[graph] +class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED: ClassManifest] +{ + val srcIds: IntArrayList = new IntArrayList + val dstIds: IntArrayList = new IntArrayList + // TODO: Specialize data. + val data: ArrayBuffer[ED] = new ArrayBuffer[ED] + + /** Add a new edge to the partition. */ + def add(src: Vid, dst: Vid, d: ED) { + srcIds.add(src) + dstIds.add(dst) + data += d + } + + def trim() { + srcIds.trim() + dstIds.trim() + } + + def size: Int = srcIds.size + + def iterator = new Iterator[Edge[ED]] { + private var edge = new Edge[ED] + private var pos = 0 + + override def hasNext: Boolean = pos < EdgePartition.this.size + + override def next(): Edge[ED] = { + edge.src = srcIds.get(pos) + edge.dst = dstIds.get(pos) + edge.data = data(pos) + pos += 1 + edge + } + } +} \ No newline at end of file diff --git a/graph/src/main/scala/spark/graph/EdgeWithVerticesRDD.scala b/graph/src/main/scala/spark/graph/impl/EdgeWithVerticesRDD.scala similarity index 97% rename from graph/src/main/scala/spark/graph/EdgeWithVerticesRDD.scala rename to graph/src/main/scala/spark/graph/impl/EdgeWithVerticesRDD.scala index 91e4d0e01718f..5370b4e1604f6 100644 --- a/graph/src/main/scala/spark/graph/EdgeWithVerticesRDD.scala +++ b/graph/src/main/scala/spark/graph/impl/EdgeWithVerticesRDD.scala @@ -1,9 +1,9 @@ -package spark.graph +package spark.graph.impl import spark.{Aggregator, HashPartitioner, Partition, RDD, SparkEnv, TaskContext} import spark.{Dependency, OneToOneDependency, ShuffleDependency} import spark.SparkContext._ -import spark.graph.Graph.EdgePartition +import spark.graph._ private[graph] diff --git a/graph/src/main/scala/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/spark/graph/impl/GraphImpl.scala new file mode 100644 index 0000000000000..c362746d22b7c --- /dev/null +++ b/graph/src/main/scala/spark/graph/impl/GraphImpl.scala @@ -0,0 +1,343 @@ +package spark.graph.impl + +import scala.collection.JavaConversions._ + +import spark.ClosureCleaner +import spark.HashPartitioner +import spark.Partitioner +import spark.RDD +import spark.SparkContext +import spark.SparkContext._ + +import spark.graph._ +import spark.graph.impl.GraphImpl._ + + +/** + * A Graph RDD that supports computation on graphs. + */ +class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( + val numVertexPartitions: Int, + val numEdgePartitions: Int, + _rawVertices: RDD[Vertex[VD]], + _rawEdges: RDD[Edge[ED]], + _rawVTable: RDD[(Vid, (VD, Array[Pid]))], + _rawETable: RDD[(Pid, EdgePartition[ED])]) + extends Graph[VD, ED] { + + def this(vertices: RDD[Vertex[VD]], edges: RDD[Edge[ED]]) = { + this(vertices.partitions.size, edges.partitions.size, vertices, edges, null, null) + } + + def withPartitioner(numVertexPartitions: Int, numEdgePartitions: Int): Graph[VD, ED] = { + if (_cached) { + (new GraphImpl(numVertexPartitions, numEdgePartitions, null, null, _rawVTable, _rawETable)) + .cache() + } else { + new GraphImpl(numVertexPartitions, numEdgePartitions, _rawVertices, _rawEdges, null, null) + } + } + + def withVertexPartitioner(numVertexPartitions: Int) = { + withPartitioner(numVertexPartitions, numEdgePartitions) + } + + def withEdgePartitioner(numEdgePartitions: Int) = { + withPartitioner(numVertexPartitions, numEdgePartitions) + } + + protected var _cached = false + + override def cache(): Graph[VD, ED] = { + eTable.cache() + vTable.cache() + _cached = true + this + } + + override def reverse: Graph[VD, ED] = { + newGraph(vertices, edges.map{ case Edge(s, t, e) => Edge(t, s, e) }) + } + + /** Return a RDD of vertices. */ + override def vertices: RDD[Vertex[VD]] = { + if (!_cached && _rawVertices != null) { + _rawVertices + } else { + vTable.map { case(vid, (data, pids)) => new Vertex(vid, data) } + } + } + + /** Return a RDD of edges. */ + override def edges: RDD[Edge[ED]] = { + if (!_cached && _rawEdges != null) { + _rawEdges + } else { + eTable.mapPartitions { iter => iter.next._2.iterator } + } + } + + /** Return a RDD that brings edges with its source and destination vertices together. */ + override def edgesWithVertices: RDD[EdgeWithVertices[VD, ED]] = { + (new EdgeWithVerticesRDD(vTableReplicated, eTable)).mapPartitions { part => part.next._2 } + } + + override def mapVertices[VD2: ClassManifest](f: Vertex[VD] => VD2): Graph[VD2, ED] = { + newGraph(vertices.map(v => Vertex(v.id, f(v))), edges) + } + + override def mapEdges[ED2: ClassManifest](f: Edge[ED] => ED2): Graph[VD, ED2] = { + newGraph(vertices, edges.map(e => Edge(e.src, e.dst, f(e)))) + } + + ////////////////////////////////////////////////////////////////////////////////////////////////// + // Lower level transformation methods + ////////////////////////////////////////////////////////////////////////////////////////////////// + + override def aggregateNeighbors[VD2: ClassManifest]( + mapFunc: (Vid, EdgeWithVertices[VD, ED]) => Option[VD2], + reduceFunc: (VD2, VD2) => VD2, + default: VD2, + gatherDirection: EdgeDirection) + : RDD[(Vid, VD2)] = { + + ClosureCleaner.clean(mapFunc) + ClosureCleaner.clean(reduceFunc) + + val newVTable = vTableReplicated.mapPartitions({ part => + part.map { v => (v._1, MutableTuple2(v._2, Option.empty[VD2])) } + }, preservesPartitioning = true) + + (new EdgeWithVerticesRDD[MutableTuple2[VD, Option[VD2]], ED](newVTable, eTable)) + .mapPartitions { part => + val (vmap, edges) = part.next() + val edgeSansAcc = new EdgeWithVertices[VD, ED]() + edgeSansAcc.src = new Vertex[VD] + edgeSansAcc.dst = new Vertex[VD] + edges.foreach { e: EdgeWithVertices[MutableTuple2[VD, Option[VD2]], ED] => + edgeSansAcc.data = e.data + edgeSansAcc.src.data = e.src.data._1 + edgeSansAcc.dst.data = e.dst.data._1 + edgeSansAcc.src.id = e.src.id + edgeSansAcc.dst.id = e.dst.id + if (gatherDirection == EdgeDirection.In || gatherDirection == EdgeDirection.Both) { + e.dst.data._2 = + if (e.dst.data._2.isEmpty) { + mapFunc(edgeSansAcc.dst.id, edgeSansAcc) + } else { + val tmp = mapFunc(edgeSansAcc.dst.id, edgeSansAcc) + if (!tmp.isEmpty) Some(reduceFunc(e.dst.data._2.get, tmp.get)) else e.dst.data._2 + } + } + if (gatherDirection == EdgeDirection.Out || gatherDirection == EdgeDirection.Both) { + e.dst.data._2 = + if (e.dst.data._2.isEmpty) { + mapFunc(edgeSansAcc.src.id, edgeSansAcc) + } else { + val tmp = mapFunc(edgeSansAcc.src.id, edgeSansAcc) + if (!tmp.isEmpty) Some(reduceFunc(e.src.data._2.get, tmp.get)) else e.src.data._2 + } + } + } + vmap.int2ObjectEntrySet().fastIterator().filter(!_.getValue()._2.isEmpty).map{ entry => + (entry.getIntKey(), entry.getValue()._2) + } + } + .map{ case (vid, aOpt) => (vid, aOpt.get) } + .combineByKey((v: VD2) => v, reduceFunc, null, vertexPartitioner, false) + } + + /** + * Same as mapReduceNeighborhood but map function can return none and there is no default value. + * As a consequence, the resulting table may be much smaller than the set of vertices. + */ + override def aggregateNeighbors[VD2: ClassManifest]( + mapFunc: (Vid, EdgeWithVertices[VD, ED]) => Option[VD2], + reduceFunc: (VD2, VD2) => VD2, + gatherDirection: EdgeDirection): RDD[(Vid, VD2)] = { + + ClosureCleaner.clean(mapFunc) + ClosureCleaner.clean(reduceFunc) + + val newVTable = vTableReplicated.mapPartitions({ part => + part.map { v => (v._1, MutableTuple2(v._2, Option.empty[VD2])) } + }, preservesPartitioning = true) + + (new EdgeWithVerticesRDD[MutableTuple2[VD, Option[VD2]], ED](newVTable, eTable)) + .mapPartitions { part => + val (vmap, edges) = part.next() + val edgeSansAcc = new EdgeWithVertices[VD, ED]() + edgeSansAcc.src = new Vertex[VD] + edgeSansAcc.dst = new Vertex[VD] + edges.foreach { e: EdgeWithVertices[MutableTuple2[VD, Option[VD2]], ED] => + edgeSansAcc.data = e.data + edgeSansAcc.src.data = e.src.data._1 + edgeSansAcc.dst.data = e.dst.data._1 + edgeSansAcc.src.id = e.src.id + edgeSansAcc.dst.id = e.dst.id + if (gatherDirection == EdgeDirection.In || gatherDirection == EdgeDirection.Both) { + e.dst.data._2 = + if (e.dst.data._2.isEmpty) { + mapFunc(edgeSansAcc.dst.id, edgeSansAcc) + } else { + val tmp = mapFunc(edgeSansAcc.dst.id, edgeSansAcc) + if (!tmp.isEmpty) Some(reduceFunc(e.dst.data._2.get, tmp.get)) else e.dst.data._2 + } + } + if (gatherDirection == EdgeDirection.Out || gatherDirection == EdgeDirection.Both) { + e.dst.data._2 = + if (e.dst.data._2.isEmpty) { + mapFunc(edgeSansAcc.src.id, edgeSansAcc) + } else { + val tmp = mapFunc(edgeSansAcc.src.id, edgeSansAcc) + if (!tmp.isEmpty) Some(reduceFunc(e.src.data._2.get, tmp.get)) else e.src.data._2 + } + } + } + vmap.int2ObjectEntrySet().fastIterator().filter(!_.getValue()._2.isEmpty).map{ entry => + (entry.getIntKey(), entry.getValue()._2) + } + } + .map{ case (vid, aOpt) => (vid, aOpt.get) } + .combineByKey((v: VD2) => v, reduceFunc, null, vertexPartitioner, false) + } + + override def updateVertices[U: ClassManifest, VD2: ClassManifest]( + updates: RDD[(Vid, U)], + updateF: (Vertex[VD], Option[U]) => VD2) + : Graph[VD2, ED] = { + + ClosureCleaner.clean(updateF) + + val newVTable = vTable.leftOuterJoin(updates).mapPartitions({ iter => + iter.map { case (vid, ((vdata, pids), update)) => + val newVdata = updateF(Vertex(vid, vdata), update) + (vid, (newVdata, pids)) + } + }, preservesPartitioning = true).cache() + + new GraphImpl(newVTable.partitions.size, eTable.partitions.size, null, null, newVTable, eTable) + } + + override def updateVertices2[U: ClassManifest]( + updates: RDD[(Vid, U)], + updateF: (Vertex[VD], U) => VD) + : Graph[VD, ED] = { + + ClosureCleaner.clean(updateF) + + val newVTable = vTable.leftOuterJoin(updates).mapPartitions({ iter => + iter.map { case (vid, ((vdata, pids), update)) => + if (update.isDefined) { + val newVdata = updateF(Vertex(vid, vdata), update.get) + (vid, (newVdata, pids)) + } else { + (vid, (vdata, pids)) + } + } + }, preservesPartitioning = true).cache() + + new GraphImpl(newVTable.partitions.size, eTable.partitions.size, null, null, newVTable, eTable) + } + + + ////////////////////////////////////////////////////////////////////////////////////////////////// + // Internals hidden from callers + ////////////////////////////////////////////////////////////////////////////////////////////////// + + // TODO: Support non-hash partitioning schemes. + protected val vertexPartitioner = new HashPartitioner(numVertexPartitions) + protected val edgePartitioner = new HashPartitioner(numEdgePartitions) + + /** Create a new graph but keep the current partitioning scheme. */ + protected def newGraph[VD2: ClassManifest, ED2: ClassManifest]( + vertices: RDD[Vertex[VD2]], edges: RDD[Edge[ED2]]): Graph[VD2, ED2] = { + (new GraphImpl[VD2, ED2](vertices, edges)).withPartitioner(numVertexPartitions, numEdgePartitions) + } + + protected lazy val eTable: RDD[(Pid, EdgePartition[ED])] = { + if (_rawETable == null) { + createETable(_rawEdges, numEdgePartitions) + } else { + _rawETable + } + } + + protected lazy val vTable: RDD[(Vid, (VD, Array[Pid]))] = { + if (_rawVTable == null) { + createVTable(_rawVertices, eTable, numVertexPartitions) + } else { + _rawVTable + } + } + + protected lazy val vTableReplicated: RDD[(Vid, VD)] = { + // Join vid2pid and vTable, generate a shuffle dependency on the joined result, and get + // the shuffle id so we can use it on the slave. + vTable + .flatMap { case (vid, (vdata, pids)) => pids.iterator.map { pid => (pid, (vid, vdata)) } } + .partitionBy(edgePartitioner) + .mapPartitions( + { part => part.map { case(pid, (vid, vdata)) => (vid, vdata) } }, + preservesPartitioning = true) + } +} + + +object GraphImpl { + + /** + * Create the edge table RDD, which is much more efficient for Java heap storage than the + * normal edges data structure (RDD[(Vid, Vid, ED)]). + * + * The edge table contains multiple partitions, and each partition contains only one RDD + * key-value pair: the key is the partition id, and the value is an EdgePartition object + * containing all the edges in a partition. + */ + protected def createETable[ED: ClassManifest](edges: RDD[Edge[ED]], numPartitions: Int) + : RDD[(Pid, EdgePartition[ED])] = { + edges + .map { e => + // Random partitioning based on the source vertex id. + (math.abs(e.src) % numPartitions, (e.src, e.dst, e.data)) + } + .partitionBy(new HashPartitioner(numPartitions)) + .mapPartitionsWithIndex({ (pid, iter) => + val edgePartition = new EdgePartition[ED] + iter.foreach { case (_, (src, dst, data)) => edgePartition.add(src, dst, data) } + Iterator((pid, edgePartition)) + }, preservesPartitioning = true) + } + + protected def createVTable[VD: ClassManifest, ED: ClassManifest]( + vertices: RDD[Vertex[VD]], + eTable: RDD[(Pid, EdgePartition[ED])], + numPartitions: Int) + : RDD[(Vid, (VD, Array[Pid]))] = { + val partitioner = new HashPartitioner(numPartitions) + + // A key-value RDD. The key is a vertex id, and the value is a list of + // partitions that contains edges referencing the vertex. + val vid2pid : RDD[(Int, Seq[Pid])] = eTable.mapPartitions { iter => + val (pid, edgePartition) = iter.next() + val vSet = new it.unimi.dsi.fastutil.ints.IntOpenHashSet + var i = 0 + while (i < edgePartition.srcIds.size) { + vSet.add(edgePartition.srcIds.getInt(i)) + vSet.add(edgePartition.dstIds.getInt(i)) + i += 1 + } + vSet.iterator.map { vid => (vid.intValue, pid) } + }.groupByKey(partitioner) + + vertices + .map { v => (v.id, v.data) } + .partitionBy(partitioner) + .leftOuterJoin(vid2pid) + .mapValues { + case (vdata, None) => (vdata, Array.empty[Pid]) + case (vdata, Some(pids)) => (vdata, pids.toArray) + } + } +} + diff --git a/graph/src/main/scala/spark/graph/BagelTest.scala b/graph/src/main/scala/spark/graph/perf/BagelTest.scala similarity index 93% rename from graph/src/main/scala/spark/graph/BagelTest.scala rename to graph/src/main/scala/spark/graph/perf/BagelTest.scala index eee53bd6f6a6b..7547292500d6c 100644 --- a/graph/src/main/scala/spark/graph/BagelTest.scala +++ b/graph/src/main/scala/spark/graph/perf/BagelTest.scala @@ -1,9 +1,10 @@ -package spark.graph +package spark.graph.perf import spark._ import spark.SparkContext._ import spark.bagel.Bagel import spark.bagel.examples._ +import spark.graph._ object BagelTest { @@ -41,7 +42,7 @@ object BagelTest { } val sc = new SparkContext(host, "PageRank(" + fname + ")") - val g = Graph.textFile(sc, fname, a => 1.0F).withPartitioner(numVPart, numEPart).cache() + val g = GraphLoader.textFile(sc, fname, a => 1.0F).withPartitioner(numVPart, numEPart).cache() val startTime = System.currentTimeMillis val numVertices = g.vertices.count() diff --git a/graph/src/main/scala/spark/graph/perf/SparkTest.scala b/graph/src/main/scala/spark/graph/perf/SparkTest.scala new file mode 100644 index 0000000000000..85ebd14bcb63c --- /dev/null +++ b/graph/src/main/scala/spark/graph/perf/SparkTest.scala @@ -0,0 +1,72 @@ +package spark.graph.perf + +import spark._ +import spark.SparkContext._ +import spark.bagel.Bagel +import spark.bagel.examples._ +import spark.graph._ + + +object SparkTest { + + def main(args: Array[String]) { + val host = args(0) + val taskType = args(1) + val fname = args(2) + val options = args.drop(3).map { arg => + arg.dropWhile(_ == '-').split('=') match { + case Array(opt, v) => (opt -> v) + case _ => throw new IllegalArgumentException("Invalid argument: " + arg) + } + } + + System.setProperty("spark.serializer", "spark.KryoSerializer") + //System.setProperty("spark.shuffle.compress", "false") + System.setProperty("spark.kryo.registrator", "spark.bagel.examples.PRKryoRegistrator") + + var numIter = Int.MaxValue + var isDynamic = false + var tol:Float = 0.001F + var outFname = "" + var numVPart = 4 + var numEPart = 4 + + options.foreach{ + case ("numIter", v) => numIter = v.toInt + case ("dynamic", v) => isDynamic = v.toBoolean + case ("tol", v) => tol = v.toFloat + case ("output", v) => outFname = v + case ("numVPart", v) => numVPart = v.toInt + case ("numEPart", v) => numEPart = v.toInt + case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) + } + + val sc = new SparkContext(host, "PageRank(" + fname + ")") + val g = GraphLoader.textFile(sc, fname, a => 1.0F).withPartitioner(numVPart, numEPart).cache() + val startTime = System.currentTimeMillis + + val numVertices = g.vertices.count() + + val vertices = g.collectNeighborIds(EdgeDirection.Out).map { case (vid, neighbors) => + (vid.toString, new PRVertex(1.0, neighbors.map(_.toString))) + } + + // Do the computation + val epsilon = 0.01 / numVertices + val messages = sc.parallelize(Array[(String, PRMessage)]()) + val utils = new PageRankUtils + val result = + Bagel.run( + sc, vertices, messages, combiner = new PRCombiner(), + numPartitions = numVPart)( + utils.computeWithCombiner(numVertices, epsilon, numIter)) + + println("Total rank: " + result.map{ case (id, r) => r.value }.reduce(_+_) ) + if (!outFname.isEmpty) { + println("Saving pageranks of pages to " + outFname) + result.map{ case (id, r) => id + "\t" + r.value }.saveAsTextFile(outFname) + } + println("Runtime: " + ((System.currentTimeMillis - startTime)/1000.0) + " seconds") + sc.stop() + } +} From 0c24305b8dbda5278c2f03d02bbee49e1df5ee44 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Thu, 9 May 2013 20:14:27 -0700 Subject: [PATCH 036/531] added documentation to graph and did some minor renaming --- .../main/scala/spark/graph/Analytics.scala | 2 +- graph/src/main/scala/spark/graph/Graph.scala | 296 +++++++++++++++++- graph/src/main/scala/spark/graph/Pregel.scala | 2 +- .../scala/spark/graph/impl/GraphImpl.scala | 10 +- 4 files changed, 292 insertions(+), 18 deletions(-) diff --git a/graph/src/main/scala/spark/graph/Analytics.scala b/graph/src/main/scala/spark/graph/Analytics.scala index 8b3271729bae9..05275bec68347 100644 --- a/graph/src/main/scala/spark/graph/Analytics.scala +++ b/graph/src/main/scala/spark/graph/Analytics.scala @@ -39,7 +39,7 @@ object Analytics extends Logging { */ def pregelPagerank[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], numIter: Int) = { // Compute the out degree of each vertex - val pagerankGraph = graph.updateVertices[Int, (Int, Double)](graph.outDegrees, + val pagerankGraph = graph.leftJoinVertices[Int, (Int, Double)](graph.outDegrees, (vertex, deg) => (deg.getOrElse(0), 1.0) ) Pregel.iterate[(Int, Double), ED, Double](pagerankGraph)( diff --git a/graph/src/main/scala/spark/graph/Graph.scala b/graph/src/main/scala/spark/graph/Graph.scala index 04b8c840fd28d..421055d319e64 100644 --- a/graph/src/main/scala/spark/graph/Graph.scala +++ b/graph/src/main/scala/spark/graph/Graph.scala @@ -3,46 +3,314 @@ package spark.graph import spark.RDD + +/** + * The Graph abstractly represents a graph with arbitrary objects associated + * with vertices and edges. The graph provides basic operations to access and + * manipulate the data associated with vertices and edges as well as the + * underlying structure. Like Spark RDDs, the graph is a functional + * data-structure in which mutating operations return new graphs. + * + * @tparam VD The type of object associated with each vertex. + * + * @tparam ED The type of object associated with each edge + */ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { + /** + * Get the vertices and their data. + * + * @see Vertex for the vertex type. + * + * @todo should vertices return tuples instead of vertex objects? + */ def vertices(): RDD[Vertex[VD]] + /** + * Get the Edges and their data as an RDD. The entries in the RDD contain + * just the source id and target id along with the edge data. + * + * + * @see Edge for the edge type. + * @see edgesWithVertices to get an RDD which contains all the edges along + * with their vertex data. + * + * @todo Should edges return 3 tuples instead of Edge objects? In this case + * we could rename EdgeWithVertices to Edge? + */ def edges(): RDD[Edge[ED]] + /** + * Get the edges with the vertex data associated with the adjacent pair of + * vertices. + * + * @example This operation might be used to evaluate a graph coloring where + * we would like to check that both vertices are a different color. + * {{{ + * type Color = Int + * val graph: Graph[Color, Int] = Graph.textFile("hdfs://file.tsv") + * val numInvalid = graph.edgesWithVertices() + * .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. + * + */ def edgesWithVertices(): RDD[EdgeWithVertices[VD, ED]] + /** + * Return a graph that is cached when first created. This is used to pin a + * graph in memory enabling multiple queries to reuse the same construction + * process. + * + * @see RDD.cache() for a more detailed explanation of caching. + */ def cache(): Graph[VD, ED] - def mapVertices[VD2: ClassManifest](f: Vertex[VD] => VD2): Graph[VD2, ED] + /** + * Construct a new graph where each vertex value has been transformed by the + * map function. + * + * @note This graph is not changed and that the new graph has the same + * structure. As a consequence the underlying index structures can be + * reused. + * + * @param map the function from a vertex object to a new vertex value. + * + * @tparam VD2 the new vertex data type + * + * @example We might use this operation to change the vertex values from one + * type to another to initialize an algorithm. + * {{{ + * val rawGraph: Graph[(), ()] = Graph.textFile("hdfs://file") + * val root = 42 + * var bfsGraph = rawGraph + * .mapVertices[Int](v => if(v.id == 0) 0 else Math.MaxValue) + * }}} + * + */ + def mapVertices[VD2: ClassManifest](map: Vertex[VD] => VD2): Graph[VD2, ED] + + /** + * Construct a new graph where each the value of each edge is transformed by + * the map operation. This function is not passed the vertex value for the + * vertices adjacent to the edge. If vertex values are desired use the + * mapEdgesWithVertices function. + * + * @note This graph is not changed and that the new graph has the same + * structure. As a consequence the underlying index structures can be + * reused. + * + * @param map the function from an edge object to a new edge value. + * + * @tparam ED2 the new edge data type + * + * @example This function might be used to initialize edge attributes. + * + */ + def mapEdges[ED2: ClassManifest](map: Edge[ED] => ED2): Graph[VD, ED2] + + /** + * Construct a new graph where each the value of each edge is transformed by + * the map operation. This function passes vertex values for the adjacent + * vertices to the map function. If adjacent vertex values are not required, + * consider using the mapEdges function instead. + * + * @note This graph is not changed and that the new graph has the same + * structure. As a consequence the underlying index structures can be + * reused. + * + * @param map the function from an edge object to a new edge value. + * + * @tparam ED2 the new edge data type + * + * @example This function might be used to initialize edge attributes based + * on the attributes associated with each vertex. + * {{{ + * val rawGraph: Graph[Int, Int] = someLoadFunction() + * val graph = rawGraph.mapEdgesWithVertices[Int]( edge => + * edge.src.data - edge.dst.data) + * }}} + * + */ + def mapEdgesWithVertices[ED2: ClassManifest]( + map: EdgeWithVertices[VD, ED] => ED2): Graph[VD, ED2] - def mapEdges[ED2: ClassManifest](f: Edge[ED] => ED2): Graph[VD, ED2] - /** Return a new graph with its edge directions reversed. */ + /** + * Construct a new graph with all the edges reversed. If this graph contains + * an edge from a to b then the returned graph contains an edge from b to a. + * + */ def reverse: Graph[VD, ED] + /** + * This function is used to compute a statistic for the neighborhood of each + * vertex. + * + * This is one of the core functions in the Graph API in that enables + * neighborhood level computation. For example this function can be used to + * count neighbors satisfying a predicate or implement PageRank. + * + * @note The returned RDD may contain fewer entries than their are vertices + * in the graph. This is because some vertices may not have neighbors or the + * map function may return None for all neighbors. + * + * @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 mergeFunc 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 VD2 The returned type of the aggregation operation. + * + * @return A Spark.RDD containing tuples of vertex identifiers and thee + * resulting value. Note that the returned RDD may contain fewer vertices + * than in the original graph since some vertices may not have neighbors or + * the map function could return None for all neighbors. + * + * @example We can use this function to compute the average follower age for + * each user + * {{{ + * val graph: Graph[Int,Int] = loadGraph() + * val averageFollowerAge: RDD[(Int, Int)] = + * graph.aggregateNeigbhros[(Int,Double)]( + * (vid, edge) => (edge.otherVertex(vid).data, 1), + * (a, b) => (a._1 + b._1, a._2 + b._2), + * EdgeDirection.In) + * .mapValues{ case (sum,followers) => sum.toDouble / followers} + * }}} + * + */ def aggregateNeighbors[VD2: ClassManifest]( mapFunc: (Vid, EdgeWithVertices[VD, ED]) => Option[VD2], - reduceFunc: (VD2, VD2) => VD2, - gatherDirection: EdgeDirection) + mergeFunc: (VD2, VD2) => VD2, + direction: EdgeDirection) : RDD[(Vid, VD2)] + + /** + * This function is used to compute a statistic for the neighborhood of each + * vertex and returns a value for all vertices (including those without + * neighbors). + * + * This is one of the core functions in the Graph API in that enables + * neighborhood level computation. For example this function can be used to + * count neighbors satisfying a predicate or implement PageRank. + * + * @note Because the a default value is provided all vertices will have a + * corresponding entry in the returned RDD. + * + * @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 mergeFunc the function used to merge the results of each map + * operation. + * @param default the default value to use for each vertex if it has no + * neighbors or the map function repeatedly evaluates to none + * @param direction the direction of edges to consider (e.g., In, Out, Both). + * @tparam VD2 The returned type of the aggregation operation. + * + * @return A Spark.RDD containing tuples of vertex identifiers and + * their resulting value. There will be exactly one entry for ever vertex in + * the original graph. + * + * @example We can use this function to compute the average follower age + * for each user + * {{{ + * val graph: Graph[Int,Int] = loadGraph() + * val averageFollowerAge: RDD[(Int, Int)] = + * graph.aggregateNeigbhros[(Int,Double)]( + * (vid, edge) => (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} + * }}} + * + * @todo Should this return a graph with the new vertex values? + * + */ def aggregateNeighbors[VD2: ClassManifest]( mapFunc: (Vid, EdgeWithVertices[VD, ED]) => Option[VD2], reduceFunc: (VD2, VD2) => VD2, default: VD2, // Should this be a function or a value? - gatherDirection: EdgeDirection) + direction: EdgeDirection) : RDD[(Vid, VD2)] - def updateVertices[U: ClassManifest, VD2: ClassManifest]( - updates: RDD[(Vid, U)], - updateFunc: (Vertex[VD], Option[U]) => VD2) + + /** + * Join the vertices with an RDD and then apply a function from the the + * vertex and RDD entry to a new vertex value and type. The input table should + * contain at most one entry for each vertex. If no entry is provided the + * map function is invoked passing none. + * + * @tparam U the type of entry in the table of updates + * @tparam VD2 the new vertex value type + * @param tlb 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 + * {{{ + * val rawGraph: Graph[(),()] = Graph.textFile("webgraph") + * val outDeg: RDD[(Int, Int)] = rawGraph.outDegrees() + * val graph = rawGraph.leftJoinVertices[Int,Int](outDeg, + * (v, deg) => deg.getOrElse(0) ) + * }}} + * + * @todo Should this function be curried to enable type inference? For + * example + * {{{ + * graph.leftJoinVertices(tbl)( (v, row) => row.getOrElse(0) ) + * }}} + * @todo Is leftJoinVertices the right name? + */ + def leftJoinVertices[U: ClassManifest, VD2: ClassManifest]( + table: RDD[(Vid, U)], + mapFunc: (Vertex[VD], Option[U]) => VD2) : Graph[VD2, ED] - // This one can be used to skip records when we can do in-place update. - // Annoying that we can't rename it ... - def updateVertices2[U: ClassManifest]( - updates: RDD[(Vid, U)], - updateFunc: (Vertex[VD], U) => VD) + /** + * 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 should + * contain at most one entry for each vertex. If no entry is provided the + * map function is skipped and the old value is used. + * + * @tparam U the type of entry in the table of updates + * @param tlb 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 only for vertices with a 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") + * .mapVertices(v => 0) + * val outDeg: RDD[(Int, Int)] = rawGraph.outDegrees() + * val graph = rawGraph.leftJoinVertices[Int,Int](outDeg, + * (v, deg) => deg ) + * }}} + * + * @todo Should this function be curried to enable type inference? For + * example + * {{{ + * graph.joinVertices(tbl)( (v, row) => row ) + * }}} + */ + def joinVertices[U: ClassManifest]( + table: RDD[(Vid, U)], + mapFunc: (Vertex[VD], U) => VD) : Graph[VD, ED] // Save a copy of the GraphOps object so there is always one unique GraphOps object diff --git a/graph/src/main/scala/spark/graph/Pregel.scala b/graph/src/main/scala/spark/graph/Pregel.scala index 7f8849e4426c7..cd2400e5fafe9 100644 --- a/graph/src/main/scala/spark/graph/Pregel.scala +++ b/graph/src/main/scala/spark/graph/Pregel.scala @@ -25,7 +25,7 @@ object Pregel { var msgs: RDD[(Vid, A)] = g.vertices.map{ v => (v.id, initialMsg) } while (i < numIter) { - g = g.updateVertices(msgs, runProg).cache() + g = g.leftJoinVertices(msgs, runProg).cache() msgs = g.aggregateNeighbors(mapF, mergeMsg, EdgeDirection.In) i += 1 } diff --git a/graph/src/main/scala/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/spark/graph/impl/GraphImpl.scala index c362746d22b7c..4565f94e8339e 100644 --- a/graph/src/main/scala/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/spark/graph/impl/GraphImpl.scala @@ -90,6 +90,12 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( newGraph(vertices, edges.map(e => Edge(e.src, e.dst, f(e)))) } + override def mapEdgesWithVertices[ED2: ClassManifest](f: EdgeWithVertices[VD, ED] => ED2): + Graph[VD, ED2] = { + newGraph(vertices, edgesWithVertices.map(e => Edge(e.src.id, e.dst.id, f(e)))) + } + + ////////////////////////////////////////////////////////////////////////////////////////////////// // Lower level transformation methods ////////////////////////////////////////////////////////////////////////////////////////////////// @@ -202,7 +208,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( .combineByKey((v: VD2) => v, reduceFunc, null, vertexPartitioner, false) } - override def updateVertices[U: ClassManifest, VD2: ClassManifest]( + override def leftJoinVertices[U: ClassManifest, VD2: ClassManifest]( updates: RDD[(Vid, U)], updateF: (Vertex[VD], Option[U]) => VD2) : Graph[VD2, ED] = { @@ -219,7 +225,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( new GraphImpl(newVTable.partitions.size, eTable.partitions.size, null, null, newVTable, eTable) } - override def updateVertices2[U: ClassManifest]( + override def joinVertices[U: ClassManifest]( updates: RDD[(Vid, U)], updateF: (Vertex[VD], U) => VD) : Graph[VD, ED] = { From a80b28a579936251874b74e0d73c72425f8c273e Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Fri, 28 Jun 2013 18:18:51 -0700 Subject: [PATCH 037/531] Renamed several functions and classes and improved documentation --- graph/src/main/scala/spark/graph/Edge.scala | 6 +++++ .../scala/spark/graph/EdgeDirection.scala | 15 ++++++++++++ ...geWithVertices.scala => EdgeTriplet.scala} | 4 ++-- graph/src/main/scala/spark/graph/Graph.scala | 17 ++++++++----- graph/src/main/scala/spark/graph/Pregel.scala | 4 ++-- graph/src/main/scala/spark/graph/Vertex.scala | 6 ++++- ...VerticesRDD.scala => EdgeTripletRDD.scala} | 18 +++++++------- .../scala/spark/graph/impl/GraphImpl.scala | 24 +++++++++---------- 8 files changed, 62 insertions(+), 32 deletions(-) rename graph/src/main/scala/spark/graph/{EdgeWithVertices.scala => EdgeTriplet.scala} (67%) rename graph/src/main/scala/spark/graph/impl/{EdgeWithVerticesRDD.scala => EdgeTripletRDD.scala} (76%) diff --git a/graph/src/main/scala/spark/graph/Edge.scala b/graph/src/main/scala/spark/graph/Edge.scala index 8f022e812a500..cb057a467a475 100644 --- a/graph/src/main/scala/spark/graph/Edge.scala +++ b/graph/src/main/scala/spark/graph/Edge.scala @@ -1,6 +1,12 @@ package spark.graph +/** + * A single directed edge consisting of a source id, target id, + * and the data associated with the Edgee. + * + * @tparam ED type of the edge attribute + */ case class Edge[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED] ( var src: Vid = 0, var dst: Vid = 0, diff --git a/graph/src/main/scala/spark/graph/EdgeDirection.scala b/graph/src/main/scala/spark/graph/EdgeDirection.scala index a0c52c7038b3c..38caac44d6f82 100644 --- a/graph/src/main/scala/spark/graph/EdgeDirection.scala +++ b/graph/src/main/scala/spark/graph/EdgeDirection.scala @@ -1,6 +1,10 @@ package spark.graph +/** + * The direction of directed edge relative to a vertex used to select + * the set of adjacent neighbors when running a neighborhood query. + */ sealed abstract class EdgeDirection { def reverse: EdgeDirection = this match { case EdgeDirection.In => EdgeDirection.In @@ -11,7 +15,18 @@ sealed abstract class EdgeDirection { object EdgeDirection { + /** + * Edges arriving at a vertex. + */ case object In extends EdgeDirection + + /** + * Edges originating from a vertex + */ case object Out extends EdgeDirection + + /** + * All edges adjacent to a vertex + */ case object Both extends EdgeDirection } diff --git a/graph/src/main/scala/spark/graph/EdgeWithVertices.scala b/graph/src/main/scala/spark/graph/EdgeTriplet.scala similarity index 67% rename from graph/src/main/scala/spark/graph/EdgeWithVertices.scala rename to graph/src/main/scala/spark/graph/EdgeTriplet.scala index a731f73709cf6..9a819481ba879 100644 --- a/graph/src/main/scala/spark/graph/EdgeWithVertices.scala +++ b/graph/src/main/scala/spark/graph/EdgeTriplet.scala @@ -1,8 +1,8 @@ package spark.graph -class EdgeWithVertices[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) VD, - @specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED] { +class EdgeTriplet[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) VD, + @specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED] { var src: Vertex[VD] = _ var dst: Vertex[VD] = _ var data: ED = _ diff --git a/graph/src/main/scala/spark/graph/Graph.scala b/graph/src/main/scala/spark/graph/Graph.scala index 421055d319e64..d0742a1c8fddd 100644 --- a/graph/src/main/scala/spark/graph/Graph.scala +++ b/graph/src/main/scala/spark/graph/Graph.scala @@ -20,6 +20,8 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { /** * Get the vertices and their data. * + * @return An RDD containing the vertices in this graph + * * @see Vertex for the vertex type. * * @todo should vertices return tuples instead of vertex objects? @@ -30,13 +32,14 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * Get the Edges and their data as an RDD. The entries in the RDD contain * just the source id and target id along with the edge data. * + * @return An RDD containing the edges in this graph * * @see Edge for the edge type. * @see edgesWithVertices to get an RDD which contains all the edges along * with their vertex data. * * @todo Should edges return 3 tuples instead of Edge objects? In this case - * we could rename EdgeWithVertices to Edge? + * we could rename EdgeTriplet to Edge? */ def edges(): RDD[Edge[ED]] @@ -44,6 +47,8 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * Get the edges with the vertex data associated with the adjacent pair of * vertices. * + * @return An RDD containing edge triplets. + * * @example This operation might be used to evaluate a graph coloring where * we would like to check that both vertices are a different color. * {{{ @@ -56,7 +61,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * @see edges() If only the edge data and adjacent vertex ids are required. * */ - def edgesWithVertices(): RDD[EdgeWithVertices[VD, ED]] + def triplets(): RDD[EdgeTriplet[VD, ED]] /** * Return a graph that is cached when first created. This is used to pin a @@ -133,8 +138,8 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * }}} * */ - def mapEdgesWithVertices[ED2: ClassManifest]( - map: EdgeWithVertices[VD, ED] => ED2): Graph[VD, ED2] + def mapTriplets[ED2: ClassManifest]( + map: EdgeTriplet[VD, ED] => ED2): Graph[VD, ED2] /** @@ -183,7 +188,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * */ def aggregateNeighbors[VD2: ClassManifest]( - mapFunc: (Vid, EdgeWithVertices[VD, ED]) => Option[VD2], + mapFunc: (Vid, EdgeTriplet[VD, ED]) => Option[VD2], mergeFunc: (VD2, VD2) => VD2, direction: EdgeDirection) : RDD[(Vid, VD2)] @@ -232,7 +237,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * */ def aggregateNeighbors[VD2: ClassManifest]( - mapFunc: (Vid, EdgeWithVertices[VD, ED]) => Option[VD2], + mapFunc: (Vid, EdgeTriplet[VD, ED]) => Option[VD2], reduceFunc: (VD2, VD2) => VD2, default: VD2, // Should this be a function or a value? direction: EdgeDirection) diff --git a/graph/src/main/scala/spark/graph/Pregel.scala b/graph/src/main/scala/spark/graph/Pregel.scala index cd2400e5fafe9..699affba81661 100644 --- a/graph/src/main/scala/spark/graph/Pregel.scala +++ b/graph/src/main/scala/spark/graph/Pregel.scala @@ -8,7 +8,7 @@ object Pregel { def iterate[VD: ClassManifest, ED: ClassManifest, A: ClassManifest](graph: Graph[VD, ED])( vprog: ( Vertex[VD], A) => VD, - sendMsg: (Vid, EdgeWithVertices[VD, ED]) => Option[A], + sendMsg: (Vid, EdgeTriplet[VD, ED]) => Option[A], mergeMsg: (A, A) => A, initialMsg: A, numIter: Int) : Graph[VD, ED] = { @@ -16,7 +16,7 @@ object Pregel { var g = graph.cache var i = 0 - def mapF(vid: Vid, edge: EdgeWithVertices[VD,ED]) = sendMsg(edge.otherVertex(vid).id, edge) + def mapF(vid: Vid, edge: EdgeTriplet[VD,ED]) = sendMsg(edge.otherVertex(vid).id, edge) def runProg(v: Vertex[VD], msg: Option[A]): VD = { if (msg.isEmpty) v.data else vprog(v, msg.get) diff --git a/graph/src/main/scala/spark/graph/Vertex.scala b/graph/src/main/scala/spark/graph/Vertex.scala index 543cc8e942778..5323643beceee 100644 --- a/graph/src/main/scala/spark/graph/Vertex.scala +++ b/graph/src/main/scala/spark/graph/Vertex.scala @@ -1,6 +1,10 @@ package spark.graph - +/** + * A graph vertex consists of a vertex id and attribute. + * + * @tparam VD the type of the vertex attribute. + */ case class Vertex[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) VD] ( var id: Vid = 0, var data: VD = nullValue[VD]) { diff --git a/graph/src/main/scala/spark/graph/impl/EdgeWithVerticesRDD.scala b/graph/src/main/scala/spark/graph/impl/EdgeTripletRDD.scala similarity index 76% rename from graph/src/main/scala/spark/graph/impl/EdgeWithVerticesRDD.scala rename to graph/src/main/scala/spark/graph/impl/EdgeTripletRDD.scala index 5370b4e1604f6..f157dd40560b0 100644 --- a/graph/src/main/scala/spark/graph/impl/EdgeWithVerticesRDD.scala +++ b/graph/src/main/scala/spark/graph/impl/EdgeTripletRDD.scala @@ -7,7 +7,7 @@ import spark.graph._ private[graph] -class EdgeWithVerticesPartition(idx: Int, val vPart: Partition, val ePart: Partition) +class EdgeTripletPartition(idx: Int, val vPart: Partition, val ePart: Partition) extends Partition { override val index: Int = idx override def hashCode(): Int = idx @@ -18,10 +18,10 @@ class EdgeWithVerticesPartition(idx: Int, val vPart: Partition, val ePart: Parti * A RDD that brings together edge data with its associated vertex data. */ private[graph] -class EdgeWithVerticesRDD[VD: ClassManifest, ED: ClassManifest]( +class EdgeTripletRDD[VD: ClassManifest, ED: ClassManifest]( vTableReplicated: RDD[(Vid, VD)], eTable: RDD[(Pid, EdgePartition[ED])]) - extends RDD[(VertexHashMap[VD], Iterator[EdgeWithVertices[VD, ED]])](eTable.context, Nil) { + extends RDD[(VertexHashMap[VD], Iterator[EdgeTriplet[VD, ED]])](eTable.context, Nil) { println(vTableReplicated.partitioner.get.numPartitions) println(eTable.partitioner.get.numPartitions) @@ -33,18 +33,18 @@ class EdgeWithVerticesRDD[VD: ClassManifest, ED: ClassManifest]( } override def getPartitions = Array.tabulate[Partition](eTable.partitions.size) { - i => new EdgeWithVerticesPartition(i, eTable.partitions(i), vTableReplicated.partitions(i)) + i => new EdgeTripletPartition(i, eTable.partitions(i), vTableReplicated.partitions(i)) } override val partitioner = eTable.partitioner override def getPreferredLocations(s: Partition) = - eTable.preferredLocations(s.asInstanceOf[EdgeWithVerticesPartition].ePart) + eTable.preferredLocations(s.asInstanceOf[EdgeTripletPartition].ePart) override def compute(s: Partition, context: TaskContext) - : Iterator[(VertexHashMap[VD], Iterator[EdgeWithVertices[VD, ED]])] = { + : Iterator[(VertexHashMap[VD], Iterator[EdgeTriplet[VD, ED]])] = { - val split = s.asInstanceOf[EdgeWithVerticesPartition] + val split = s.asInstanceOf[EdgeTripletPartition] // Fetch the vertices and put them in a hashmap. // TODO: use primitive hashmaps for primitive VD types. @@ -55,9 +55,9 @@ class EdgeWithVerticesRDD[VD: ClassManifest, ED: ClassManifest]( .asInstanceOf[(Pid, EdgePartition[ED])] // Return an iterator that looks up the hash map to find matching vertices for each edge. - val iter = new Iterator[EdgeWithVertices[VD, ED]] { + val iter = new Iterator[EdgeTriplet[VD, ED]] { private var pos = 0 - private val e = new EdgeWithVertices[VD, ED] + private val e = new EdgeTriplet[VD, ED] e.src = new Vertex[VD] e.dst = new Vertex[VD] diff --git a/graph/src/main/scala/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/spark/graph/impl/GraphImpl.scala index 4565f94e8339e..6ee7b8a062093 100644 --- a/graph/src/main/scala/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/spark/graph/impl/GraphImpl.scala @@ -78,8 +78,8 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( } /** Return a RDD that brings edges with its source and destination vertices together. */ - override def edgesWithVertices: RDD[EdgeWithVertices[VD, ED]] = { - (new EdgeWithVerticesRDD(vTableReplicated, eTable)).mapPartitions { part => part.next._2 } + override def triplets: RDD[EdgeTriplet[VD, ED]] = { + (new EdgeTripletRDD(vTableReplicated, eTable)).mapPartitions { part => part.next._2 } } override def mapVertices[VD2: ClassManifest](f: Vertex[VD] => VD2): Graph[VD2, ED] = { @@ -90,9 +90,9 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( newGraph(vertices, edges.map(e => Edge(e.src, e.dst, f(e)))) } - override def mapEdgesWithVertices[ED2: ClassManifest](f: EdgeWithVertices[VD, ED] => ED2): + override def mapTriplets[ED2: ClassManifest](f: EdgeTriplet[VD, ED] => ED2): Graph[VD, ED2] = { - newGraph(vertices, edgesWithVertices.map(e => Edge(e.src.id, e.dst.id, f(e)))) + newGraph(vertices, triplets.map(e => Edge(e.src.id, e.dst.id, f(e)))) } @@ -101,7 +101,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( ////////////////////////////////////////////////////////////////////////////////////////////////// override def aggregateNeighbors[VD2: ClassManifest]( - mapFunc: (Vid, EdgeWithVertices[VD, ED]) => Option[VD2], + mapFunc: (Vid, EdgeTriplet[VD, ED]) => Option[VD2], reduceFunc: (VD2, VD2) => VD2, default: VD2, gatherDirection: EdgeDirection) @@ -114,13 +114,13 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( part.map { v => (v._1, MutableTuple2(v._2, Option.empty[VD2])) } }, preservesPartitioning = true) - (new EdgeWithVerticesRDD[MutableTuple2[VD, Option[VD2]], ED](newVTable, eTable)) + (new EdgeTripletRDD[MutableTuple2[VD, Option[VD2]], ED](newVTable, eTable)) .mapPartitions { part => val (vmap, edges) = part.next() - val edgeSansAcc = new EdgeWithVertices[VD, ED]() + val edgeSansAcc = new EdgeTriplet[VD, ED]() edgeSansAcc.src = new Vertex[VD] edgeSansAcc.dst = new Vertex[VD] - edges.foreach { e: EdgeWithVertices[MutableTuple2[VD, Option[VD2]], ED] => + edges.foreach { e: EdgeTriplet[MutableTuple2[VD, Option[VD2]], ED] => edgeSansAcc.data = e.data edgeSansAcc.src.data = e.src.data._1 edgeSansAcc.dst.data = e.dst.data._1 @@ -158,7 +158,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( * As a consequence, the resulting table may be much smaller than the set of vertices. */ override def aggregateNeighbors[VD2: ClassManifest]( - mapFunc: (Vid, EdgeWithVertices[VD, ED]) => Option[VD2], + mapFunc: (Vid, EdgeTriplet[VD, ED]) => Option[VD2], reduceFunc: (VD2, VD2) => VD2, gatherDirection: EdgeDirection): RDD[(Vid, VD2)] = { @@ -169,13 +169,13 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( part.map { v => (v._1, MutableTuple2(v._2, Option.empty[VD2])) } }, preservesPartitioning = true) - (new EdgeWithVerticesRDD[MutableTuple2[VD, Option[VD2]], ED](newVTable, eTable)) + (new EdgeTripletRDD[MutableTuple2[VD, Option[VD2]], ED](newVTable, eTable)) .mapPartitions { part => val (vmap, edges) = part.next() - val edgeSansAcc = new EdgeWithVertices[VD, ED]() + val edgeSansAcc = new EdgeTriplet[VD, ED]() edgeSansAcc.src = new Vertex[VD] edgeSansAcc.dst = new Vertex[VD] - edges.foreach { e: EdgeWithVertices[MutableTuple2[VD, Option[VD2]], ED] => + edges.foreach { e: EdgeTriplet[MutableTuple2[VD, Option[VD2]], ED] => edgeSansAcc.data = e.data edgeSansAcc.src.data = e.src.data._1 edgeSansAcc.dst.data = e.dst.data._1 From ae12d163dc2462ededefc8d31900803cf9a782a5 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 29 Jun 2013 15:22:15 -0700 Subject: [PATCH 038/531] Added the BytecodeUtils class for analyzing bytecode. --- graph/src/main/scala/spark/graph/Graph.scala | 6 +- .../spark/graph/util/BytecodeUtils.scala | 113 ++++++++++++++++++ .../test/scala/spark/graph/GraphSuite.scala | 74 ++++++------ .../spark/graph/util/BytecodeUtilsSuite.scala | 93 ++++++++++++++ 4 files changed, 246 insertions(+), 40 deletions(-) create mode 100644 graph/src/main/scala/spark/graph/util/BytecodeUtils.scala create mode 100644 graph/src/test/scala/spark/graph/util/BytecodeUtilsSuite.scala diff --git a/graph/src/main/scala/spark/graph/Graph.scala b/graph/src/main/scala/spark/graph/Graph.scala index 421055d319e64..7d296bc9dc8d2 100644 --- a/graph/src/main/scala/spark/graph/Graph.scala +++ b/graph/src/main/scala/spark/graph/Graph.scala @@ -204,7 +204,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * @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 mergeFunc the function used to merge the results of each map + * @param reduceFunc the function used to merge the results of each map * operation. * @param default the default value to use for each vertex if it has no * neighbors or the map function repeatedly evaluates to none @@ -247,7 +247,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * * @tparam U the type of entry in the table of updates * @tparam VD2 the new vertex value type - * @param tlb the table to join with the vertices in the graph. The table + * @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 @@ -282,7 +282,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * map function is skipped and the old value is used. * * @tparam U the type of entry in the table of updates - * @param tlb the table to join with the vertices in the graph. The table + * @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 only for vertices with a corresponding entry in diff --git a/graph/src/main/scala/spark/graph/util/BytecodeUtils.scala b/graph/src/main/scala/spark/graph/util/BytecodeUtils.scala new file mode 100644 index 0000000000000..268a3c2bcf571 --- /dev/null +++ b/graph/src/main/scala/spark/graph/util/BytecodeUtils.scala @@ -0,0 +1,113 @@ +package spark.graph.util + +import java.io.{ByteArrayInputStream, ByteArrayOutputStream} + +import scala.collection.mutable.HashSet + +import org.objectweb.asm.{ClassReader, MethodVisitor} +import org.objectweb.asm.commons.EmptyVisitor +import org.objectweb.asm.Opcodes._ + +import spark.Utils + + +private[graph] object BytecodeUtils { + + /** + * Test whether the given closure invokes the specified method in the specified class. + */ + def invokedMethod(closure: AnyRef, targetClass: Class[_], targetMethod: String): Boolean = { + if (_invokedMethod(closure.getClass, "apply", targetClass, targetMethod)) { + true + } else { + // look at closures enclosed in this closure + for (f <- closure.getClass.getDeclaredFields + if f.getType.getName.startsWith("scala.Function")) { + f.setAccessible(true) + if (invokedMethod(f.get(closure), targetClass, targetMethod)) { + return true + } + } + return false + } + } + + private def _invokedMethod(cls: Class[_], method: String, + targetClass: Class[_], targetMethod: String): Boolean = { + + val seen = new HashSet[(Class[_], String)] + var stack = List[(Class[_], String)]((cls, method)) + + while (stack.nonEmpty) { + val (c, m) = stack.head + stack = stack.tail + seen.add((c, m)) + val finder = new MethodInvocationFinder(c.getName, m) + getClassReader(c).accept(finder, 0) + for (classMethod <- finder.methodsInvoked) { + println(classMethod) + if (classMethod._1 == targetClass && classMethod._2 == targetMethod) { + return true + } else if (!seen.contains(classMethod)) { + stack = classMethod :: stack + } + } + } + return false + } + + /** + * Get an ASM class reader for a given class from the JAR that loaded it. + */ + private def getClassReader(cls: Class[_]): ClassReader = { + // Copy data over, before delegating to ClassReader - else we can run out of open file handles. + val className = cls.getName.replaceFirst("^.*\\.", "") + ".class" + val resourceStream = cls.getResourceAsStream(className) + // todo: Fixme - continuing with earlier behavior ... + if (resourceStream == null) return new ClassReader(resourceStream) + + val baos = new ByteArrayOutputStream(128) + Utils.copyStream(resourceStream, baos, true) + new ClassReader(new ByteArrayInputStream(baos.toByteArray)) + } + + /** + * Given the class name, return whether we should look into the class or not. This is used to + * skip examing a large quantity of Java or Scala classes that we know for sure wouldn't access + * the closures. Note that the class name is expected in ASM style (i.e. use "/" instead of "."). + */ + private def skipClass(className: String): Boolean = { + val c = className + c.startsWith("java/") || c.startsWith("scala/") || c.startsWith("javax/") + } + + /** + * Find the set of methods invoked by the specified method in the specified class. + * For example, after running the visitor, + * MethodInvocationFinder("spark/graph/Foo", "test") + * its methodsInvoked variable will contain the set of methods invoked directly by + * Foo.test(). Interface invocations are not returned as part of the result set because we cannot + * determine the actual metod invoked by inspecting the bytecode. + */ + private class MethodInvocationFinder(className: String, methodName: String) extends EmptyVisitor { + + val methodsInvoked = new HashSet[(Class[_], String)] + + override def visitMethod(access: Int, name: String, desc: String, + sig: String, exceptions: Array[String]): MethodVisitor = { + if (name == methodName) { + new EmptyVisitor { + override def visitMethodInsn(op: Int, owner: String, name: String, desc: String) { + if (op == INVOKEVIRTUAL || op == INVOKESPECIAL || op == INVOKESTATIC) { + if (!skipClass(owner)) { + methodsInvoked.add((Class.forName(owner.replace("/", ".")), name)) + } + } + } + } + } else { + null + } + } + } +} diff --git a/graph/src/test/scala/spark/graph/GraphSuite.scala b/graph/src/test/scala/spark/graph/GraphSuite.scala index 64a7aa063b057..4eb469a71f417 100644 --- a/graph/src/test/scala/spark/graph/GraphSuite.scala +++ b/graph/src/test/scala/spark/graph/GraphSuite.scala @@ -6,41 +6,41 @@ import spark.SparkContext class GraphSuite extends FunSuite with LocalSparkContext { - - test("graph partitioner") { - sc = new SparkContext("local", "test") - val vertices = sc.parallelize(Seq(Vertex(1, "one"), Vertex(2, "two"))) - val edges = sc.parallelize(Seq(Edge(1, 2, "onlyedge"))) - var g = new Graph(vertices, edges) - - g = g.withPartitioner(4, 7) - assert(g.numVertexPartitions === 4) - assert(g.numEdgePartitions === 7) - - g = g.withVertexPartitioner(5) - assert(g.numVertexPartitions === 5) - - g = g.withEdgePartitioner(8) - assert(g.numEdgePartitions === 8) - - g = g.mapVertices(x => x) - assert(g.numVertexPartitions === 5) - assert(g.numEdgePartitions === 8) - - g = g.mapEdges(x => x) - assert(g.numVertexPartitions === 5) - assert(g.numEdgePartitions === 8) - - val updates = sc.parallelize(Seq((1, " more"))) - g = g.updateVertices( - updates, - (v, u: Option[String]) => if (u.isDefined) v.data + u.get else v.data) - assert(g.numVertexPartitions === 5) - assert(g.numEdgePartitions === 8) - - g = g.reverse - assert(g.numVertexPartitions === 5) - assert(g.numEdgePartitions === 8) - - } +// +// test("graph partitioner") { +// sc = new SparkContext("local", "test") +// val vertices = sc.parallelize(Seq(Vertex(1, "one"), Vertex(2, "two"))) +// val edges = sc.parallelize(Seq(Edge(1, 2, "onlyedge"))) +// var g = new Graph(vertices, edges) +// +// g = g.withPartitioner(4, 7) +// assert(g.numVertexPartitions === 4) +// assert(g.numEdgePartitions === 7) +// +// g = g.withVertexPartitioner(5) +// assert(g.numVertexPartitions === 5) +// +// g = g.withEdgePartitioner(8) +// assert(g.numEdgePartitions === 8) +// +// g = g.mapVertices(x => x) +// assert(g.numVertexPartitions === 5) +// assert(g.numEdgePartitions === 8) +// +// g = g.mapEdges(x => x) +// assert(g.numVertexPartitions === 5) +// assert(g.numEdgePartitions === 8) +// +// val updates = sc.parallelize(Seq((1, " more"))) +// g = g.updateVertices( +// updates, +// (v, u: Option[String]) => if (u.isDefined) v.data + u.get else v.data) +// assert(g.numVertexPartitions === 5) +// assert(g.numEdgePartitions === 8) +// +// g = g.reverse +// assert(g.numVertexPartitions === 5) +// assert(g.numEdgePartitions === 8) +// +// } } diff --git a/graph/src/test/scala/spark/graph/util/BytecodeUtilsSuite.scala b/graph/src/test/scala/spark/graph/util/BytecodeUtilsSuite.scala new file mode 100644 index 0000000000000..8d18cf39e8919 --- /dev/null +++ b/graph/src/test/scala/spark/graph/util/BytecodeUtilsSuite.scala @@ -0,0 +1,93 @@ +package spark.graph.util + +import org.scalatest.FunSuite + + +class BytecodeUtilsSuite extends FunSuite { + + import BytecodeUtilsSuite.TestClass + + test("closure invokes a method") { + val c1 = {e: TestClass => println(e.foo); println(e.bar); println(e.baz); } + assert(BytecodeUtils.invokedMethod(c1, classOf[TestClass], "foo")) + assert(BytecodeUtils.invokedMethod(c1, classOf[TestClass], "bar")) + assert(BytecodeUtils.invokedMethod(c1, classOf[TestClass], "baz")) + + val c2 = {e: TestClass => println(e.foo); println(e.bar); } + assert(BytecodeUtils.invokedMethod(c2, classOf[TestClass], "foo")) + assert(BytecodeUtils.invokedMethod(c2, classOf[TestClass], "bar")) + assert(!BytecodeUtils.invokedMethod(c2, classOf[TestClass], "baz")) + + val c3 = {e: TestClass => println(e.foo); } + assert(BytecodeUtils.invokedMethod(c3, classOf[TestClass], "foo")) + assert(!BytecodeUtils.invokedMethod(c3, classOf[TestClass], "bar")) + assert(!BytecodeUtils.invokedMethod(c3, classOf[TestClass], "baz")) + } + + test("closure inside a closure invokes a method") { + val c1 = {e: TestClass => println(e.foo); println(e.bar); println(e.baz); } + val c2 = {e: TestClass => c1(e); println(e.foo); } + assert(BytecodeUtils.invokedMethod(c2, classOf[TestClass], "foo")) + assert(BytecodeUtils.invokedMethod(c2, classOf[TestClass], "bar")) + assert(BytecodeUtils.invokedMethod(c2, classOf[TestClass], "baz")) + } + + test("closure inside a closure inside a closure invokes a method") { + val c1 = {e: TestClass => println(e.baz); } + val c2 = {e: TestClass => c1(e); println(e.foo); } + val c3 = {e: TestClass => c2(e) } + assert(BytecodeUtils.invokedMethod(c3, classOf[TestClass], "foo")) + assert(!BytecodeUtils.invokedMethod(c3, classOf[TestClass], "bar")) + assert(BytecodeUtils.invokedMethod(c3, classOf[TestClass], "baz")) + } + + test("closure calling a function that invokes a method") { + def zoo(e: TestClass) { + println(e.baz) + } + val c1 = {e: TestClass => zoo(e)} + assert(!BytecodeUtils.invokedMethod(c1, classOf[TestClass], "foo")) + assert(!BytecodeUtils.invokedMethod(c1, classOf[TestClass], "bar")) + assert(BytecodeUtils.invokedMethod(c1, classOf[TestClass], "baz")) + } + + test("closure calling a function that invokes a method which uses another closure") { + val c2 = {e: TestClass => println(e.baz)} + def zoo(e: TestClass) { + c2(e) + } + val c1 = {e: TestClass => zoo(e)} + assert(!BytecodeUtils.invokedMethod(c1, classOf[TestClass], "foo")) + assert(!BytecodeUtils.invokedMethod(c1, classOf[TestClass], "bar")) + assert(BytecodeUtils.invokedMethod(c1, classOf[TestClass], "baz")) + } + + test("nested closure") { + val c2 = {e: TestClass => println(e.baz)} + def zoo(e: TestClass, c: TestClass => Unit) { + c(e) + } + val c1 = {e: TestClass => zoo(e, c2)} + assert(!BytecodeUtils.invokedMethod(c1, classOf[TestClass], "foo")) + assert(!BytecodeUtils.invokedMethod(c1, classOf[TestClass], "bar")) + assert(BytecodeUtils.invokedMethod(c1, classOf[TestClass], "baz")) + } + + // The following doesn't work yet, because the byte code doesn't contain any information + // about what exactly "c" is. +// test("invoke interface") { +// val c1 = {e: TestClass => c(e)} +// assert(!BytecodeUtils.invokedMethod(c1, classOf[TestClass], "foo")) +// assert(!BytecodeUtils.invokedMethod(c1, classOf[TestClass], "bar")) +// assert(BytecodeUtils.invokedMethod(c1, classOf[TestClass], "baz")) +// } + + private val c = {e: TestClass => println(e.baz)} +} + + +object BytecodeUtilsSuite { + class TestClass(val foo: Int, val bar: Long) { + def baz: Boolean = false + } +} From 758ceff778a5590297fe9b712d9642f009f9b628 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 29 Jun 2013 15:39:48 -0700 Subject: [PATCH 039/531] Updated BytecodeUtils to ASM4. --- .../main/scala/spark/graph/util/BytecodeUtils.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/graph/src/main/scala/spark/graph/util/BytecodeUtils.scala b/graph/src/main/scala/spark/graph/util/BytecodeUtils.scala index 268a3c2bcf571..ac3a1fb9573fa 100644 --- a/graph/src/main/scala/spark/graph/util/BytecodeUtils.scala +++ b/graph/src/main/scala/spark/graph/util/BytecodeUtils.scala @@ -4,8 +4,7 @@ import java.io.{ByteArrayInputStream, ByteArrayOutputStream} import scala.collection.mutable.HashSet -import org.objectweb.asm.{ClassReader, MethodVisitor} -import org.objectweb.asm.commons.EmptyVisitor +import org.objectweb.asm.{ClassReader, ClassVisitor, MethodVisitor} import org.objectweb.asm.Opcodes._ import spark.Utils @@ -45,7 +44,7 @@ private[graph] object BytecodeUtils { val finder = new MethodInvocationFinder(c.getName, m) getClassReader(c).accept(finder, 0) for (classMethod <- finder.methodsInvoked) { - println(classMethod) + //println(classMethod) if (classMethod._1 == targetClass && classMethod._2 == targetMethod) { return true } else if (!seen.contains(classMethod)) { @@ -89,14 +88,15 @@ private[graph] object BytecodeUtils { * Foo.test(). Interface invocations are not returned as part of the result set because we cannot * determine the actual metod invoked by inspecting the bytecode. */ - private class MethodInvocationFinder(className: String, methodName: String) extends EmptyVisitor { + private class MethodInvocationFinder(className: String, methodName: String) + extends ClassVisitor(ASM4) { val methodsInvoked = new HashSet[(Class[_], String)] override def visitMethod(access: Int, name: String, desc: String, sig: String, exceptions: Array[String]): MethodVisitor = { if (name == methodName) { - new EmptyVisitor { + new MethodVisitor(ASM4) { override def visitMethodInsn(op: Int, owner: String, name: String, desc: String) { if (op == INVOKEVIRTUAL || op == INVOKESPECIAL || op == INVOKESTATIC) { if (!skipClass(owner)) { From 79b5eaa4e2a32817a50e583554a53ed7ab72e0b2 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 29 Jun 2013 15:58:59 -0700 Subject: [PATCH 040/531] Added a 64bit string hash function. --- .../scala/spark/graph/util/HashUtils.scala | 21 +++++++++++++++++++ 1 file changed, 21 insertions(+) create mode 100644 graph/src/main/scala/spark/graph/util/HashUtils.scala diff --git a/graph/src/main/scala/spark/graph/util/HashUtils.scala b/graph/src/main/scala/spark/graph/util/HashUtils.scala new file mode 100644 index 0000000000000..0dfaef4c48dda --- /dev/null +++ b/graph/src/main/scala/spark/graph/util/HashUtils.scala @@ -0,0 +1,21 @@ +package spark.graph.util + + +object HashUtils { + + /** + * Compute a 64-bit hash value for the given string. + * See http://stackoverflow.com/questions/1660501/what-is-a-good-64bit-hash-function-in-java-for-textual-strings + */ + def hash(str: String): Long = { + var h = 1125899906842597L + val len = str.length + var i = 0 + + while (i < len) { + h = 31 * h + str(i) + i += 1 + } + h + } +} From 2964df7a4e9d39c6206709680b9828e0cb7b1430 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Sat, 29 Jun 2013 21:04:35 -0700 Subject: [PATCH 041/531] Commenting out unused test code. --- .../test/scala/spark/graph/GraphSuite.scala | 72 +++++++++---------- 1 file changed, 36 insertions(+), 36 deletions(-) diff --git a/graph/src/test/scala/spark/graph/GraphSuite.scala b/graph/src/test/scala/spark/graph/GraphSuite.scala index 64a7aa063b057..3d250aa18ce09 100644 --- a/graph/src/test/scala/spark/graph/GraphSuite.scala +++ b/graph/src/test/scala/spark/graph/GraphSuite.scala @@ -7,40 +7,40 @@ import spark.SparkContext class GraphSuite extends FunSuite with LocalSparkContext { - test("graph partitioner") { - sc = new SparkContext("local", "test") - val vertices = sc.parallelize(Seq(Vertex(1, "one"), Vertex(2, "two"))) - val edges = sc.parallelize(Seq(Edge(1, 2, "onlyedge"))) - var g = new Graph(vertices, edges) - - g = g.withPartitioner(4, 7) - assert(g.numVertexPartitions === 4) - assert(g.numEdgePartitions === 7) - - g = g.withVertexPartitioner(5) - assert(g.numVertexPartitions === 5) - - g = g.withEdgePartitioner(8) - assert(g.numEdgePartitions === 8) - - g = g.mapVertices(x => x) - assert(g.numVertexPartitions === 5) - assert(g.numEdgePartitions === 8) - - g = g.mapEdges(x => x) - assert(g.numVertexPartitions === 5) - assert(g.numEdgePartitions === 8) - - val updates = sc.parallelize(Seq((1, " more"))) - g = g.updateVertices( - updates, - (v, u: Option[String]) => if (u.isDefined) v.data + u.get else v.data) - assert(g.numVertexPartitions === 5) - assert(g.numEdgePartitions === 8) - - g = g.reverse - assert(g.numVertexPartitions === 5) - assert(g.numEdgePartitions === 8) - - } +// test("graph partitioner") { +// sc = new SparkContext("local", "test") +// val vertices = sc.parallelize(Seq(Vertex(1, "one"), Vertex(2, "two"))) +// val edges = sc.parallelize(Seq(Edge(1, 2, "onlyedge"))) +// var g = Graph(vertices, edges) +// +// g = g.withPartitioner(4, 7) +// assert(g.numVertexPartitions === 4) +// assert(g.numEdgePartitions === 7) +// +// g = g.withVertexPartitioner(5) +// assert(g.numVertexPartitions === 5) +// +// g = g.withEdgePartitioner(8) +// assert(g.numEdgePartitions === 8) +// +// g = g.mapVertices(x => x) +// assert(g.numVertexPartitions === 5) +// assert(g.numEdgePartitions === 8) +// +// g = g.mapEdges(x => x) +// assert(g.numVertexPartitions === 5) +// assert(g.numEdgePartitions === 8) +// +// val updates = sc.parallelize(Seq((1, " more"))) +// g = g.updateVertices( +// updates, +// (v, u: Option[String]) => if (u.isDefined) v.data + u.get else v.data) +// assert(g.numVertexPartitions === 5) +// assert(g.numEdgePartitions === 8) +// +// g = g.reverse +// assert(g.numVertexPartitions === 5) +// assert(g.numEdgePartitions === 8) +// +// } } From f24548da88cb4283f5214643de4c9a0320db4143 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Sat, 29 Jun 2013 21:04:53 -0700 Subject: [PATCH 042/531] Adding graph cosntruction code to graph singleton object. --- graph/src/main/scala/spark/graph/Graph.scala | 22 ++++++++++++++++++++ 1 file changed, 22 insertions(+) diff --git a/graph/src/main/scala/spark/graph/Graph.scala b/graph/src/main/scala/spark/graph/Graph.scala index d0742a1c8fddd..359f295f5680e 100644 --- a/graph/src/main/scala/spark/graph/Graph.scala +++ b/graph/src/main/scala/spark/graph/Graph.scala @@ -326,5 +326,27 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { object Graph { + import spark.graph.impl._ + import spark.SparkContext._ + + def apply(rawEdges: RDD[(Int,Int)], uniqueEdges: Boolean = true): Graph[Int, Int] = { + // Reduce to unique edges + val edges = + if(uniqueEdges) rawEdges.map{ case (s,t) => ((s,t),1) }.reduceByKey( _ + _ ) + .map{ case ((s,t), cnt) => Edge(s,t,cnt) } + else rawEdges.map{ case (s,t) => Edge(s,t,1) } + // Determine unique vertices + val vertices = edges.flatMap{ case Edge(s, t, cnt) => Array((s,1), (t,1)) }.reduceByKey( _ + _ ) + .map{ case (id, deg) => Vertex(id, deg) } + // Return graph + new GraphImpl(vertices, edges) + } + + def apply[VD: ClassManifest, ED: ClassManifest](vertices: RDD[Vertex[VD]], edges: RDD[Edge[ED]]): Graph[VD, ED] = { + new GraphImpl(vertices, edges) + + } + + implicit def graphToGraphOps[VD: ClassManifest, ED: ClassManifest](g: Graph[VD, ED]) = g.ops } From ce7e270bb4c6374b0ad0cce0eabd4ebf1566718a Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 29 Jun 2013 21:28:22 -0700 Subject: [PATCH 043/531] Added graph package to the classpath. --- bin/compute-classpath.sh | 2 ++ 1 file changed, 2 insertions(+) diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index 3a78880290455..4b772e6f2bc57 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -18,6 +18,7 @@ REPL_DIR="$FWDIR/repl" REPL_BIN_DIR="$FWDIR/repl-bin" EXAMPLES_DIR="$FWDIR/examples" BAGEL_DIR="$FWDIR/bagel" +GRAPH_DIR="$FWDIR/graph" STREAMING_DIR="$FWDIR/streaming" PYSPARK_DIR="$FWDIR/python" @@ -49,6 +50,7 @@ if [ -e $REPL_BIN_DIR/target ]; then CLASSPATH+=":$EXAMPLES_JAR" fi CLASSPATH="$CLASSPATH:$BAGEL_DIR/target/scala-$SCALA_VERSION/classes" +CLASSPATH="$CLASSPATH:$GRAPH_DIR/target/scala-$SCALA_VERSION/classes" for jar in `find $PYSPARK_DIR/lib -name '*jar'`; do CLASSPATH="$CLASSPATH:$jar" done From 6acc2a7b3d3c2a1b449c2c50ece7ad34b0863ed1 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 29 Jun 2013 21:28:31 -0700 Subject: [PATCH 044/531] Various minor changes. --- .../src/main/scala/spark/graph/Analytics.scala | 4 ++++ graph/src/main/scala/spark/graph/Graph.scala | 6 +++--- graph/src/main/scala/spark/graph/GraphOps.scala | 4 ++-- graph/src/main/scala/spark/graph/Pregel.scala | 17 +++++++++-------- graph/src/main/scala/spark/graph/Vertex.scala | 2 +- .../scala/spark/graph/impl/EdgePartition.scala | 8 ++++---- .../main/scala/spark/graph/impl/GraphImpl.scala | 12 ++++-------- 7 files changed, 27 insertions(+), 26 deletions(-) diff --git a/graph/src/main/scala/spark/graph/Analytics.scala b/graph/src/main/scala/spark/graph/Analytics.scala index 05275bec68347..8f76622db0eb2 100644 --- a/graph/src/main/scala/spark/graph/Analytics.scala +++ b/graph/src/main/scala/spark/graph/Analytics.scala @@ -7,6 +7,10 @@ import spark.SparkContext._ object Analytics extends Logging { + def main(args: Array[String]) { + //pregelPagerank() + } + // /** // * Compute the PageRank of a graph returning the pagerank of each vertex as an RDD // */ diff --git a/graph/src/main/scala/spark/graph/Graph.scala b/graph/src/main/scala/spark/graph/Graph.scala index fb5a86c41fc8a..506e53df08236 100644 --- a/graph/src/main/scala/spark/graph/Graph.scala +++ b/graph/src/main/scala/spark/graph/Graph.scala @@ -26,7 +26,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * * @todo should vertices return tuples instead of vertex objects? */ - def vertices(): RDD[Vertex[VD]] + def vertices: RDD[Vertex[VD]] /** * Get the Edges and their data as an RDD. The entries in the RDD contain @@ -41,7 +41,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * @todo Should edges return 3 tuples instead of Edge objects? In this case * we could rename EdgeTriplet to Edge? */ - def edges(): RDD[Edge[ED]] + def edges: RDD[Edge[ED]] /** * Get the edges with the vertex data associated with the adjacent pair of @@ -61,7 +61,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * @see edges() If only the edge data and adjacent vertex ids are required. * */ - def triplets(): RDD[EdgeTriplet[VD, ED]] + def triplets: RDD[EdgeTriplet[VD, ED]] /** * Return a graph that is cached when first created. This is used to pin a diff --git a/graph/src/main/scala/spark/graph/GraphOps.scala b/graph/src/main/scala/spark/graph/GraphOps.scala index 4fba8d19768e5..d98cd8d44cbb2 100644 --- a/graph/src/main/scala/spark/graph/GraphOps.scala +++ b/graph/src/main/scala/spark/graph/GraphOps.scala @@ -14,11 +14,11 @@ class GraphOps[VD: ClassManifest, ED: ClassManifest](g: Graph[VD, ED]) { } lazy val outDegrees: RDD[(Vid, Int)] = { - g.aggregateNeighbors((vid,edge) => Some(1), _+_, EdgeDirection.Out) + g.aggregateNeighbors((vid, edge) => Some(1), _+_, EdgeDirection.Out) } lazy val degrees: RDD[(Vid, Int)] = { - g.aggregateNeighbors((vid,edge) => Some(1), _+_, EdgeDirection.Both) + g.aggregateNeighbors((vid, edge) => Some(1), _+_, EdgeDirection.Both) } def collectNeighborIds(edgeDirection: EdgeDirection) : RDD[(Vid, Array[Vid])] = { diff --git a/graph/src/main/scala/spark/graph/Pregel.scala b/graph/src/main/scala/spark/graph/Pregel.scala index 699affba81661..0a564b8041f4d 100644 --- a/graph/src/main/scala/spark/graph/Pregel.scala +++ b/graph/src/main/scala/spark/graph/Pregel.scala @@ -1,19 +1,20 @@ package spark.graph -import scala.collection.JavaConversions._ import spark.RDD object Pregel { def iterate[VD: ClassManifest, ED: ClassManifest, A: ClassManifest](graph: Graph[VD, ED])( - vprog: ( Vertex[VD], A) => VD, - sendMsg: (Vid, EdgeTriplet[VD, ED]) => Option[A], - mergeMsg: (A, A) => A, - initialMsg: A, - numIter: Int) : Graph[VD, ED] = { - - var g = graph.cache + vprog: (Vertex[VD], A) => VD, + sendMsg: (Vid, EdgeTriplet[VD, ED]) => Option[A], + mergeMsg: (A, A) => A, + initialMsg: A, + numIter: Int) + : Graph[VD, ED] = { + + var g = graph + //var g = graph.cache() var i = 0 def mapF(vid: Vid, edge: EdgeTriplet[VD,ED]) = sendMsg(edge.otherVertex(vid).id, edge) diff --git a/graph/src/main/scala/spark/graph/Vertex.scala b/graph/src/main/scala/spark/graph/Vertex.scala index 5323643beceee..32653571f79a4 100644 --- a/graph/src/main/scala/spark/graph/Vertex.scala +++ b/graph/src/main/scala/spark/graph/Vertex.scala @@ -9,7 +9,7 @@ case class Vertex[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) VD var id: Vid = 0, var data: VD = nullValue[VD]) { - def this(tuple: Tuple2[Vid, VD]) = this(tuple._1, tuple._2) + def this(tuple: (Vid, VD)) = this(tuple._1, tuple._2) def tuple = (id, data) } diff --git a/graph/src/main/scala/spark/graph/impl/EdgePartition.scala b/graph/src/main/scala/spark/graph/impl/EdgePartition.scala index e5ed2db0f2947..0008534c0f82c 100644 --- a/graph/src/main/scala/spark/graph/impl/EdgePartition.scala +++ b/graph/src/main/scala/spark/graph/impl/EdgePartition.scala @@ -11,8 +11,8 @@ import spark.graph._ * A partition of edges in 3 large columnar arrays. */ private[graph] -class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED: ClassManifest] -{ +class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED: ClassManifest] { + val srcIds: IntArrayList = new IntArrayList val dstIds: IntArrayList = new IntArrayList // TODO: Specialize data. @@ -33,7 +33,7 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) def size: Int = srcIds.size def iterator = new Iterator[Edge[ED]] { - private var edge = new Edge[ED] + private val edge = new Edge[ED] private var pos = 0 override def hasNext: Boolean = pos < EdgePartition.this.size @@ -46,4 +46,4 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) edge } } -} \ No newline at end of file +} diff --git a/graph/src/main/scala/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/spark/graph/impl/GraphImpl.scala index 6ee7b8a062093..e18a2f6b0973c 100644 --- a/graph/src/main/scala/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/spark/graph/impl/GraphImpl.scala @@ -2,11 +2,7 @@ package spark.graph.impl import scala.collection.JavaConversions._ -import spark.ClosureCleaner -import spark.HashPartitioner -import spark.Partitioner -import spark.RDD -import spark.SparkContext +import spark.{ClosureCleaner, HashPartitioner, RDD} import spark.SparkContext._ import spark.graph._ @@ -31,7 +27,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( def withPartitioner(numVertexPartitions: Int, numEdgePartitions: Int): Graph[VD, ED] = { if (_cached) { - (new GraphImpl(numVertexPartitions, numEdgePartitions, null, null, _rawVTable, _rawETable)) + new GraphImpl(numVertexPartitions, numEdgePartitions, null, null, _rawVTable, _rawETable) .cache() } else { new GraphImpl(numVertexPartitions, numEdgePartitions, _rawVertices, _rawEdges, null, null) @@ -73,13 +69,13 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( if (!_cached && _rawEdges != null) { _rawEdges } else { - eTable.mapPartitions { iter => iter.next._2.iterator } + eTable.mapPartitions { iter => iter.next()._2.iterator } } } /** Return a RDD that brings edges with its source and destination vertices together. */ override def triplets: RDD[EdgeTriplet[VD, ED]] = { - (new EdgeTripletRDD(vTableReplicated, eTable)).mapPartitions { part => part.next._2 } + new EdgeTripletRDD(vTableReplicated, eTable).mapPartitions { part => part.next()._2 } } override def mapVertices[VD2: ClassManifest](f: Vertex[VD] => VD2): Graph[VD2, ED] = { From f269e5975bee4581297e017fc0617df00e726c62 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Sat, 29 Jun 2013 21:53:13 -0700 Subject: [PATCH 045/531] Adding additional assertions and documenting the edge triplet class --- .../main/scala/spark/graph/EdgeTriplet.scala | 48 ++++++++++++++++--- 1 file changed, 42 insertions(+), 6 deletions(-) diff --git a/graph/src/main/scala/spark/graph/EdgeTriplet.scala b/graph/src/main/scala/spark/graph/EdgeTriplet.scala index 9a819481ba879..3ed8052794af8 100644 --- a/graph/src/main/scala/spark/graph/EdgeTriplet.scala +++ b/graph/src/main/scala/spark/graph/EdgeTriplet.scala @@ -1,17 +1,53 @@ package spark.graph - +/** + * An edge triplet represents two vertices and edge along with their attributes. + * + * @tparam VD the type of the vertex attribute. + * @tparam ED the type of the edge attribute + */ class EdgeTriplet[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) VD, @specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED] { + /** + * The vertex (id and attribute) corresponding to the source vertex. + */ var src: Vertex[VD] = _ + + /** + * The vertex (id and attribute) corresponding to the target vertex. + */ var dst: Vertex[VD] = _ + + /** + * The attribute associated with the edge. + */ var data: ED = _ - def otherVertex(vid: Vid): Vertex[VD] = if (src.id == vid) dst else src + /** + * Given one vertex in the edge return the other vertex. + * + * @param vid the id one of the two vertices on the edge. + * @return the other vertex on the edge. + */ + def otherVertex(vid: Vid): Vertex[VD] = + if (src.id == vid) dst else { assert(dst.id == vid); src } + + /** + * Get the vertex object for the given vertex in the edge. + * + * @param vid the id of one of the two vertices on the edge + * @return the vertex object with that id. + */ + def vertex(vid: Vid): Vertex[VD] = + if (src.id == vid) src else { assert(dst.id == vid); dst } - def vertex(vid: Vid): Vertex[VD] = if (src.id == vid) src else dst + /** + * Return the relative direction of the edge to the corresponding vertex. + * + * @param vid the id of one of the two vertices in the edge. + * @return the relative direction of the edge to the corresponding vertex. + */ + def relativeDirection(vid: Vid): EdgeDirection = + if (vid == src.id) EdgeDirection.Out else { assert(vid == dst.id); EdgeDirection.In } - def relativeDirection(vid: Vid): EdgeDirection = { - if (vid == src.id) EdgeDirection.Out else EdgeDirection.In - } } From f776301241ee92eccd4f1c90d511b3ae568242eb Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Sat, 29 Jun 2013 21:53:38 -0700 Subject: [PATCH 046/531] Resurrecting the GraphLab gather-apply-scatter api --- .../src/main/scala/spark/graph/GraphLab.scala | 232 +++++++++--------- 1 file changed, 119 insertions(+), 113 deletions(-) diff --git a/graph/src/main/scala/spark/graph/GraphLab.scala b/graph/src/main/scala/spark/graph/GraphLab.scala index 9c157b9361dac..504cd162abaeb 100644 --- a/graph/src/main/scala/spark/graph/GraphLab.scala +++ b/graph/src/main/scala/spark/graph/GraphLab.scala @@ -3,119 +3,125 @@ package spark.graph import scala.collection.JavaConversions._ import spark.RDD - +/** + * This object implement the graphlab gather-apply-scatter api. + */ object GraphLab { - // def iterateGA2[VD: ClassManifest, ED: ClassManifest, A: ClassManifest](graph: Graph[VD, ED])( - // gather: (Vid, EdgeWithVertices[VD, ED]) => A, - // merge: (A, A) => A, - // default: A, - // apply: (Vertex[VD], A) => VD, - // numIter: Int, - // gatherDirection: EdgeDirection = EdgeDirection.In) : Graph[VD, ED] = { - - // var g = graph.cache() - - // var i = 0 - // while (i < numIter) { - - // val accUpdates: RDD[(Vid, A)] = - // g.aggregateNeighbors(gather, merge, default, gatherDirection) - - // def applyFunc(v: Vertex[VD], update: Option[A]): VD = { apply(v, update.get) } - // g = g.updateVertices(accUpdates, applyFunc).cache() - - // i += 1 - // } - // g - // } - - // def iterateGA[VD: ClassManifest, ED: ClassManifest, A: ClassManifest](graph: Graph[VD, ED])( - // gatherFunc: (Vid, EdgeWithVertices[VD, ED]) => A, - // mergeFunc: (A, A) => A, - // applyFunc: (Vertex[VD], Option[A]) => VD, - // numIter: Int, - // gatherDirection: EdgeDirection = EdgeDirection.In) : Graph[VD, ED] = { - - // var g = graph.cache() - - // def someGather(vid: Vid, edge: EdgeWithVertices[VD, ED]) = Some(gatherFunc(vid, edge)) - - // var i = 0 - // while (i < numIter) { - - // val accUpdates: RDD[(Vid, A)] = - // g.flatMapReduceNeighborhood(someGather, mergeFunc, gatherDirection) - - // g = g.updateVertices(accUpdates, applyFunc).cache() - - // i += 1 - // } - // g - // } - - // def iterateGAS[VD: ClassManifest, ED: ClassManifest, A: ClassManifest](graph: Graph[VD, ED])( - // gatherFunc: (Vid, EdgeWithVertices[VD, ED]) => A, - // mergeFunc: (A, A) => A, - // applyFunc: (Vertex[VD], Option[A]) => VD, - // scatterFunc: (Vid, EdgeWithVertices[VD, ED]) => Boolean, - // numIter: Int, - // gatherDirection: EdgeDirection = EdgeDirection.In, - // scatterDirection: EdgeDirection = EdgeDirection.Out) : Graph[VD, ED] = { - - // var g = graph.mapVertices{ case Vertex(id,data) => Vertex(id, (true, data)) }.cache() - - // def gather(vid: Vid, e: EdgeWithVertices[(Boolean, VD), ED]) = { - // if(e.vertex(vid).data._1) { - // val edge = new EdgeWithVertices[VD,ED] - // edge.src = Vertex(e.src.id, e.src.data._2) - // edge.dst = Vertex(e.dst.id, e.dst.data._2) - // Some(gatherFunc(vid, edge)) - // } else { - // None - // } - // } - - // def apply(v: Vertex[(Boolean, VD)], accum: Option[A]) = { - // if(v.data._1) (true, applyFunc(Vertex(v.id, v.data._2), accum)) - // else (false, v.data._2) - // } - - // def scatter(rawVid: Vid, e: EdgeWithVertices[(Boolean, VD),ED]) = { - // val vid = e.otherVertex(rawVid).id - // if(e.vertex(vid).data._1) { - // val edge = new EdgeWithVertices[VD,ED] - // edge.src = Vertex(e.src.id, e.src.data._2) - // edge.dst = Vertex(e.dst.id, e.dst.data._2) - // Some(scatterFunc(vid, edge)) - // } else { - // None - // } - // } - - // def applyActive(v: Vertex[(Boolean, VD)], accum: Option[Boolean]) = - // (accum.getOrElse(false), v.data._2) - - // var i = 0 - // var numActive = g.numVertices - // while (i < numIter && numActive > 0) { - - // val accUpdates: RDD[(Vid, A)] = - // g.flatMapReduceNeighborhood(gather, mergeFunc, gatherDirection) - - // g = g.updateVertices(accUpdates, apply).cache() - - // // Scatter is basically a gather in the opposite direction so we reverse the edge direction - // val activeVertices: RDD[(Vid, Boolean)] = - // g.flatMapReduceNeighborhood(scatter, _ || _, scatterDirection.reverse) - - // g = g.updateVertices(activeVertices, applyActive).cache() - - // numActive = g.vertices.map(v => if (v.data._1) 1 else 0).reduce( _ + _ ) - // println("Number active vertices: " + numActive) - // i += 1 - // } - - // g.mapVertices(v => Vertex(v.id, v.data._2)) - // } + /** + * Execute the GraphLab Gather-Apply-Scatter API + * + * @todo finish documenting GraphLab Gather-Apply-Scatter API + * + * @param graph The graph on which to execute the GraphLab API + * @param gatherFunc The gather function is executed on each edge triplet + * adjacent to a vertex and returns an accumulator which + * is then merged using the merge function. + * @param mergeFunc An accumulative associative operation on the result of + * the gather type. + * @param applyFunc Takes a vertex and the final result of the merge operations + * on the adjacent edges and returns a new vertex value. + * @param scatterFunc Executed after the apply function the scatter function takes + * a triplet and signals whether the neighboring vertex program + * must be recomputed. + * @param numIter The maximum number of iterations to run. + * @param gatherDirection The direction of edges to consider during the gather phase + * @param scatterDirection The direction of edges to consider during the scatter phase + * + * @tparam VD The graph vertex attribute type + * @tparam ED The graph edge attribute type + * @tparam A The type accumulated during the gather phase + * @return the resulting graph after the algorithm converges + */ + def apply[VD: ClassManifest, ED: ClassManifest, A: ClassManifest](graph: Graph[VD, ED])( + gatherFunc: (Vid, EdgeTriplet[VD, ED]) => A, + mergeFunc: (A, A) => A, + applyFunc: (Vertex[VD], Option[A]) => VD, + scatterFunc: (Vid, EdgeTriplet[VD, ED]) => Boolean, + numIter: Int, + gatherDirection: EdgeDirection = EdgeDirection.In, + scatterDirection: EdgeDirection = EdgeDirection.Out): Graph[VD, ED] = { + + + // Add an active attribute to all vertices to track convergence. + var activeGraph = graph.mapVertices { + case Vertex(id, data) => (true, data) + }.cache() + + // The gather function wrapper strips the active attribute and + // only invokes the gather function on active vertices + def gather(vid: Vid, e: EdgeTriplet[(Boolean, VD), ED]) = { + if (e.vertex(vid).data._1) { + val edge = new EdgeTriplet[VD,ED] + edge.src = Vertex(e.src.id, e.src.data._2) + edge.dst = Vertex(e.dst.id, e.dst.data._2) + edge.data = e.data + Some(gatherFunc(vid, edge)) + } else { + None + } + } + + // The apply function wrapper strips the vertex of the active attribute + // and only invokes the apply function on active vertices + def apply(v: Vertex[(Boolean, VD)], accum: Option[A]) = { + if (v.data._1) (true, applyFunc(Vertex(v.id, v.data._2), accum)) + else (false, v.data._2) + } + + // The scatter function wrapper strips the vertex of the active attribute + // and only invokes the scatter function on active vertices + def scatter(rawVid: Vid, e: EdgeTriplet[(Boolean, VD), ED]) = { + val vid = e.otherVertex(rawVid).id + if (e.vertex(vid).data._1) { + val edge = new EdgeTriplet[VD,ED] + edge.src = Vertex(e.src.id, e.src.data._2) + edge.dst = Vertex(e.dst.id, e.dst.data._2) + edge.data = e.data +// val src = Vertex(e.src.id, e.src.data._2) +// val dst = Vertex(e.dst.id, e.dst.data._2) +// val edge = new EdgeTriplet[VD,ED](src, dst, e.data) + Some(scatterFunc(vid, edge)) + } else { + None + } + } + + // Used to set the active status of vertices for the next round + def applyActive(v: Vertex[(Boolean, VD)], accum: Option[Boolean]) = + (accum.getOrElse(false), v.data._2) + + // Main Loop --------------------------------------------------------------------- + var i = 0 + var numActive = activeGraph.numVertices + while (i < numIter && numActive > 0) { + + val accUpdates: RDD[(Vid, A)] = + activeGraph.aggregateNeighbors(gather, mergeFunc, gatherDirection) + + activeGraph = activeGraph.leftJoinVertices(accUpdates, apply).cache() + + // Scatter is basically a gather in the opposite direction so we reverse the edge direction + val activeVertices: RDD[(Vid, Boolean)] = + activeGraph.aggregateNeighbors(scatter, _ || _, scatterDirection.reverse) + + activeGraph = activeGraph.leftJoinVertices(activeVertices, applyActive).cache() + + numActive = activeGraph.vertices.map(v => if (v.data._1) 1 else 0).reduce(_ + _) + println("Number active vertices: " + numActive) + i += 1 + } + + // Remove the active attribute from the vertex data before returning the graph + activeGraph.mapVertices(v => v.data._2) + } } + + + + + + + + + From 0791581346517c8fa55540703f667f30abba73a0 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 30 Jun 2013 23:07:40 -0700 Subject: [PATCH 047/531] More bug fixes --- .../main/scala/spark/graph/Analytics.scala | 2 +- graph/src/main/scala/spark/graph/Graph.scala | 23 +++++++++++-------- .../spark/graph/impl/EdgePartition.scala | 14 +++++++---- .../scala/spark/graph/impl/GraphImpl.scala | 11 +++++---- .../test/scala/spark/graph/GraphSuite.scala | 23 +++++++++++++++++++ 5 files changed, 52 insertions(+), 21 deletions(-) diff --git a/graph/src/main/scala/spark/graph/Analytics.scala b/graph/src/main/scala/spark/graph/Analytics.scala index 8f76622db0eb2..8acf863ff83d8 100644 --- a/graph/src/main/scala/spark/graph/Analytics.scala +++ b/graph/src/main/scala/spark/graph/Analytics.scala @@ -51,7 +51,7 @@ object Analytics extends Logging { (me_id, edge) => Some(edge.src.data._2 / edge.src.data._1), // gather (a: Double, b: Double) => a + b, // merge 1.0, - numIter).mapVertices{ case Vertex(id, (outDeg, r)) => Vertex(id, r) } + numIter).mapVertices{ case Vertex(id, (outDeg, r)) => r } } // /** diff --git a/graph/src/main/scala/spark/graph/Graph.scala b/graph/src/main/scala/spark/graph/Graph.scala index ef058a1fb537b..6724e4ede5ec3 100644 --- a/graph/src/main/scala/spark/graph/Graph.scala +++ b/graph/src/main/scala/spark/graph/Graph.scala @@ -252,6 +252,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * * @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 @@ -329,24 +330,26 @@ object Graph { import spark.graph.impl._ import spark.SparkContext._ - def apply(rawEdges: RDD[(Int,Int)], uniqueEdges: Boolean = true): Graph[Int, Int] = { - // Reduce to unique edges - val edges = - if(uniqueEdges) rawEdges.map{ case (s,t) => ((s,t),1) }.reduceByKey( _ + _ ) - .map{ case ((s,t), cnt) => Edge(s,t,cnt) } - else rawEdges.map{ case (s,t) => Edge(s,t,1) } + def apply(rawEdges: RDD[(Int, Int)], uniqueEdges: Boolean = true): Graph[Int, Int] = { + // Reduce to unique edges. + val edges: RDD[Edge[Int]] = + if (uniqueEdges) { + rawEdges.map((_, 1)).reduceByKey(_ + _).map { case ((s, t), cnt) => Edge(s, t, cnt) } + } else { + rawEdges.map { case (s, t) => Edge(s, t, 1) } + } // Determine unique vertices - val vertices = edges.flatMap{ case Edge(s, t, cnt) => Array((s,1), (t,1)) }.reduceByKey( _ + _ ) + val vertices: RDD[Vertex[Int]] = edges.flatMap{ case Edge(s, t, cnt) => Array((s, 1), (t, 1)) } + .reduceByKey(_ + _) .map{ case (id, deg) => Vertex(id, deg) } // Return graph new GraphImpl(vertices, edges) } - def apply[VD: ClassManifest, ED: ClassManifest](vertices: RDD[Vertex[VD]], edges: RDD[Edge[ED]]): Graph[VD, ED] = { + def apply[VD: ClassManifest, ED: ClassManifest]( + vertices: RDD[Vertex[VD]], edges: RDD[Edge[ED]]): Graph[VD, ED] = { new GraphImpl(vertices, edges) - } - implicit def graphToGraphOps[VD: ClassManifest, ED: ClassManifest](g: Graph[VD, ED]) = g.ops } diff --git a/graph/src/main/scala/spark/graph/impl/EdgePartition.scala b/graph/src/main/scala/spark/graph/impl/EdgePartition.scala index 0008534c0f82c..0e092541c9fd3 100644 --- a/graph/src/main/scala/spark/graph/impl/EdgePartition.scala +++ b/graph/src/main/scala/spark/graph/impl/EdgePartition.scala @@ -1,6 +1,6 @@ package spark.graph.impl -import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.ArrayBuilder import it.unimi.dsi.fastutil.ints.IntArrayList @@ -13,21 +13,25 @@ import spark.graph._ private[graph] class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED: ClassManifest] { + private var _data: Array[ED] = _ + private var _dataBuilder = ArrayBuilder.make[ED] + val srcIds: IntArrayList = new IntArrayList val dstIds: IntArrayList = new IntArrayList - // TODO: Specialize data. - val data: ArrayBuffer[ED] = new ArrayBuffer[ED] + + def data: Array[ED] = _data /** Add a new edge to the partition. */ def add(src: Vid, dst: Vid, d: ED) { srcIds.add(src) dstIds.add(dst) - data += d + _dataBuilder += d } def trim() { srcIds.trim() dstIds.trim() + _data = _dataBuilder.result() } def size: Int = srcIds.size @@ -41,7 +45,7 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) override def next(): Edge[ED] = { edge.src = srcIds.get(pos) edge.dst = dstIds.get(pos) - edge.data = data(pos) + edge.data = _data(pos) pos += 1 edge } diff --git a/graph/src/main/scala/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/spark/graph/impl/GraphImpl.scala index e18a2f6b0973c..3f5cbc3d75b45 100644 --- a/graph/src/main/scala/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/spark/graph/impl/GraphImpl.scala @@ -150,7 +150,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( } /** - * Same as mapReduceNeighborhood but map function can return none and there is no default value. + * Same as aggregateNeighbors but map function can return none and there is no default value. * As a consequence, the resulting table may be much smaller than the set of vertices. */ override def aggregateNeighbors[VD2: ClassManifest]( @@ -165,7 +165,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( part.map { v => (v._1, MutableTuple2(v._2, Option.empty[VD2])) } }, preservesPartitioning = true) - (new EdgeTripletRDD[MutableTuple2[VD, Option[VD2]], ED](newVTable, eTable)) + new EdgeTripletRDD[MutableTuple2[VD, Option[VD2]], ED](newVTable, eTable) .mapPartitions { part => val (vmap, edges) = part.next() val edgeSansAcc = new EdgeTriplet[VD, ED]() @@ -188,7 +188,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( } if (gatherDirection == EdgeDirection.Out || gatherDirection == EdgeDirection.Both) { e.dst.data._2 = - if (e.dst.data._2.isEmpty) { + if (e.src.data._2.isEmpty) { mapFunc(edgeSansAcc.src.id, edgeSansAcc) } else { val tmp = mapFunc(edgeSansAcc.src.id, edgeSansAcc) @@ -218,7 +218,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( } }, preservesPartitioning = true).cache() - new GraphImpl(newVTable.partitions.size, eTable.partitions.size, null, null, newVTable, eTable) + new GraphImpl(newVTable.partitions.length, eTable.partitions.length, null, null, newVTable, eTable) } override def joinVertices[U: ClassManifest]( @@ -239,7 +239,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( } }, preservesPartitioning = true).cache() - new GraphImpl(newVTable.partitions.size, eTable.partitions.size, null, null, newVTable, eTable) + new GraphImpl(newVTable.partitions.length, eTable.partitions.length, null, null, newVTable, eTable) } @@ -307,6 +307,7 @@ object GraphImpl { .mapPartitionsWithIndex({ (pid, iter) => val edgePartition = new EdgePartition[ED] iter.foreach { case (_, (src, dst, data)) => edgePartition.add(src, dst, data) } + edgePartition.trim() Iterator((pid, edgePartition)) }, preservesPartitioning = true) } diff --git a/graph/src/test/scala/spark/graph/GraphSuite.scala b/graph/src/test/scala/spark/graph/GraphSuite.scala index 3d250aa18ce09..87c8c158af581 100644 --- a/graph/src/test/scala/spark/graph/GraphSuite.scala +++ b/graph/src/test/scala/spark/graph/GraphSuite.scala @@ -3,10 +3,33 @@ package spark.graph import org.scalatest.FunSuite import spark.SparkContext +import spark.graph.impl.GraphImpl class GraphSuite extends FunSuite with LocalSparkContext { + test("aggregateNeighbors") { + + } + + test("joinVertices") { + sc = new SparkContext("local", "test") + val vertices = sc.parallelize(Seq(Vertex(1, "one"), Vertex(2, "two"), Vertex(3, "three")), 2) + val edges = sc.parallelize((Seq(Edge(1, 2, "onetwo")))) + val g: Graph[String, String] = new GraphImpl(vertices, edges) + + val tbl = sc.parallelize(Seq((1, 10), (2, 20))) + val g1 = g.joinVertices(tbl, (v: Vertex[String], u: Int) => v.data + u) + + val v = g1.vertices.collect().sortBy(_.id) + assert(v(0).data === "one10") + assert(v(1).data === "two20") + assert(v(2).data === "three") + + val e = g1.edges.collect() + assert(e(0).data === "onetwo") + } + // test("graph partitioner") { // sc = new SparkContext("local", "test") // val vertices = sc.parallelize(Seq(Vertex(1, "one"), Vertex(2, "two"))) From 2943edf8eece20b21ff568b401cc5e8323ce9c07 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 1 Jul 2013 00:24:30 -0700 Subject: [PATCH 048/531] Fixed another bug .. --- graph/src/main/scala/spark/graph/impl/GraphImpl.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/graph/src/main/scala/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/spark/graph/impl/GraphImpl.scala index 3f5cbc3d75b45..711446d8e0c08 100644 --- a/graph/src/main/scala/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/spark/graph/impl/GraphImpl.scala @@ -110,7 +110,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( part.map { v => (v._1, MutableTuple2(v._2, Option.empty[VD2])) } }, preservesPartitioning = true) - (new EdgeTripletRDD[MutableTuple2[VD, Option[VD2]], ED](newVTable, eTable)) + new EdgeTripletRDD[MutableTuple2[VD, Option[VD2]], ED](newVTable, eTable) .mapPartitions { part => val (vmap, edges) = part.next() val edgeSansAcc = new EdgeTriplet[VD, ED]() @@ -187,7 +187,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( } } if (gatherDirection == EdgeDirection.Out || gatherDirection == EdgeDirection.Both) { - e.dst.data._2 = + e.src.data._2 = if (e.src.data._2.isEmpty) { mapFunc(edgeSansAcc.src.id, edgeSansAcc) } else { From 2f2c7e6a294ae9297ae7c9b84cfc352384d373aa Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 1 Jul 2013 16:23:23 -0700 Subject: [PATCH 049/531] Added a correctEdges function. --- graph/src/main/scala/spark/graph/Graph.scala | 1 + graph/src/main/scala/spark/graph/impl/GraphImpl.scala | 6 ++++++ 2 files changed, 7 insertions(+) diff --git a/graph/src/main/scala/spark/graph/Graph.scala b/graph/src/main/scala/spark/graph/Graph.scala index 6724e4ede5ec3..f51171f0d627e 100644 --- a/graph/src/main/scala/spark/graph/Graph.scala +++ b/graph/src/main/scala/spark/graph/Graph.scala @@ -141,6 +141,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { def mapTriplets[ED2: ClassManifest]( map: EdgeTriplet[VD, ED] => ED2): Graph[VD, ED2] + def correctEdges(): Graph[VD, ED] /** * Construct a new graph with all the edges reversed. If this graph contains diff --git a/graph/src/main/scala/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/spark/graph/impl/GraphImpl.scala index 711446d8e0c08..775d0686e75ae 100644 --- a/graph/src/main/scala/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/spark/graph/impl/GraphImpl.scala @@ -91,6 +91,12 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( newGraph(vertices, triplets.map(e => Edge(e.src.id, e.dst.id, f(e)))) } + override def correctEdges(): Graph[VD, ED] = { + val sc = vertices.context + val vset = sc.broadcast(vertices.map(_.id).collect().toSet) + val newEdges = edges.filter(e => vset.value.contains(e.src) && vset.value.contains(e.dst)) + Graph(vertices, newEdges) + } ////////////////////////////////////////////////////////////////////////////////////////////////// // Lower level transformation methods From 42942fc1a917c8ec31f045e7aecfd99bcb0961f6 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Tue, 6 Aug 2013 11:59:31 -0700 Subject: [PATCH 050/531] In the process of bringing the GraphLab api back and fixing the analytics toolkit --- .../main/scala/spark/graph/Analytics.scala | 860 +++++++++--------- .../src/main/scala/spark/graph/GraphLab.scala | 4 +- 2 files changed, 433 insertions(+), 431 deletions(-) diff --git a/graph/src/main/scala/spark/graph/Analytics.scala b/graph/src/main/scala/spark/graph/Analytics.scala index 8f76622db0eb2..77d234a5ff829 100644 --- a/graph/src/main/scala/spark/graph/Analytics.scala +++ b/graph/src/main/scala/spark/graph/Analytics.scala @@ -1,15 +1,14 @@ package spark.graph import spark._ -import spark.SparkContext._ -// import breeze.linalg._ + object Analytics extends Logging { - def main(args: Array[String]) { - //pregelPagerank() - } +// def main(args: Array[String]) { +// //pregelPagerank() +// } // /** // * Compute the PageRank of a graph returning the pagerank of each vertex as an RDD @@ -41,387 +40,394 @@ object Analytics extends Logging { /** * Compute the PageRank of a graph returning the pagerank of each vertex as an RDD */ - def pregelPagerank[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], numIter: Int) = { + def pagerank[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], + numIter: Int, + resetProb: Double = 0.15) = { // Compute the out degree of each vertex val pagerankGraph = graph.leftJoinVertices[Int, (Int, Double)](graph.outDegrees, (vertex, deg) => (deg.getOrElse(0), 1.0) ) Pregel.iterate[(Int, Double), ED, Double](pagerankGraph)( - (vertex, a: Double) => (vertex.data._1, (0.15 + 0.85 * a)), // apply + (vertex, a: Double) => (vertex.data._1, (resetProb + (1.0 - resetProb) * a)), // apply (me_id, edge) => Some(edge.src.data._2 / edge.src.data._1), // gather (a: Double, b: Double) => a + b, // merge 1.0, - numIter).mapVertices{ case Vertex(id, (outDeg, r)) => Vertex(id, r) } + numIter).mapVertices { + case Vertex(id, (outDeg, r)) => Vertex(id, r) + } + } + + /** + * Compute the PageRank of a graph returning the pagerank of each vertex as an RDD + */ + def dynamicPagerank[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], + tol: Float, + maxIter: Int = Integer.MAX_VALUE, + resetProb: Double = 0.15) = { + // Compute the out degree of each vertex + val pagerankGraph = graph.leftJoinVertices[Int, (Int, Double, Double)](graph.outDegrees, + (vertex, degIter) => (degIter.sum, 1.0, 1.0) + ) + + // Run PageRank + GraphLab.iterate(pagerankGraph)( + (me_id, edge) => edge.src.data._2 / edge.src.data._1, // gather + (a: Double, b: Double) => a + b, + (vertex, a: Option[Double]) => + (vertex.data._1, (resetProb + (1.0 - resetProb) * a.getOrElse(0.0)), vertex.data._2), // apply + (me_id, edge) => math.abs(edge.src.data._3 - edge.src.data._2) > tol, // scatter + maxIter).mapVertices { + case Vertex(vid, data) => Vertex(vid, data._2) + } } -// /** -// * Compute the PageRank of a graph returning the pagerank of each vertex as an RDD -// */ -// def dynamicPagerank[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], -// tol: Float, maxIter: Int = 10) = { -// // Compute the out degree of each vertex -// val pagerankGraph = graph.updateVertices[Int, (Int, Float, Float)](graph.outDegrees, -// (vertex, degIter) => (degIter.sum, 1.0F, 1.0F) -// ) - -// // Run PageRank -// GraphLab.iterateGAS(pagerankGraph)( -// (me_id, edge) => edge.src.data._2 / edge.src.data._1, // gather -// (a: Float, b: Float) => a + b, -// (vertex, a: Option[Float]) => -// (vertex.data._1, (0.15F + 0.85F * a.getOrElse(0F)), vertex.data._2), // apply -// (me_id, edge) => math.abs(edge.src.data._2 - edge.dst.data._1) > tol, // scatter -// maxIter).mapVertices { case Vertex(vid, data) => Vertex(vid, data._2) } -// } - -// /** -// * Compute the connected component membership of each vertex -// * and return an RDD with the vertex value containing the -// * lowest vertex id in the connected component containing -// * that vertex. -// */ -// def connectedComponents[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], numIter: Int) = { -// val ccGraph = graph.mapVertices { case Vertex(vid, _) => Vertex(vid, vid) } -// GraphLab.iterateGA[Int, ED, Int](ccGraph)( -// (me_id, edge) => edge.otherVertex(me_id).data, // gather -// (a: Int, b: Int) => math.min(a, b), // merge -// (v, a: Option[Int]) => math.min(v.data, a.getOrElse(Integer.MAX_VALUE)), // apply -// numIter, -// gatherDirection = EdgeDirection.Both) -// } - -// /** -// * Compute the shortest path to a set of markers -// */ -// def shortestPath[VD: Manifest](graph: Graph[VD, Float], sources: List[Int], numIter: Int) = { -// val sourceSet = sources.toSet -// val spGraph = graph.mapVertices { -// case Vertex(vid, _) => Vertex(vid, (if(sourceSet.contains(vid)) 0.0F else Float.MaxValue)) -// } -// GraphLab.iterateGA[Float, Float, Float](spGraph)( -// (me_id, edge) => edge.otherVertex(me_id).data + edge.data, // gather -// (a: Float, b: Float) => math.min(a, b), // merge -// (v, a: Option[Float]) => math.min(v.data, a.getOrElse(Float.MaxValue)), // apply -// numIter, -// gatherDirection = EdgeDirection.In) -// } - -// // /** -// // * Compute the connected component membership of each vertex -// // * and return an RDD with the vertex value containing the -// // * lowest vertex id in the connected component containing -// // * that vertex. -// // */ -// // def dynamicConnectedComponents[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], -// // numIter: Int = Int.MaxValue) = { - -// // val vertices = graph.vertices.mapPartitions(iter => iter.map { case (vid, _) => (vid, vid) }) -// // val edges = graph.edges // .mapValues(v => None) -// // val ccGraph = new Graph(vertices, edges) - -// // ccGraph.iterateDynamic( -// // (me_id, edge) => edge.otherVertex(me_id).data, // gather -// // (a: Int, b: Int) => math.min(a, b), // merge -// // Integer.MAX_VALUE, -// // (v, a: Int) => math.min(v.data, a), // apply -// // (me_id, edge) => edge.otherVertex(me_id).data > edge.vertex(me_id).data, // scatter -// // numIter, -// // gatherEdges = EdgeDirection.Both, -// // scatterEdges = EdgeDirection.Both).vertices -// // // -// // // graph_ret.vertices.collect.foreach(println) -// // // graph_ret.edges.take(10).foreach(println) -// // } - - - - -// // /** -// // * Compute the shortest path to a set of markers -// // */ -// // def dynamicShortestPath[VD: Manifest, ED: Manifest](graph: Graph[VD, Float], -// // sources: List[Int], numIter: Int) = { -// // val sourceSet = sources.toSet -// // val vertices = graph.vertices.mapPartitions( -// // iter => iter.map { -// // case (vid, _) => (vid, (if(sourceSet.contains(vid)) 0.0F else Float.MaxValue) ) -// // }); - -// // val edges = graph.edges // .mapValues(v => None) -// // val spGraph = new Graph(vertices, edges) - -// // val niterations = Int.MaxValue -// // spGraph.iterateDynamic( -// // (me_id, edge) => edge.otherVertex(me_id).data + edge.data, // gather -// // (a: Float, b: Float) => math.min(a, b), // merge -// // Float.MaxValue, -// // (v, a: Float) => math.min(v.data, a), // apply -// // (me_id, edge) => edge.vertex(me_id).data + edge.data < edge.otherVertex(me_id).data, // scatter -// // numIter, -// // gatherEdges = EdgeDirection.In, -// // scatterEdges = EdgeDirection.Out).vertices -// // } - - - - -// // /** -// // * -// // */ -// // def alternatingLeastSquares[VD: ClassManifest, ED: ClassManifest](graph: Graph[VD, Double], -// // latentK: Int, lambda: Double, numIter: Int) = { -// // val vertices = graph.vertices.mapPartitions( _.map { -// // case (vid, _) => (vid, Array.fill(latentK){ scala.util.Random.nextDouble() } ) -// // }).cache -// // val maxUser = graph.edges.map(_._1).reduce(math.max(_,_)) -// // val edges = graph.edges // .mapValues(v => None) -// // val alsGraph = new Graph(vertices, edges) -// // alsGraph.numVPart = graph.numVPart -// // alsGraph.numEPart = graph.numEPart - -// // val niterations = Int.MaxValue -// // alsGraph.iterateDynamic[(Array[Double], Array[Double])]( -// // (me_id, edge) => { // gather -// // val X = edge.otherVertex(me_id).data -// // val y = edge.data -// // val Xy = X.map(_ * y) -// // val XtX = (for(i <- 0 until latentK; j <- i until latentK) yield(X(i) * X(j))).toArray -// // (Xy, XtX) -// // }, -// // (a, b) => { -// // // The difference between the while loop and the zip is a FACTOR OF TWO in overall -// // // runtime -// // var i = 0 -// // while(i < a._1.length) { a._1(i) += b._1(i); i += 1 } -// // i = 0 -// // while(i < a._2.length) { a._2(i) += b._2(i); i += 1 } -// // a -// // // (a._1.zip(b._1).map{ case (q,r) => q+r }, a._2.zip(b._2).map{ case (q,r) => q+r }) -// // }, -// // (Array.empty[Double], Array.empty[Double]), // default value is empty -// // (vertex, accum) => { // apply -// // val XyArray = accum._1 -// // val XtXArray = accum._2 -// // if(XyArray.isEmpty) vertex.data // no neighbors -// // else { -// // val XtX = DenseMatrix.tabulate(latentK,latentK){ (i,j) => -// // (if(i < j) XtXArray(i + (j+1)*j/2) else XtXArray(i + (j+1)*j/2)) + -// // (if(i == j) lambda else 1.0F) //regularization -// // } -// // val Xy = DenseMatrix.create(latentK,1,XyArray) -// // val w = XtX \ Xy -// // w.data -// // } -// // }, -// // (me_id, edge) => true, -// // numIter, -// // gatherEdges = EdgeDirection.Both, -// // scatterEdges = EdgeDirection.Both, -// // vertex => vertex.id < maxUser).vertices -// // } - -// def main(args: Array[String]) = { -// val host = args(0) -// val taskType = args(1) -// val fname = args(2) -// val options = args.drop(3).map { arg => -// arg.dropWhile(_ == '-').split('=') match { -// case Array(opt, v) => (opt -> v) -// case _ => throw new IllegalArgumentException("Invalid argument: " + arg) -// } -// } - -// System.setProperty("spark.serializer", "spark.KryoSerializer") -// //System.setProperty("spark.shuffle.compress", "false") -// System.setProperty("spark.kryo.registrator", "spark.graph.GraphKryoRegistrator") - -// taskType match { -// case "pagerank" => { - -// var numIter = Int.MaxValue -// var isDynamic = false -// var tol:Float = 0.001F -// var outFname = "" -// var numVPart = 4 -// var numEPart = 4 - -// options.foreach{ -// case ("numIter", v) => numIter = v.toInt -// case ("dynamic", v) => isDynamic = v.toBoolean -// case ("tol", v) => tol = v.toFloat -// case ("output", v) => outFname = v -// case ("numVPart", v) => numVPart = v.toInt -// case ("numEPart", v) => numEPart = v.toInt -// case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) -// } - -// if(!isDynamic && numIter == Int.MaxValue) { -// println("Set number of iterations!") -// sys.exit(1) -// } -// println("======================================") -// println("| PageRank |") -// println("--------------------------------------") -// println(" Using parameters:") -// println(" \tDynamic: " + isDynamic) -// if(isDynamic) println(" \t |-> Tolerance: " + tol) -// println(" \tNumIter: " + numIter) -// println("======================================") - -// val sc = new SparkContext(host, "PageRank(" + fname + ")") - -// val graph = Graph.textFile(sc, fname, a => 1.0F).withPartitioner(numVPart, numEPart).cache() - -// val startTime = System.currentTimeMillis -// logInfo("GRAPHX: starting tasks") -// logInfo("GRAPHX: Number of vertices " + graph.vertices.count) -// logInfo("GRAPHX: Number of edges " + graph.edges.count) - -// val pr = Analytics.pagerank(graph, numIter) -// // val pr = if(isDynamic) Analytics.dynamicPagerank(graph, tol, numIter) -// // else Analytics.pagerank(graph, numIter) -// logInfo("GRAPHX: Total rank: " + pr.vertices.map{ case Vertex(id,r) => r }.reduce(_+_) ) -// if (!outFname.isEmpty) { -// println("Saving pageranks of pages to " + outFname) -// pr.vertices.map{case Vertex(id, r) => id + "\t" + r}.saveAsTextFile(outFname) -// } -// logInfo("GRAPHX: Runtime: " + ((System.currentTimeMillis - startTime)/1000.0) + " seconds") -// sc.stop() -// } - -// case "cc" => { - -// var numIter = Int.MaxValue -// var isDynamic = false - -// options.foreach{ -// case ("numIter", v) => numIter = v.toInt -// case ("dynamic", v) => isDynamic = v.toBoolean -// case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) -// } - -// 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 + ")") -// val graph = Graph.textFile(sc, fname, a => 1.0F) -// val cc = Analytics.connectedComponents(graph, numIter) -// // val cc = if(isDynamic) Analytics.dynamicConnectedComponents(graph, numIter) -// // else Analytics.connectedComponents(graph, numIter) -// println("Components: " + cc.vertices.map(_.data).distinct()) - -// sc.stop() -// } - -// case "shortestpath" => { - -// var numIter = Int.MaxValue -// var isDynamic = true -// var sources: List[Int] = List.empty - -// options.foreach{ -// case ("numIter", v) => numIter = v.toInt -// case ("dynamic", v) => isDynamic = v.toBoolean -// case ("source", v) => sources ++= List(v.toInt) -// case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) -// } - - -// if(!isDynamic && numIter == Int.MaxValue) { -// println("Set number of iterations!") -// sys.exit(1) -// } - -// if(sources.isEmpty) { -// println("No sources provided!") -// sys.exit(1) -// } - -// println("======================================") -// println("| Shortest Path |") -// println("--------------------------------------") -// println(" Using parameters:") -// println(" \tDynamic: " + isDynamic) -// println(" \tNumIter: " + numIter) -// println(" \tSources: [" + sources.mkString(", ") + "]") -// println("======================================") - -// val sc = new SparkContext(host, "ShortestPath(" + fname + ")") -// val graph = Graph.textFile(sc, fname, a => (if(a.isEmpty) 1.0F else a(0).toFloat ) ) -// val sp = Analytics.shortestPath(graph, sources, numIter) -// // val cc = if(isDynamic) Analytics.dynamicShortestPath(graph, sources, numIter) -// // else Analytics.shortestPath(graph, sources, numIter) -// println("Longest Path: " + sp.vertices.map(_.data).reduce(math.max(_,_))) - -// sc.stop() -// } - - -// // case "als" => { - -// // var numIter = 5 -// // var lambda = 0.01 -// // var latentK = 10 -// // var usersFname = "usersFactors.tsv" -// // var moviesFname = "moviesFname.tsv" -// // var numVPart = 4 -// // var numEPart = 4 - -// // options.foreach{ -// // case ("numIter", v) => numIter = v.toInt -// // case ("lambda", v) => lambda = v.toDouble -// // case ("latentK", v) => latentK = v.toInt -// // case ("usersFname", v) => usersFname = v -// // case ("moviesFname", v) => moviesFname = v -// // case ("numVPart", v) => numVPart = v.toInt -// // case ("numEPart", v) => numEPart = v.toInt -// // case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) -// // } - -// // println("======================================") -// // println("| Alternating Least Squares |") -// // println("--------------------------------------") -// // println(" Using parameters:") -// // println(" \tNumIter: " + numIter) -// // println(" \tLambda: " + lambda) -// // println(" \tLatentK: " + latentK) -// // println(" \tusersFname: " + usersFname) -// // println(" \tmoviesFname: " + moviesFname) -// // println("======================================") - -// // val sc = new SparkContext(host, "ALS(" + fname + ")") -// // val graph = Graph.textFile(sc, fname, a => a(0).toDouble ) -// // graph.numVPart = numVPart -// // graph.numEPart = numEPart - -// // val maxUser = graph.edges.map(_._1).reduce(math.max(_,_)) -// // val minMovie = graph.edges.map(_._2).reduce(math.min(_,_)) -// // assert(maxUser < minMovie) - -// // val factors = Analytics.alternatingLeastSquares(graph, latentK, lambda, numIter).cache -// // factors.filter(_._1 <= maxUser).map(r => r._1 + "\t" + r._2.mkString("\t")) -// // .saveAsTextFile(usersFname) -// // factors.filter(_._1 >= minMovie).map(r => r._1 + "\t" + r._2.mkString("\t")) -// // .saveAsTextFile(moviesFname) - -// // sc.stop() -// // } - - -// case _ => { -// println("Invalid task type.") -// } -// } -// } + + /** + * Compute the connected component membership of each vertex + * and return an RDD with the vertex value containing the + * lowest vertex id in the connected component containing + * that vertex. + */ + def connectedComponents[VD: Manifest, ED: Manifest](graph: Graph[VD, ED]) = { + val ccGraph = graph.mapVertices { case Vertex(vid, _) => vid } + + GraphLab.iterate[Int, ED, Int](ccGraph)( + (me_id, edge) => edge.otherVertex(me_id).data, // gather + (a: Int, b: Int) => math.min(a, b), // merge + (v, a: Option[Int]) => math.min(v.data, a.getOrElse(Integer.MAX_VALUE)), // apply + (me_id, edge) => (edge.vertex(me_id).data < edge.otherVertex(me_id).data), // scatter + gatherDirection = EdgeDirection.Both, scatterDirection = EdgeDirection.Both + ) + } + + // /** + // * Compute the shortest path to a set of markers + // */ + // def shortestPath[VD: Manifest](graph: Graph[VD, Float], sources: List[Int], numIter: Int) = { + // val sourceSet = sources.toSet + // val spGraph = graph.mapVertices { + // case Vertex(vid, _) => Vertex(vid, (if(sourceSet.contains(vid)) 0.0F else Float.MaxValue)) + // } + // GraphLab.iterateGA[Float, Float, Float](spGraph)( + // (me_id, edge) => edge.otherVertex(me_id).data + edge.data, // gather + // (a: Float, b: Float) => math.min(a, b), // merge + // (v, a: Option[Float]) => math.min(v.data, a.getOrElse(Float.MaxValue)), // apply + // numIter, + // gatherDirection = EdgeDirection.In) + // } + + // // /** + // // * Compute the connected component membership of each vertex + // // * and return an RDD with the vertex value containing the + // // * lowest vertex id in the connected component containing + // // * that vertex. + // // */ + // // def dynamicConnectedComponents[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], + // // numIter: Int = Int.MaxValue) = { + + // // val vertices = graph.vertices.mapPartitions(iter => iter.map { case (vid, _) => (vid, vid) }) + // // val edges = graph.edges // .mapValues(v => None) + // // val ccGraph = new Graph(vertices, edges) + + // // ccGraph.iterateDynamic( + // // (me_id, edge) => edge.otherVertex(me_id).data, // gather + // // (a: Int, b: Int) => math.min(a, b), // merge + // // Integer.MAX_VALUE, + // // (v, a: Int) => math.min(v.data, a), // apply + // // (me_id, edge) => edge.otherVertex(me_id).data > edge.vertex(me_id).data, // scatter + // // numIter, + // // gatherEdges = EdgeDirection.Both, + // // scatterEdges = EdgeDirection.Both).vertices + // // // + // // // graph_ret.vertices.collect.foreach(println) + // // // graph_ret.edges.take(10).foreach(println) + // // } + + + // // /** + // // * Compute the shortest path to a set of markers + // // */ + // // def dynamicShortestPath[VD: Manifest, ED: Manifest](graph: Graph[VD, Float], + // // sources: List[Int], numIter: Int) = { + // // val sourceSet = sources.toSet + // // val vertices = graph.vertices.mapPartitions( + // // iter => iter.map { + // // case (vid, _) => (vid, (if(sourceSet.contains(vid)) 0.0F else Float.MaxValue) ) + // // }); + + // // val edges = graph.edges // .mapValues(v => None) + // // val spGraph = new Graph(vertices, edges) + + // // val niterations = Int.MaxValue + // // spGraph.iterateDynamic( + // // (me_id, edge) => edge.otherVertex(me_id).data + edge.data, // gather + // // (a: Float, b: Float) => math.min(a, b), // merge + // // Float.MaxValue, + // // (v, a: Float) => math.min(v.data, a), // apply + // // (me_id, edge) => edge.vertex(me_id).data + edge.data < edge.otherVertex(me_id).data, // scatter + // // numIter, + // // gatherEdges = EdgeDirection.In, + // // scatterEdges = EdgeDirection.Out).vertices + // // } + + + // // /** + // // * + // // */ + // // def alternatingLeastSquares[VD: ClassManifest, ED: ClassManifest](graph: Graph[VD, Double], + // // latentK: Int, lambda: Double, numIter: Int) = { + // // val vertices = graph.vertices.mapPartitions( _.map { + // // case (vid, _) => (vid, Array.fill(latentK){ scala.util.Random.nextDouble() } ) + // // }).cache + // // val maxUser = graph.edges.map(_._1).reduce(math.max(_,_)) + // // val edges = graph.edges // .mapValues(v => None) + // // val alsGraph = new Graph(vertices, edges) + // // alsGraph.numVPart = graph.numVPart + // // alsGraph.numEPart = graph.numEPart + + // // val niterations = Int.MaxValue + // // alsGraph.iterateDynamic[(Array[Double], Array[Double])]( + // // (me_id, edge) => { // gather + // // val X = edge.otherVertex(me_id).data + // // val y = edge.data + // // val Xy = X.map(_ * y) + // // val XtX = (for(i <- 0 until latentK; j <- i until latentK) yield(X(i) * X(j))).toArray + // // (Xy, XtX) + // // }, + // // (a, b) => { + // // // The difference between the while loop and the zip is a FACTOR OF TWO in overall + // // // runtime + // // var i = 0 + // // while(i < a._1.length) { a._1(i) += b._1(i); i += 1 } + // // i = 0 + // // while(i < a._2.length) { a._2(i) += b._2(i); i += 1 } + // // a + // // // (a._1.zip(b._1).map{ case (q,r) => q+r }, a._2.zip(b._2).map{ case (q,r) => q+r }) + // // }, + // // (Array.empty[Double], Array.empty[Double]), // default value is empty + // // (vertex, accum) => { // apply + // // val XyArray = accum._1 + // // val XtXArray = accum._2 + // // if(XyArray.isEmpty) vertex.data // no neighbors + // // else { + // // val XtX = DenseMatrix.tabulate(latentK,latentK){ (i,j) => + // // (if(i < j) XtXArray(i + (j+1)*j/2) else XtXArray(i + (j+1)*j/2)) + + // // (if(i == j) lambda else 1.0F) //regularization + // // } + // // val Xy = DenseMatrix.create(latentK,1,XyArray) + // // val w = XtX \ Xy + // // w.data + // // } + // // }, + // // (me_id, edge) => true, + // // numIter, + // // gatherEdges = EdgeDirection.Both, + // // scatterEdges = EdgeDirection.Both, + // // vertex => vertex.id < maxUser).vertices + // // } + + // def main(args: Array[String]) = { + // val host = args(0) + // val taskType = args(1) + // val fname = args(2) + // val options = args.drop(3).map { arg => + // arg.dropWhile(_ == '-').split('=') match { + // case Array(opt, v) => (opt -> v) + // case _ => throw new IllegalArgumentException("Invalid argument: " + arg) + // } + // } + + // System.setProperty("spark.serializer", "spark.KryoSerializer") + // //System.setProperty("spark.shuffle.compress", "false") + // System.setProperty("spark.kryo.registrator", "spark.graph.GraphKryoRegistrator") + + // taskType match { + // case "pagerank" => { + + // var numIter = Int.MaxValue + // var isDynamic = false + // var tol:Float = 0.001F + // var outFname = "" + // var numVPart = 4 + // var numEPart = 4 + + // options.foreach{ + // case ("numIter", v) => numIter = v.toInt + // case ("dynamic", v) => isDynamic = v.toBoolean + // case ("tol", v) => tol = v.toFloat + // case ("output", v) => outFname = v + // case ("numVPart", v) => numVPart = v.toInt + // case ("numEPart", v) => numEPart = v.toInt + // case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) + // } + + // if(!isDynamic && numIter == Int.MaxValue) { + // println("Set number of iterations!") + // sys.exit(1) + // } + // println("======================================") + // println("| PageRank |") + // println("--------------------------------------") + // println(" Using parameters:") + // println(" \tDynamic: " + isDynamic) + // if(isDynamic) println(" \t |-> Tolerance: " + tol) + // println(" \tNumIter: " + numIter) + // println("======================================") + + // val sc = new SparkContext(host, "PageRank(" + fname + ")") + + // val graph = Graph.textFile(sc, fname, a => 1.0F).withPartitioner(numVPart, numEPart).cache() + + // val startTime = System.currentTimeMillis + // logInfo("GRAPHX: starting tasks") + // logInfo("GRAPHX: Number of vertices " + graph.vertices.count) + // logInfo("GRAPHX: Number of edges " + graph.edges.count) + + // val pr = Analytics.pagerank(graph, numIter) + // // val pr = if(isDynamic) Analytics.dynamicPagerank(graph, tol, numIter) + // // else Analytics.pagerank(graph, numIter) + // logInfo("GRAPHX: Total rank: " + pr.vertices.map{ case Vertex(id,r) => r }.reduce(_+_) ) + // if (!outFname.isEmpty) { + // println("Saving pageranks of pages to " + outFname) + // pr.vertices.map{case Vertex(id, r) => id + "\t" + r}.saveAsTextFile(outFname) + // } + // logInfo("GRAPHX: Runtime: " + ((System.currentTimeMillis - startTime)/1000.0) + " seconds") + // sc.stop() + // } + + // case "cc" => { + + // var numIter = Int.MaxValue + // var isDynamic = false + + // options.foreach{ + // case ("numIter", v) => numIter = v.toInt + // case ("dynamic", v) => isDynamic = v.toBoolean + // case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) + // } + + // 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 + ")") + // val graph = Graph.textFile(sc, fname, a => 1.0F) + // val cc = Analytics.connectedComponents(graph, numIter) + // // val cc = if(isDynamic) Analytics.dynamicConnectedComponents(graph, numIter) + // // else Analytics.connectedComponents(graph, numIter) + // println("Components: " + cc.vertices.map(_.data).distinct()) + + // sc.stop() + // } + + // case "shortestpath" => { + + // var numIter = Int.MaxValue + // var isDynamic = true + // var sources: List[Int] = List.empty + + // options.foreach{ + // case ("numIter", v) => numIter = v.toInt + // case ("dynamic", v) => isDynamic = v.toBoolean + // case ("source", v) => sources ++= List(v.toInt) + // case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) + // } + + + // if(!isDynamic && numIter == Int.MaxValue) { + // println("Set number of iterations!") + // sys.exit(1) + // } + + // if(sources.isEmpty) { + // println("No sources provided!") + // sys.exit(1) + // } + + // println("======================================") + // println("| Shortest Path |") + // println("--------------------------------------") + // println(" Using parameters:") + // println(" \tDynamic: " + isDynamic) + // println(" \tNumIter: " + numIter) + // println(" \tSources: [" + sources.mkString(", ") + "]") + // println("======================================") + + // val sc = new SparkContext(host, "ShortestPath(" + fname + ")") + // val graph = Graph.textFile(sc, fname, a => (if(a.isEmpty) 1.0F else a(0).toFloat ) ) + // val sp = Analytics.shortestPath(graph, sources, numIter) + // // val cc = if(isDynamic) Analytics.dynamicShortestPath(graph, sources, numIter) + // // else Analytics.shortestPath(graph, sources, numIter) + // println("Longest Path: " + sp.vertices.map(_.data).reduce(math.max(_,_))) + + // sc.stop() + // } + + + // // case "als" => { + + // // var numIter = 5 + // // var lambda = 0.01 + // // var latentK = 10 + // // var usersFname = "usersFactors.tsv" + // // var moviesFname = "moviesFname.tsv" + // // var numVPart = 4 + // // var numEPart = 4 + + // // options.foreach{ + // // case ("numIter", v) => numIter = v.toInt + // // case ("lambda", v) => lambda = v.toDouble + // // case ("latentK", v) => latentK = v.toInt + // // case ("usersFname", v) => usersFname = v + // // case ("moviesFname", v) => moviesFname = v + // // case ("numVPart", v) => numVPart = v.toInt + // // case ("numEPart", v) => numEPart = v.toInt + // // case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) + // // } + + // // println("======================================") + // // println("| Alternating Least Squares |") + // // println("--------------------------------------") + // // println(" Using parameters:") + // // println(" \tNumIter: " + numIter) + // // println(" \tLambda: " + lambda) + // // println(" \tLatentK: " + latentK) + // // println(" \tusersFname: " + usersFname) + // // println(" \tmoviesFname: " + moviesFname) + // // println("======================================") + + // // val sc = new SparkContext(host, "ALS(" + fname + ")") + // // val graph = Graph.textFile(sc, fname, a => a(0).toDouble ) + // // graph.numVPart = numVPart + // // graph.numEPart = numEPart + + // // val maxUser = graph.edges.map(_._1).reduce(math.max(_,_)) + // // val minMovie = graph.edges.map(_._2).reduce(math.min(_,_)) + // // assert(maxUser < minMovie) + + // // val factors = Analytics.alternatingLeastSquares(graph, latentK, lambda, numIter).cache + // // factors.filter(_._1 <= maxUser).map(r => r._1 + "\t" + r._2.mkString("\t")) + // // .saveAsTextFile(usersFname) + // // factors.filter(_._1 >= minMovie).map(r => r._1 + "\t" + r._2.mkString("\t")) + // // .saveAsTextFile(moviesFname) + + // // sc.stop() + // // } + + + // case _ => { + // println("Invalid task type.") + // } + // } + // } // /** // * Compute the PageRank of a graph returning the pagerank of each vertex as an RDD @@ -503,8 +509,6 @@ object Analytics extends Logging { // } - - // /** // * Compute the shortest path to a set of markers // */ @@ -532,8 +536,6 @@ object Analytics extends Logging { // } - - // /** // * // */ @@ -735,55 +737,55 @@ object Analytics extends Logging { // } - // case "als" => { - - // var numIter = 5 - // var lambda = 0.01 - // var latentK = 10 - // var usersFname = "usersFactors.tsv" - // var moviesFname = "moviesFname.tsv" - // var numVPart = 4 - // var numEPart = 4 - - // options.foreach{ - // case ("numIter", v) => numIter = v.toInt - // case ("lambda", v) => lambda = v.toDouble - // case ("latentK", v) => latentK = v.toInt - // case ("usersFname", v) => usersFname = v - // case ("moviesFname", v) => moviesFname = v - // case ("numVPart", v) => numVPart = v.toInt - // case ("numEPart", v) => numEPart = v.toInt - // case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) - // } - - // println("======================================") - // println("| Alternating Least Squares |") - // println("--------------------------------------") - // println(" Using parameters:") - // println(" \tNumIter: " + numIter) - // println(" \tLambda: " + lambda) - // println(" \tLatentK: " + latentK) - // println(" \tusersFname: " + usersFname) - // println(" \tmoviesFname: " + moviesFname) - // println("======================================") - - // val sc = new SparkContext(host, "ALS(" + fname + ")") - // val graph = Graph.textFile(sc, fname, a => a(0).toDouble ) - // graph.numVPart = numVPart - // graph.numEPart = numEPart - - // val maxUser = graph.edges.map(_._1).reduce(math.max(_,_)) - // val minMovie = graph.edges.map(_._2).reduce(math.min(_,_)) - // assert(maxUser < minMovie) - - // val factors = Analytics.alternatingLeastSquares(graph, latentK, lambda, numIter).cache - // factors.filter(_._1 <= maxUser).map(r => r._1 + "\t" + r._2.mkString("\t")) - // .saveAsTextFile(usersFname) - // factors.filter(_._1 >= minMovie).map(r => r._1 + "\t" + r._2.mkString("\t")) - // .saveAsTextFile(moviesFname) - - // sc.stop() - // } + // case "als" => { + + // var numIter = 5 + // var lambda = 0.01 + // var latentK = 10 + // var usersFname = "usersFactors.tsv" + // var moviesFname = "moviesFname.tsv" + // var numVPart = 4 + // var numEPart = 4 + + // options.foreach{ + // case ("numIter", v) => numIter = v.toInt + // case ("lambda", v) => lambda = v.toDouble + // case ("latentK", v) => latentK = v.toInt + // case ("usersFname", v) => usersFname = v + // case ("moviesFname", v) => moviesFname = v + // case ("numVPart", v) => numVPart = v.toInt + // case ("numEPart", v) => numEPart = v.toInt + // case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) + // } + + // println("======================================") + // println("| Alternating Least Squares |") + // println("--------------------------------------") + // println(" Using parameters:") + // println(" \tNumIter: " + numIter) + // println(" \tLambda: " + lambda) + // println(" \tLatentK: " + latentK) + // println(" \tusersFname: " + usersFname) + // println(" \tmoviesFname: " + moviesFname) + // println("======================================") + + // val sc = new SparkContext(host, "ALS(" + fname + ")") + // val graph = Graph.textFile(sc, fname, a => a(0).toDouble ) + // graph.numVPart = numVPart + // graph.numEPart = numEPart + + // val maxUser = graph.edges.map(_._1).reduce(math.max(_,_)) + // val minMovie = graph.edges.map(_._2).reduce(math.min(_,_)) + // assert(maxUser < minMovie) + + // val factors = Analytics.alternatingLeastSquares(graph, latentK, lambda, numIter).cache + // factors.filter(_._1 <= maxUser).map(r => r._1 + "\t" + r._2.mkString("\t")) + // .saveAsTextFile(usersFname) + // factors.filter(_._1 >= minMovie).map(r => r._1 + "\t" + r._2.mkString("\t")) + // .saveAsTextFile(moviesFname) + + // sc.stop() + // } // case _ => { diff --git a/graph/src/main/scala/spark/graph/GraphLab.scala b/graph/src/main/scala/spark/graph/GraphLab.scala index 504cd162abaeb..f89c2a39d79e0 100644 --- a/graph/src/main/scala/spark/graph/GraphLab.scala +++ b/graph/src/main/scala/spark/graph/GraphLab.scala @@ -33,12 +33,12 @@ object GraphLab { * @tparam A The type accumulated during the gather phase * @return the resulting graph after the algorithm converges */ - def apply[VD: ClassManifest, ED: ClassManifest, A: ClassManifest](graph: Graph[VD, ED])( + def iterate[VD: ClassManifest, ED: ClassManifest, A: ClassManifest](graph: Graph[VD, ED])( gatherFunc: (Vid, EdgeTriplet[VD, ED]) => A, mergeFunc: (A, A) => A, applyFunc: (Vertex[VD], Option[A]) => VD, scatterFunc: (Vid, EdgeTriplet[VD, ED]) => Boolean, - numIter: Int, + numIter: Int = Integer.MAX_VALUE, gatherDirection: EdgeDirection = EdgeDirection.In, scatterDirection: EdgeDirection = EdgeDirection.Out): Graph[VD, ED] = { From 7ae83f6ef4200eb992aa764fc6be987f55e1ba47 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Tue, 6 Aug 2013 14:05:54 -0700 Subject: [PATCH 051/531] Switching to Long vids instead of integers. This required a surprising number of changes since the fastutil library function names include the type (e.g., getLong() instead of just get()) --- .../src/main/scala/spark/graph/Analytics.scala | 6 +++--- .../scala/spark/graph/impl/EdgePartition.scala | 4 ++-- .../spark/graph/impl/EdgeTripletRDD.scala | 4 ++-- .../scala/spark/graph/impl/GraphImpl.scala | 18 +++++++++--------- graph/src/main/scala/spark/graph/package.scala | 6 ++++-- 5 files changed, 20 insertions(+), 18 deletions(-) diff --git a/graph/src/main/scala/spark/graph/Analytics.scala b/graph/src/main/scala/spark/graph/Analytics.scala index df4db603390c5..601a0785e1b2e 100644 --- a/graph/src/main/scala/spark/graph/Analytics.scala +++ b/graph/src/main/scala/spark/graph/Analytics.scala @@ -87,10 +87,10 @@ object Analytics extends Logging { def connectedComponents[VD: Manifest, ED: Manifest](graph: Graph[VD, ED]) = { val ccGraph = graph.mapVertices { case Vertex(vid, _) => vid } - GraphLab.iterate[Int, ED, Int](ccGraph)( + GraphLab.iterate(ccGraph)( (me_id, edge) => edge.otherVertex(me_id).data, // gather - (a: Int, b: Int) => math.min(a, b), // merge - (v, a: Option[Int]) => math.min(v.data, a.getOrElse(Integer.MAX_VALUE)), // apply + (a: Vid, b: Vid) => math.min(a, b), // merge + (v, a: Option[Vid]) => math.min(v.data, a.getOrElse(Long.MaxValue)), // apply (me_id, edge) => (edge.vertex(me_id).data < edge.otherVertex(me_id).data), // scatter gatherDirection = EdgeDirection.Both, scatterDirection = EdgeDirection.Both ) diff --git a/graph/src/main/scala/spark/graph/impl/EdgePartition.scala b/graph/src/main/scala/spark/graph/impl/EdgePartition.scala index 0e092541c9fd3..4e0d5f41b9060 100644 --- a/graph/src/main/scala/spark/graph/impl/EdgePartition.scala +++ b/graph/src/main/scala/spark/graph/impl/EdgePartition.scala @@ -16,8 +16,8 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) private var _data: Array[ED] = _ private var _dataBuilder = ArrayBuilder.make[ED] - val srcIds: IntArrayList = new IntArrayList - val dstIds: IntArrayList = new IntArrayList + val srcIds = new VertexArrayList + val dstIds = new VertexArrayList def data: Array[ED] = _data diff --git a/graph/src/main/scala/spark/graph/impl/EdgeTripletRDD.scala b/graph/src/main/scala/spark/graph/impl/EdgeTripletRDD.scala index f157dd40560b0..f6de8e59af0fd 100644 --- a/graph/src/main/scala/spark/graph/impl/EdgeTripletRDD.scala +++ b/graph/src/main/scala/spark/graph/impl/EdgeTripletRDD.scala @@ -63,11 +63,11 @@ class EdgeTripletRDD[VD: ClassManifest, ED: ClassManifest]( override def hasNext: Boolean = pos < edgePartition.size override def next() = { - e.src.id = edgePartition.srcIds.getInt(pos) + e.src.id = edgePartition.srcIds.getLong(pos) // assert(vmap.containsKey(e.src.id)) e.src.data = vmap.get(e.src.id) - e.dst.id = edgePartition.dstIds.getInt(pos) + e.dst.id = edgePartition.dstIds.getLong(pos) // assert(vmap.containsKey(e.dst.id)) e.dst.data = vmap.get(e.dst.id) diff --git a/graph/src/main/scala/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/spark/graph/impl/GraphImpl.scala index 775d0686e75ae..024e3e8d02b41 100644 --- a/graph/src/main/scala/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/spark/graph/impl/GraphImpl.scala @@ -147,8 +147,8 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( } } } - vmap.int2ObjectEntrySet().fastIterator().filter(!_.getValue()._2.isEmpty).map{ entry => - (entry.getIntKey(), entry.getValue()._2) + vmap.long2ObjectEntrySet().fastIterator().filter(!_.getValue()._2.isEmpty).map{ entry => + (entry.getLongKey(), entry.getValue()._2) } } .map{ case (vid, aOpt) => (vid, aOpt.get) } @@ -202,8 +202,8 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( } } } - vmap.int2ObjectEntrySet().fastIterator().filter(!_.getValue()._2.isEmpty).map{ entry => - (entry.getIntKey(), entry.getValue()._2) + vmap.long2ObjectEntrySet().fastIterator().filter(!_.getValue()._2.isEmpty).map{ entry => + (entry.getLongKey(), entry.getValue()._2) } } .map{ case (vid, aOpt) => (vid, aOpt.get) } @@ -327,16 +327,16 @@ object GraphImpl { // A key-value RDD. The key is a vertex id, and the value is a list of // partitions that contains edges referencing the vertex. - val vid2pid : RDD[(Int, Seq[Pid])] = eTable.mapPartitions { iter => + val vid2pid : RDD[(Vid, Seq[Pid])] = eTable.mapPartitions { iter => val (pid, edgePartition) = iter.next() - val vSet = new it.unimi.dsi.fastutil.ints.IntOpenHashSet + val vSet = new VertexSet var i = 0 while (i < edgePartition.srcIds.size) { - vSet.add(edgePartition.srcIds.getInt(i)) - vSet.add(edgePartition.dstIds.getInt(i)) + vSet.add(edgePartition.srcIds.getLong(i)) + vSet.add(edgePartition.dstIds.getLong(i)) i += 1 } - vSet.iterator.map { vid => (vid.intValue, pid) } + vSet.iterator.map { vid => (vid.toLong, pid) } }.groupByKey(partitioner) vertices diff --git a/graph/src/main/scala/spark/graph/package.scala b/graph/src/main/scala/spark/graph/package.scala index cf1b23ca5d4e1..d95dcdce0848c 100644 --- a/graph/src/main/scala/spark/graph/package.scala +++ b/graph/src/main/scala/spark/graph/package.scala @@ -2,10 +2,12 @@ package spark package object graph { - type Vid = Int + type Vid = Long type Pid = Int - type VertexHashMap[T] = it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap[T] + type VertexHashMap[T] = it.unimi.dsi.fastutil.longs.Long2ObjectOpenHashMap[T] + type VertexSet = it.unimi.dsi.fastutil.longs.LongOpenHashSet + type VertexArrayList = it.unimi.dsi.fastutil.longs.LongArrayList /** * Return the default null-like value for a data type T. From b454314e070a70e51b41c76d6a54bdd038d22418 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Tue, 6 Aug 2013 15:14:13 -0700 Subject: [PATCH 052/531] Added 2d partitioning --- .../scala/spark/graph/impl/GraphImpl.scala | 76 ++++++++++++++++++- 1 file changed, 74 insertions(+), 2 deletions(-) diff --git a/graph/src/main/scala/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/spark/graph/impl/GraphImpl.scala index 024e3e8d02b41..cbd1f27afb6da 100644 --- a/graph/src/main/scala/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/spark/graph/impl/GraphImpl.scala @@ -2,13 +2,16 @@ package spark.graph.impl import scala.collection.JavaConversions._ -import spark.{ClosureCleaner, HashPartitioner, RDD} +import spark.{ClosureCleaner, Partitioner, HashPartitioner, RDD} import spark.SparkContext._ import spark.graph._ import spark.graph.impl.GraphImpl._ + + + /** * A Graph RDD that supports computation on graphs. */ @@ -294,6 +297,67 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( object GraphImpl { + + protected def edgePartitionFunction1D(src: Vid, dst: Vid, numParts: Pid): Pid = { + val mixingPrime: Vid = 1125899906842597L + (math.abs(src) * mixingPrime).toInt % numParts + } + + + + /** + * This function implements a classic 2D-Partitioning of a sparse matrix. + * Suppose we have a graph with 11 vertices that we want to partition + * over 9 machines. We can use the following sparse matrix representation: + * + * __________________________________ + * v0 | P0 * | P1 | P2 * | + * v1 | **** | * | | + * v2 | ******* | ** | **** | + * v3 | ***** | * * | * | + * ---------------------------------- + * v4 | P3 * | P4 *** | P5 ** * | + * v5 | * * | * | | + * v6 | * | ** | **** | + * v7 | * * * | * * | * | + * ---------------------------------- + * v8 | P6 * | P7 * | P8 * *| + * v9 | * | * * | | + * v10 | * | ** | * * | + * v11 | * <-E | *** | ** | + * ---------------------------------- + * + * 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(numProc) by sqrt(numProc) blocks. Notice that edges + * adjacent to v11 can only be in the first colum 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(numProc) 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 effectively suffle 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. + * + * + */ + protected def edgePartitionFunction2D(src: Vid, dst: Vid, + numParts: Pid, ceilSqrtNumParts: Pid): Pid = { + val mixingPrime: Vid = 1125899906842597L + val col: Pid = ((math.abs(src) * mixingPrime) % ceilSqrtNumParts).toInt + val row: Pid = ((math.abs(dst) * mixingPrime) % ceilSqrtNumParts).toInt + (col * ceilSqrtNumParts + row) % numParts + } + + /** * Create the edge table RDD, which is much more efficient for Java heap storage than the * normal edges data structure (RDD[(Vid, Vid, ED)]). @@ -304,10 +368,18 @@ object GraphImpl { */ protected def createETable[ED: ClassManifest](edges: RDD[Edge[ED]], numPartitions: Int) : RDD[(Pid, EdgePartition[ED])] = { + val ceilSqrt: Pid = math.ceil(math.sqrt(numPartitions)).toInt + edges .map { e => // Random partitioning based on the source vertex id. - (math.abs(e.src) % numPartitions, (e.src, e.dst, e.data)) + // val part: Pid = edgePartitionFunction1D(e.src, e.dst, numPartitions) + val part: Pid = edgePartitionFunction2D(e.src, e.dst, numPartitions, ceilSqrt) + + // Should we be using 3-tuple or an optimized class + (part, (e.src, e.dst, e.data)) + // (math.abs(e.src) % numPartitions, (e.src, e.dst, e.data)) + } .partitionBy(new HashPartitioner(numPartitions)) .mapPartitionsWithIndex({ (pid, iter) => From ddf126edad11f0a3e0c854beb32ec9543e98bd06 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Tue, 6 Aug 2013 17:48:04 -0700 Subject: [PATCH 053/531] added subgraph --- graph/src/main/scala/spark/graph/Graph.scala | 29 +++++++++++++++++++ .../scala/spark/graph/impl/GraphImpl.scala | 17 +++++++++-- .../test/scala/spark/graph/GraphSuite.scala | 3 +- 3 files changed, 46 insertions(+), 3 deletions(-) diff --git a/graph/src/main/scala/spark/graph/Graph.scala b/graph/src/main/scala/spark/graph/Graph.scala index f51171f0d627e..095ac2f7d9c73 100644 --- a/graph/src/main/scala/spark/graph/Graph.scala +++ b/graph/src/main/scala/spark/graph/Graph.scala @@ -141,6 +141,13 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { def mapTriplets[ED2: ClassManifest]( map: EdgeTriplet[VD, ED] => ED2): Graph[VD, ED2] + + /** + * Remove edges conntecting vertices that are not in the graph. + * + * @todo remove this function and ensure that for a graph G=(V,E): + * if (u,v) in E then u in V and v in V + */ def correctEdges(): Graph[VD, ED] /** @@ -150,6 +157,28 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { */ def reverse: Graph[VD, ED] + + /** + * This function takes a vertex and edge predicate and constructs the subgraph + * that consists of vertices and edges that satisfy the predict. The resulting + * graph contains the vertices and edges that satisfy: + * + * V' = {v : for all v in V where vpred(v)} + * E' = {(u,v): for all (u,v) in E where epred((u,v)) && vpred(u) && vpred(v)} + * + * @param epred the edge predicate which takes a triplet and evaluates to true + * if the edge is to remain in the subgraph. Note that only edges in which both + * vertices satisfy the vertex predicate are considered. + * + * @param vpred the vertex predicate which takes a vertex object and evaluates + * to true if the vertex is to be included in the subgraph + * + * @return the subgraph containing only the vertices and edges that satisfy the + * predicates. + */ + def subgraph(epred: EdgeTriplet[VD,ED] => Boolean = (_ => true), + vpred: Vertex[VD] => Boolean = (_ => true) ): Graph[VD, ED] + /** * This function is used to compute a statistic for the neighborhood of each * vertex. diff --git a/graph/src/main/scala/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/spark/graph/impl/GraphImpl.scala index cbd1f27afb6da..08fc016a4300b 100644 --- a/graph/src/main/scala/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/spark/graph/impl/GraphImpl.scala @@ -101,6 +101,19 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( Graph(vertices, newEdges) } + + override def subgraph(epred: EdgeTriplet[VD,ED] => Boolean = (_ => true), + vpred: Vertex[VD] => Boolean = (_ => true) ): Graph[VD, ED] = { + + // Restrict the set of vertices to those that satisfy the vertex predicate + val newVertices = vertices.filter(vpred) + // Restrict the set of edges to those that satisfy the vertex and the edge predicate. + val newEdges = triplets.filter(t => vpred(t.src) && vpred(t.dst) && epred(t)) + .map( t => Edge(t.src.id, t.dst.id, t.data) ) + + new GraphImpl(newVertices, newEdges) + } + ////////////////////////////////////////////////////////////////////////////////////////////////// // Lower level transformation methods ////////////////////////////////////////////////////////////////////////////////////////////////// @@ -303,7 +316,7 @@ object GraphImpl { (math.abs(src) * mixingPrime).toInt % numParts } - + /** * This function implements a classic 2D-Partitioning of a sparse matrix. @@ -340,7 +353,7 @@ object GraphImpl { * balance we first multiply each vertex id by a large prime * to effectively suffle the vertex locations. * - * One of the limitations of this approach is that the number of + * 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 diff --git a/graph/src/test/scala/spark/graph/GraphSuite.scala b/graph/src/test/scala/spark/graph/GraphSuite.scala index 87c8c158af581..ba438a720791d 100644 --- a/graph/src/test/scala/spark/graph/GraphSuite.scala +++ b/graph/src/test/scala/spark/graph/GraphSuite.scala @@ -12,7 +12,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { } - test("joinVertices") { + /* test("joinVertices") { sc = new SparkContext("local", "test") val vertices = sc.parallelize(Seq(Vertex(1, "one"), Vertex(2, "two"), Vertex(3, "three")), 2) val edges = sc.parallelize((Seq(Edge(1, 2, "onetwo")))) @@ -29,6 +29,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { val e = g1.edges.collect() assert(e(0).data === "onetwo") } + */ // test("graph partitioner") { // sc = new SparkContext("local", "test") From 5ccb60d467f58c104f37e05e99a50fdf06301e5e Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Sun, 11 Aug 2013 14:49:22 -0700 Subject: [PATCH 054/531] Working on graph test suite --- graph/src/main/scala/spark/graph/Graph.scala | 11 ++++++++++- graph/src/test/scala/spark/graph/GraphSuite.scala | 12 +++++++++++- 2 files changed, 21 insertions(+), 2 deletions(-) diff --git a/graph/src/main/scala/spark/graph/Graph.scala b/graph/src/main/scala/spark/graph/Graph.scala index 095ac2f7d9c73..594b3b54955c6 100644 --- a/graph/src/main/scala/spark/graph/Graph.scala +++ b/graph/src/main/scala/spark/graph/Graph.scala @@ -179,6 +179,15 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { def subgraph(epred: EdgeTriplet[VD,ED] => Boolean = (_ => true), vpred: Vertex[VD] => Boolean = (_ => true) ): Graph[VD, ED] + + // /** + // * Combine the attrributes of edges connecting the same vertices. + // * + // * @todo Do we want to support this function + // */ + // def combineEdges(reduce: (ED, ED) => ED): Graph[VD, ED] + + /** * This function is used to compute a statistic for the neighborhood of each * vertex. @@ -360,7 +369,7 @@ object Graph { import spark.graph.impl._ import spark.SparkContext._ - def apply(rawEdges: RDD[(Int, Int)], uniqueEdges: Boolean = true): Graph[Int, Int] = { + def apply(rawEdges: RDD[(Vid, Vid)], uniqueEdges: Boolean = true): Graph[Int, Int] = { // Reduce to unique edges. val edges: RDD[Edge[Int]] = if (uniqueEdges) { diff --git a/graph/src/test/scala/spark/graph/GraphSuite.scala b/graph/src/test/scala/spark/graph/GraphSuite.scala index ba438a720791d..804542f1bd9cc 100644 --- a/graph/src/test/scala/spark/graph/GraphSuite.scala +++ b/graph/src/test/scala/spark/graph/GraphSuite.scala @@ -3,11 +3,21 @@ package spark.graph import org.scalatest.FunSuite import spark.SparkContext -import spark.graph.impl.GraphImpl +import spark.graph._ + class GraphSuite extends FunSuite with LocalSparkContext { +// val sc = new SparkContext("local[4]", "test") + + test("Graph Creation") { + val rawEdges = (0L to 100L).zip((1L to 99L) :+ 0L) + val edges = sc.parallelize(rawEdges) + val graph = Graph(edges) + assert( graph.edges.count() === rawEdges.size ) + } + test("aggregateNeighbors") { } From f2b8dd3929ce2a8ccfea8bcf05b3c72a7acd63fd Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Tue, 13 Aug 2013 14:21:49 -0700 Subject: [PATCH 055/531] second indexedrdd design --- .../main/scala/spark/PairRDDFunctions.scala | 4 + .../src/main/scala/spark/rdd/IndexedRDD.scala | 132 ++++++++++++++++++ 2 files changed, 136 insertions(+) create mode 100644 core/src/main/scala/spark/rdd/IndexedRDD.scala diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index 6b0cc2fbf15e6..28b46990f8517 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -715,6 +715,10 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( */ def values: RDD[V] = self.map(_._2) + + def index(existingIndex: RDD[JHashMap[K,Int]] = null): IndexedRDD[K,V] = + IndexedRDD(self, existingIndex) + private[spark] def getKeyClass() = implicitly[ClassManifest[K]].erasure private[spark] def getValueClass() = implicitly[ClassManifest[V]].erasure diff --git a/core/src/main/scala/spark/rdd/IndexedRDD.scala b/core/src/main/scala/spark/rdd/IndexedRDD.scala new file mode 100644 index 0000000000000..55ca353618e4d --- /dev/null +++ b/core/src/main/scala/spark/rdd/IndexedRDD.scala @@ -0,0 +1,132 @@ +/* + * 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 spark.rdd + +import java.util.{HashMap => JHashMap, BitSet => JBitSet, HashSet => JHashSet} + +import scala.collection.JavaConversions._ +import scala.collection.mutable.ArrayBuffer + + +import spark.{Utils, OneToOneDependency, RDD, SparkContext, Partition, TaskContext} + +import spark.PairRDDFunctions +import spark.SparkContext._ + + +// import java.io.{ObjectOutputStream, IOException} + + +/** + * An IndexedRDD is an RDD[(K,V)] where each K is unique. + * + * The IndexedRDD contains an index datastructure that can + * be used to accelerate join and aggregation operations. + */ +class IndexedRDD[K: ClassManifest, V: ClassManifest]( + sc: SparkContext, + val indexRDD: RDD[ JHashMap[K, Int] ], + val valuesRDD: RDD[ Array[Seq[V]] ]) + extends RDD[(K, V)](sc, + List(new OneToOneDependency(indexRDD), new OneToOneDependency(valuesRDD)) ) { + + + + val tuples = new ZippedRDD[ JHashMap[K, Int], Array[Seq[V]] ](sc, indexRDD, valuesRDD) + + + override val partitioner = indexRDD.partitioner + override def getPartitions: Array[Partition] = tuples.getPartitions + override def getPreferredLocations(s: Partition): Seq[String] = tuples.getPreferredLocations(s) + + + + + + /** + * Provide the RDD[(K,V)] equivalent output. + */ + override def compute(part: Partition, context: TaskContext): Iterator[(K, V)] = { + tuples.compute(part, context).flatMap { case (indexMap, values) => + // Walk the index to construct the key, value pairs + indexMap.iterator + // Extract rows with key value pairs and indicators + .map{ case (k, ind) => (k, values(ind)) } + // Remove tuples that aren't actually present in the array + .filter{ case (_, valar) => valar != null } + // Extract the pair (removing the indicator from the tuple) + .flatMap{ case (k, valar) => valar.map(v => (k,v))} + } + } +} + +object IndexedRDD { + def apply[K: ClassManifest, V: ClassManifest]( + tbl: RDD[(K,V)], + existingIndex: RDD[JHashMap[K,Int]] = null ): IndexedRDD[K, V] = { + + if(existingIndex == null) { + // build th index + val groups = tbl.groupByKey().mapPartitions( iter => { + val indexMap = new JHashMap[K, Int]() + val values = new ArrayBuffer[Seq[V]]() + for((k,ar) <- iter){ + val ind = values.size + indexMap.put(k, ind) + values.append(ar) + } + List((indexMap, values.toArray)).iterator + }, true).cache + // extract the index and the values + val index = groups.mapPartitions(_.map{ case (kMap,vAr) => kMap }, true) + val values = groups.mapPartitions(_.map{ case (kMap,vAr) => vAr }, true) + new IndexedRDD[K,V](tbl.context, index, values) + } else { + val index = existingIndex + // Shuffle the table according to the index (if necessary) + val shuffledTbl = + if (tbl.partitioner == Some(index.partitioner)) { + tbl + } else { + new ShuffledRDD[K,V](tbl, index.partitioner.get()) + } + + // Use the index to build the new values table + val values = index.zipPartitions[ (K, Seq[V]), Array[Seq[V]] ]( + (indexIter, tblIter) => { + // There is only one map + val index: JHashMap[K,Int] = iter.next() + assert(!iter.hasNext()) + val values = new Array[Seq[V]](index.size) + for((k,a) <- tblIter) { + assert(index.contains(k)) + values(index.get(k)) = a + } + values + }, shuffleTbl) + + new IndexedRDD[K,V](index, values) + } + } + +} + + + + + From b71d4febbc4ed4760a3e2ddeecf9e677b5742635 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Wed, 14 Aug 2013 15:25:56 -0700 Subject: [PATCH 056/531] Finished early prototype of IndexedRDD --- .../src/main/scala/spark/rdd/IndexedRDD.scala | 272 ++++++++++++++++-- 1 file changed, 251 insertions(+), 21 deletions(-) diff --git a/core/src/main/scala/spark/rdd/IndexedRDD.scala b/core/src/main/scala/spark/rdd/IndexedRDD.scala index 55ca353618e4d..0f6e29ad58979 100644 --- a/core/src/main/scala/spark/rdd/IndexedRDD.scala +++ b/core/src/main/scala/spark/rdd/IndexedRDD.scala @@ -27,11 +27,12 @@ import spark.{Utils, OneToOneDependency, RDD, SparkContext, Partition, TaskConte import spark.PairRDDFunctions import spark.SparkContext._ +import spark.SparkException +import spark.Partitioner // import java.io.{ObjectOutputStream, IOException} - /** * An IndexedRDD is an RDD[(K,V)] where each K is unique. * @@ -39,22 +40,238 @@ import spark.SparkContext._ * be used to accelerate join and aggregation operations. */ class IndexedRDD[K: ClassManifest, V: ClassManifest]( - sc: SparkContext, - val indexRDD: RDD[ JHashMap[K, Int] ], - val valuesRDD: RDD[ Array[Seq[V]] ]) - extends RDD[(K, V)](sc, - List(new OneToOneDependency(indexRDD), new OneToOneDependency(valuesRDD)) ) { + val index: RDD[ JHashMap[K, Int] ], + val valuesRDD: RDD[ Seq[Seq[V]] ]) + extends RDD[(K, V)](index.context, + List(new OneToOneDependency(index), new OneToOneDependency(valuesRDD)) ) { - val tuples = new ZippedRDD[ JHashMap[K, Int], Array[Seq[V]] ](sc, indexRDD, valuesRDD) + val tuples = new ZippedRDD[JHashMap[K, Int], Seq[Seq[V]]](index.context, index, valuesRDD) - override val partitioner = indexRDD.partitioner + override val partitioner = index.partitioner override def getPartitions: Array[Partition] = tuples.getPartitions override def getPreferredLocations(s: Partition): Seq[String] = tuples.getPreferredLocations(s) + /** + * Construct a new IndexedRDD that is indexed by only the keys in the RDD + */ + def reindex(): IndexedRDD[K,V] = IndexedRDD(this) + + + /** + * Pass each value in the key-value pair RDD through a map function without changing the keys; + * this also retains the original RDD's partitioning. + */ + def mapValues[U: ClassManifest](f: V => U): IndexedRDD[K, U] = { + val cleanF = index.context.clean(f) + val newValues = valuesRDD.mapPartitions(_.map{ values => + values.map{_.map(x => f(x))} + }, true) + new IndexedRDD[K,U](index, newValues) + } + + + /** + * Pass each value in the key-value pair RDD through a flatMap function without changing the + * keys; this also retains the original RDD's partitioning. + */ + def flatMapValues[U: ClassManifest](f: V => TraversableOnce[U]): IndexedRDD[K,U] = { + val cleanF = index.context.clean(f) + val newValues = valuesRDD.mapPartitions(_.map{ values => + values.map{_.flatMap(x => f(x))} + }, true) + new IndexedRDD[K,U](index, newValues) + } + + + /** + * Generic function to combine the elements for each key using a custom set of aggregation + * functions. Turns an RDD[(K, V)] into a result of type RDD[(K, C)], for a "combined type" C + * Note that V and C can be different -- for example, one might group an RDD of type + * (Int, Int) into an RDD of type (Int, Seq[Int]). Users provide three functions: + * + * - `createCombiner`, which turns a V into a C (e.g., creates a one-element list) + * - `mergeValue`, to merge a V into a C (e.g., adds it to the end of a list) + * - `mergeCombiners`, to combine two C's into a single one. + * + * In addition, users can control the partitioning of the output RDD, and whether to perform + * map-side aggregation (if a mapper can produce multiple items with the same key). + */ + def combineByKey[C: ClassManifest](createCombiner: V => C, + mergeValue: (C, V) => C, + mergeCombiners: (C, C) => C, + partitioner: Partitioner, + mapSideCombine: Boolean = true, + serializerClass: String = null): IndexedRDD[K, C] = { + val newValues = valuesRDD.mapPartitions( + _.map{ groups: Seq[Seq[V]] => + groups.map{ group: Seq[V] => + if(group != null && !group.isEmpty) { + val c: C = createCombiner(group.head) + val sum: C = group.tail.foldLeft(c)(mergeValue) + Seq(sum) + } else { + null + } + } + }, true) + new IndexedRDD[K,C](index, newValues) + } + + + + /** + * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the + * list of values for that key in `this` as well as `other`. + */ + def cogroup[W: ClassManifest](other: RDD[(K, W)], partitionerUnused: Partitioner): + IndexedRDD[K, (Seq[V], Seq[W])] = { + other match { + case other: IndexedRDD[_, _] if other.index == index => { + // if both RDDs share exactly the same index and therefore the same super set of keys + // then we simply merge the value RDDs. + // However it is possible that both RDDs are missing a value for a given key in + // which case the returned RDD should have a null value + val newValues = + valuesRDD.zipPartitions[ Seq[Seq[W]], Seq[Seq[(Seq[V], Seq[W])]] ]( + (thisIter, otherIter) => { + val thisValues: Seq[Seq[V]] = thisIter.next() + assert(!thisIter.hasNext()) + val otherValues: Seq[Seq[W]] = otherIter.next() + assert(!otherIter.hasNext()) + // Zip the values and if both arrays are null then the key is not present and + // so the resulting value must be null (not a tuple of empty sequences) + val tmp: Seq[Seq[(Seq[V], Seq[W])]] = thisValues.view.zip(otherValues).map{ + case (null, null) => null // The key is not present in either RDD + case (a, null) => Seq((a, Seq.empty[W])) + case (null, b) => Seq((Seq.empty[V], b)) + case (a, b) => Seq((a,b)) + }.toSeq + List(tmp).iterator + }, other.valuesRDD) + new IndexedRDD[K, (Seq[V], Seq[W])](index, newValues) + } + case other: IndexedRDD[_, _] if other.index.partitioner == index.partitioner => { + // If both RDDs are indexed using different indices but with the same partitioners + // then we we need to first merge the indicies and then use the merged index to + // merge the values. + val newIndex = + index.zipPartitions[JHashMap[K,Int], JHashMap[K,Int]]( (thisIter, otherIter) => { + val thisIndex = thisIter.next() + assert(!thisIter.hasNext()) + val otherIndex = otherIter.next() + assert(!otherIter.hasNext()) + val newIndex = new JHashMap[K, Int]() + // @todo Merge only the keys that correspond to non-null values + // Merge the keys + newIndex.putAll(thisIndex) + newIndex.putAll(otherIndex) + // We need to rekey the index + var ctr = 0 + for(e <- newIndex.entrySet) { + e.setValue(ctr) + ctr += 1 + } + List(newIndex).iterator + }, other.index).cache() + // Use the new index along with the this and the other indices to merge the values + val newValues = + newIndex.zipPartitions[ + (JHashMap[K, Int], Seq[Seq[V]]), + (JHashMap[K, Int], Seq[Seq[W]]), + Seq[Seq[(Seq[V],Seq[W])]] ]( + (newIndexIter, thisTuplesIter, otherTuplesIter) => { + // Get the new index for this partition + val newIndex = newIndexIter.next() + assert(!newIndexIter.hasNext()) + // Get the corresponding indicies and values for this and the other IndexedRDD + val (thisIndex, thisValues) = thisTuplesIter.next() + assert(!thisTuplesIter.hasNext()) + val (otherIndex, otherValues) = otherTuplesIter.next() + assert(!otherTuplesIter.hasNext()) + // Preallocate the new Values array + val newValues = new Array[Seq[(Seq[V],Seq[W])]](newIndex.size) + // Lookup the sequences in both submaps + for((k,ind) <- newIndex) { + val thisSeq = if(thisIndex.contains(k)) thisValues(thisIndex.get(k)) else null + val otherSeq = if(otherIndex.contains(k)) otherValues(otherIndex.get(k)) else null + // if either of the sequences is not null then the key was in one of the two tables + // and so the value should appear in the returned table + newValues(ind) = (thisSeq, otherSeq) match { + case (null, null) => null + case (a, null) => Seq( (a, Seq.empty[W]) ) + case (null, b) => Seq( (Seq.empty[V], b) ) + case (a, b) => Seq( (a,b) ) + } + } + List(newValues.toSeq).iterator + }, tuples, other.tuples) + new IndexedRDD(newIndex, newValues) + } + case _ => { + // Get the partitioner from the index + val partitioner = index.partitioner match { + case Some(p) => p + case None => throw new SparkException("An index must have a partitioner.") + } + // Shuffle the other RDD using the partitioner for this index + val otherShuffled = + if (other.partitioner == Some(partitioner)) { + other + } else { + new ShuffledRDD[K,W](other, partitioner) + } + // Join the other RDD with this RDD building a new valueset and new index on the fly + val groups = tuples.zipPartitions[(K, W), (JHashMap[K, Int], Seq[Seq[(Seq[V],Seq[W])]]) ]( + (thisTuplesIter, otherTuplesIter) => { + // Get the corresponding indicies and values for this IndexedRDD + val (thisIndex, thisValues) = thisTuplesIter.next() + assert(!thisTuplesIter.hasNext()) + // Construct a new index + val newIndex = thisIndex.clone().asInstanceOf[JHashMap[K, Int]] + // Construct a new array Buffer to store the values + val newValues = new ArrayBuffer[(Seq[V], ArrayBuffer[W])](thisValues.size) + // populate the newValues with the values in this IndexedRDD + for((k,i) <- thisIndex) { + if(thisValues(i) != null) { + newValues(i) = (thisValues(i), new ArrayBuffer[W]()) + } + } + // Now iterate through the other tuples updating the map + for((k,w) <- otherTuplesIter){ + if(!newIndex.contains(k)) { + // update the index + val ind = newIndex.size + newIndex.put(k, ind) + // Create the buffer for w + val wbuffer = new ArrayBuffer[W]() + wbuffer.append(w) + // Update the values + newValues.append( (Seq.empty[V], wbuffer) ) + } else { + val ind = newIndex.get(k) + newValues(ind)._2.append(w) + } + } + // Finalize the new values array + val newValuesArray: Seq[Seq[(Seq[V],Seq[W])]] = + newValues.view.map{ case (s, ab) => Seq((s, ab.toSeq)) }.toSeq + List( (newIndex, newValuesArray) ).iterator + }, otherShuffled).cache() + + // Extract the index and values from the above RDD + val newIndex = groups.mapPartitions(_.map{ case (kMap,vAr) => kMap }, true) + val newValues = groups.mapPartitions(_.map{ case (kMap,vAr) => vAr }, true) + + new IndexedRDD[K, (Seq[V], Seq[W])](newIndex, newValues) + + } + } + } + @@ -68,12 +285,16 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( // Extract rows with key value pairs and indicators .map{ case (k, ind) => (k, values(ind)) } // Remove tuples that aren't actually present in the array - .filter{ case (_, valar) => valar != null } + .filter{ case (_, valar) => valar != null && !valar.isEmpty()} // Extract the pair (removing the indicator from the tuple) .flatMap{ case (k, valar) => valar.map(v => (k,v))} } } -} + +} // End of IndexedRDD + + + object IndexedRDD { def apply[K: ClassManifest, V: ClassManifest]( @@ -90,35 +311,44 @@ object IndexedRDD { indexMap.put(k, ind) values.append(ar) } - List((indexMap, values.toArray)).iterator + List((indexMap, values.toSeq)).iterator }, true).cache // extract the index and the values val index = groups.mapPartitions(_.map{ case (kMap,vAr) => kMap }, true) val values = groups.mapPartitions(_.map{ case (kMap,vAr) => vAr }, true) - new IndexedRDD[K,V](tbl.context, index, values) + new IndexedRDD[K,V](index, values) } else { val index = existingIndex + val partitioner = index.partitioner match { + case Some(p) => p + case None => throw new SparkException("An index must have a partitioner.") + } + // Shuffle the table according to the index (if necessary) val shuffledTbl = - if (tbl.partitioner == Some(index.partitioner)) { + if (tbl.partitioner == Some(partitioner)) { tbl } else { - new ShuffledRDD[K,V](tbl, index.partitioner.get()) + new ShuffledRDD[K,V](tbl, partitioner) } // Use the index to build the new values table - val values = index.zipPartitions[ (K, Seq[V]), Array[Seq[V]] ]( + val values = index.zipPartitions[ (K, V), Seq[Seq[V]] ]( (indexIter, tblIter) => { // There is only one map - val index: JHashMap[K,Int] = iter.next() - assert(!iter.hasNext()) + val index: JHashMap[K,Int] = indexIter.next() + assert(!indexIter.hasNext()) val values = new Array[Seq[V]](index.size) - for((k,a) <- tblIter) { + for((k,v) <- tblIter) { assert(index.contains(k)) - values(index.get(k)) = a + val ind = index(k) + if(values(ind) == null){ + values(ind) = new ArrayBuffer[V]() + } + values(ind).asInstanceOf[ArrayBuffer[V]].append(v) } - values - }, shuffleTbl) + List(values.toSeq).iterator + }, shuffledTbl) new IndexedRDD[K,V](index, values) } From 54b54903c316096b0ef1cda48670d32adeb746e1 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Wed, 14 Aug 2013 16:35:20 -0700 Subject: [PATCH 057/531] Adding testing code for indexedrdd --- .../src/main/scala/spark/rdd/IndexedRDD.scala | 24 +- .../test/scala/spark/IndexedRDDSuite.scala | 354 ++++++++++++++++++ 2 files changed, 372 insertions(+), 6 deletions(-) create mode 100644 core/src/test/scala/spark/IndexedRDDSuite.scala diff --git a/core/src/main/scala/spark/rdd/IndexedRDD.scala b/core/src/main/scala/spark/rdd/IndexedRDD.scala index 0f6e29ad58979..a6852f3f8a611 100644 --- a/core/src/main/scala/spark/rdd/IndexedRDD.scala +++ b/core/src/main/scala/spark/rdd/IndexedRDD.scala @@ -44,6 +44,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( val valuesRDD: RDD[ Seq[Seq[V]] ]) extends RDD[(K, V)](index.context, List(new OneToOneDependency(index), new OneToOneDependency(valuesRDD)) ) { + //with PairRDDFunctions[K,V] { @@ -129,6 +130,8 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( */ def cogroup[W: ClassManifest](other: RDD[(K, W)], partitionerUnused: Partitioner): IndexedRDD[K, (Seq[V], Seq[W])] = { + //RDD[(K, (Seq[V], Seq[W]))] = { + assert(false) other match { case other: IndexedRDD[_, _] if other.index == index => { // if both RDDs share exactly the same index and therefore the same super set of keys @@ -302,14 +305,23 @@ object IndexedRDD { existingIndex: RDD[JHashMap[K,Int]] = null ): IndexedRDD[K, V] = { if(existingIndex == null) { - // build th index - val groups = tbl.groupByKey().mapPartitions( iter => { + // Shuffle the table (if necessary) + val shuffledTbl = + if (tbl.partitioner.isEmpty) { + new ShuffledRDD[K,V](tbl, Partitioner.defaultPartitioner(tbl)) + } else { tbl } + + val groups = shuffledTbl.mapPartitions( iter => { val indexMap = new JHashMap[K, Int]() val values = new ArrayBuffer[Seq[V]]() - for((k,ar) <- iter){ - val ind = values.size - indexMap.put(k, ind) - values.append(ar) + for((k,v) <- iter){ + if(!indexMap.contains(k)) { + val ind = indexMap.size + indexMap.put(k, ind) + values.append(new ArrayBuffer[V]()) + } + val ind = indexMap.get(k) + values(ind).asInstanceOf[ArrayBuffer[V]].append(v) } List((indexMap, values.toSeq)).iterator }, true).cache diff --git a/core/src/test/scala/spark/IndexedRDDSuite.scala b/core/src/test/scala/spark/IndexedRDDSuite.scala new file mode 100644 index 0000000000000..aacb6423ee17f --- /dev/null +++ b/core/src/test/scala/spark/IndexedRDDSuite.scala @@ -0,0 +1,354 @@ +/* + * 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 spark + + +import org.scalatest.FunSuite +import org.scalatest.prop.Checkers +import org.scalacheck.Arbitrary._ +import org.scalacheck.Gen +import org.scalacheck.Prop._ + +import com.google.common.io.Files + +import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.HashSet + +import spark.rdd.ShuffledRDD +import spark.SparkContext._ + +class IndexedRDDSuite extends FunSuite with SharedSparkContext { + + def lineage(rdd: RDD[_]): collection.mutable.HashSet[RDD[_]] = { + val set = new collection.mutable.HashSet[RDD[_]] + def visit(rdd: RDD[_]) { + for (dep <- rdd.dependencies) { + set += dep.rdd + visit(dep.rdd) + } + } + visit(rdd) + set + } + + test("groupByKey") { + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1))).index() + val groups = pairs.groupByKey().collect() + assert(groups.size === 2) + val valuesFor1 = groups.find(_._1 == 1).get._2 + assert(valuesFor1.toList.sorted === List(1, 2, 3)) + val valuesFor2 = groups.find(_._1 == 2).get._2 + assert(valuesFor2.toList.sorted === List(1)) + } + + test("groupByKey with duplicates") { + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))).index() + val groups = pairs.groupByKey().collect() + assert(groups.size === 2) + val valuesFor1 = groups.find(_._1 == 1).get._2 + assert(valuesFor1.toList.sorted === List(1, 1, 2, 3)) + val valuesFor2 = groups.find(_._1 == 2).get._2 + assert(valuesFor2.toList.sorted === List(1)) + } + + test("groupByKey with negative key hash codes") { + val pairs = sc.parallelize(Array((-1, 1), (-1, 2), (-1, 3), (2, 1))).index() + val groups = pairs.groupByKey().collect() + assert(groups.size === 2) + val valuesForMinus1 = groups.find(_._1 == -1).get._2 + assert(valuesForMinus1.toList.sorted === List(1, 2, 3)) + val valuesFor2 = groups.find(_._1 == 2).get._2 + assert(valuesFor2.toList.sorted === List(1)) + } + + test("groupByKey with many output partitions") { + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1))).index() + val groups = pairs.groupByKey(10).collect() + assert(groups.size === 2) + val valuesFor1 = groups.find(_._1 == 1).get._2 + assert(valuesFor1.toList.sorted === List(1, 2, 3)) + val valuesFor2 = groups.find(_._1 == 2).get._2 + assert(valuesFor2.toList.sorted === List(1)) + } + + test("reduceByKey") { + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))).index() + val sums = pairs.reduceByKey(_+_).collect() + assert(sums.toSet === Set((1, 7), (2, 1))) + } + + test("reduceByKey with collectAsMap") { + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))).index() + val sums = pairs.reduceByKey(_+_).collectAsMap() + assert(sums.size === 2) + assert(sums(1) === 7) + assert(sums(2) === 1) + } + + test("reduceByKey with many output partitons") { + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))).index() + val sums = pairs.reduceByKey(_+_, 10).collect() + assert(sums.toSet === Set((1, 7), (2, 1))) + } + + test("reduceByKey with partitioner") { + val p = new Partitioner() { + def numPartitions = 2 + def getPartition(key: Any) = key.asInstanceOf[Int] + } + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 1), (0, 1))).partitionBy(p).index() + val sums = pairs.reduceByKey(_+_) + assert(sums.collect().toSet === Set((1, 4), (0, 1))) + assert(sums.partitioner === Some(p)) + // count the dependencies to make sure there is only 1 ShuffledRDD + val deps = lineage(sums) + + assert(deps.filter(_.isInstanceOf[ShuffledRDD[_,_]]).size === 1) // ShuffledRDD, ParallelCollection + } + + test("join") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) + val joined = rdd1.join(rdd2).collect() + assert(joined.size === 4) + assert(joined.toSet === Set( + (1, (1, 'x')), + (1, (2, 'x')), + (2, (1, 'y')), + (2, (1, 'z')) + )) + } + + + test("joinIndexVsPair") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).index() + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) + val joined = rdd1.join(rdd2).collect() + assert(joined.size === 4) + assert(joined.toSet === Set( + (1, (1, 'x')), + (1, (2, 'x')), + (2, (1, 'y')), + (2, (1, 'z')) + )) + } + + test("joinIndexVsIndex") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).index() + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).index() + val joined = rdd1.join(rdd2).collect() + assert(joined.size === 4) + assert(joined.toSet === Set( + (1, (1, 'x')), + (1, (2, 'x')), + (2, (1, 'y')), + (2, (1, 'z')) + )) + } + + test("joinSharedIndex") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1), (4,-4), (4, 4) )).index() + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).index(rdd1.index) + val joined = rdd1.join(rdd2).collect() + assert(joined.toSet === Set( + (1, (1, 'x')), + (1, (2, 'x')), + (2, (1, 'y')), + (2, (1, 'z')), + (4, (-4, 'w')), + (4, (4, 'w')) + )) + } + + + test("join all-to-all") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (1, 3))) + val rdd2 = sc.parallelize(Array((1, 'x'), (1, 'y'))) + val joined = rdd1.join(rdd2).collect() + assert(joined.size === 6) + assert(joined.toSet === Set( + (1, (1, 'x')), + (1, (1, 'y')), + (1, (2, 'x')), + (1, (2, 'y')), + (1, (3, 'x')), + (1, (3, 'y')) + )) + } + + test("leftOuterJoin") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) + val joined = rdd1.leftOuterJoin(rdd2).collect() + assert(joined.size === 5) + assert(joined.toSet === Set( + (1, (1, Some('x'))), + (1, (2, Some('x'))), + (2, (1, Some('y'))), + (2, (1, Some('z'))), + (3, (1, None)) + )) + } + + test("rightOuterJoin") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) + val joined = rdd1.rightOuterJoin(rdd2).collect() + assert(joined.size === 5) + assert(joined.toSet === Set( + (1, (Some(1), 'x')), + (1, (Some(2), 'x')), + (2, (Some(1), 'y')), + (2, (Some(1), 'z')), + (4, (None, 'w')) + )) + } + + test("join with no matches") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) + val rdd2 = sc.parallelize(Array((4, 'x'), (5, 'y'), (5, 'z'), (6, 'w'))) + val joined = rdd1.join(rdd2).collect() + assert(joined.size === 0) + } + + test("join with many output partitions") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) + val joined = rdd1.join(rdd2, 10).collect() + assert(joined.size === 4) + assert(joined.toSet === Set( + (1, (1, 'x')), + (1, (2, 'x')), + (2, (1, 'y')), + (2, (1, 'z')) + )) + } + + test("groupWith") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) + val joined = rdd1.groupWith(rdd2).collect() + assert(joined.size === 4) + assert(joined.toSet === Set( + (1, (ArrayBuffer(1, 2), ArrayBuffer('x'))), + (2, (ArrayBuffer(1), ArrayBuffer('y', 'z'))), + (3, (ArrayBuffer(1), ArrayBuffer())), + (4, (ArrayBuffer(), ArrayBuffer('w'))) + )) + } + + test("zero-partition RDD") { + val emptyDir = Files.createTempDir() + val file = sc.textFile(emptyDir.getAbsolutePath) + assert(file.partitions.size == 0) + assert(file.collect().toList === Nil) + // Test that a shuffle on the file works, because this used to be a bug + assert(file.map(line => (line, 1)).reduceByKey(_ + _).collect().toList === Nil) + } + + test("keys and values") { + val rdd = sc.parallelize(Array((1, "a"), (2, "b"))) + assert(rdd.keys.collect().toList === List(1, 2)) + assert(rdd.values.collect().toList === List("a", "b")) + } + + test("default partitioner uses partition size") { + // specify 2000 partitions + val a = sc.makeRDD(Array(1, 2, 3, 4), 2000) + // do a map, which loses the partitioner + val b = a.map(a => (a, (a * 2).toString)) + // then a group by, and see we didn't revert to 2 partitions + val c = b.groupByKey() + assert(c.partitions.size === 2000) + } + + test("default partitioner uses largest partitioner") { + val a = sc.makeRDD(Array((1, "a"), (2, "b")), 2) + val b = sc.makeRDD(Array((1, "a"), (2, "b")), 2000) + val c = a.join(b) + assert(c.partitions.size === 2000) + } + + test("subtract") { + val a = sc.parallelize(Array(1, 2, 3), 2) + val b = sc.parallelize(Array(2, 3, 4), 4) + val c = a.subtract(b) + assert(c.collect().toSet === Set(1)) + assert(c.partitions.size === a.partitions.size) + } + + test("subtract with narrow dependency") { + // use a deterministic partitioner + val p = new Partitioner() { + def numPartitions = 5 + def getPartition(key: Any) = key.asInstanceOf[Int] + } + // partitionBy so we have a narrow dependency + val a = sc.parallelize(Array((1, "a"), (2, "b"), (3, "c"))).partitionBy(p) + // more partitions/no partitioner so a shuffle dependency + val b = sc.parallelize(Array((2, "b"), (3, "cc"), (4, "d")), 4) + val c = a.subtract(b) + assert(c.collect().toSet === Set((1, "a"), (3, "c"))) + // Ideally we could keep the original partitioner... + assert(c.partitioner === None) + } + + test("subtractByKey") { + val a = sc.parallelize(Array((1, "a"), (1, "a"), (2, "b"), (3, "c")), 2) + val b = sc.parallelize(Array((2, 20), (3, 30), (4, 40)), 4) + val c = a.subtractByKey(b) + assert(c.collect().toSet === Set((1, "a"), (1, "a"))) + assert(c.partitions.size === a.partitions.size) + } + + test("subtractByKey with narrow dependency") { + // use a deterministic partitioner + val p = new Partitioner() { + def numPartitions = 5 + def getPartition(key: Any) = key.asInstanceOf[Int] + } + // partitionBy so we have a narrow dependency + val a = sc.parallelize(Array((1, "a"), (1, "a"), (2, "b"), (3, "c"))).partitionBy(p) + // more partitions/no partitioner so a shuffle dependency + val b = sc.parallelize(Array((2, "b"), (3, "cc"), (4, "d")), 4) + val c = a.subtractByKey(b) + assert(c.collect().toSet === Set((1, "a"), (1, "a"))) + assert(c.partitioner.get === p) + } + + test("foldByKey") { + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) + val sums = pairs.foldByKey(0)(_+_).collect() + assert(sums.toSet === Set((1, 7), (2, 1))) + } + + test("foldByKey with mutable result type") { + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) + val bufs = pairs.mapValues(v => ArrayBuffer(v)).cache() + // Fold the values using in-place mutation + val sums = bufs.foldByKey(new ArrayBuffer[Int])(_ ++= _).collect() + assert(sums.toSet === Set((1, ArrayBuffer(1, 2, 3, 1)), (2, ArrayBuffer(1)))) + // Check that the mutable objects in the original RDD were not changed + assert(bufs.collect().toSet === Set( + (1, ArrayBuffer(1)), + (1, ArrayBuffer(2)), + (1, ArrayBuffer(3)), + (1, ArrayBuffer(1)), + (2, ArrayBuffer(1)))) + } +} From 61281756f225902386e8f41d12a3c58f06da3977 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Thu, 15 Aug 2013 14:20:59 -0700 Subject: [PATCH 058/531] IndexedRDD passes all PairRDD Function tests --- .../src/main/scala/spark/rdd/IndexedRDD.scala | 337 ++++++++++++++++-- .../test/scala/spark/IndexedRDDSuite.scala | 40 ++- 2 files changed, 338 insertions(+), 39 deletions(-) diff --git a/core/src/main/scala/spark/rdd/IndexedRDD.scala b/core/src/main/scala/spark/rdd/IndexedRDD.scala index a6852f3f8a611..b65efa4447b96 100644 --- a/core/src/main/scala/spark/rdd/IndexedRDD.scala +++ b/core/src/main/scala/spark/rdd/IndexedRDD.scala @@ -17,18 +17,21 @@ package spark.rdd +import java.nio.ByteBuffer + + import java.util.{HashMap => JHashMap, BitSet => JBitSet, HashSet => JHashSet} import scala.collection.JavaConversions._ import scala.collection.mutable.ArrayBuffer -import spark.{Utils, OneToOneDependency, RDD, SparkContext, Partition, TaskContext} - -import spark.PairRDDFunctions +import spark._ +import spark.rdd._ import spark.SparkContext._ -import spark.SparkException -import spark.Partitioner +import spark.Partitioner._ + + // import java.io.{ObjectOutputStream, IOException} @@ -110,7 +113,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( val newValues = valuesRDD.mapPartitions( _.map{ groups: Seq[Seq[V]] => groups.map{ group: Seq[V] => - if(group != null && !group.isEmpty) { + if (group != null && !group.isEmpty) { val c: C = createCombiner(group.head) val sum: C = group.tail.foldLeft(c)(mergeValue) Seq(sum) @@ -123,15 +126,129 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( } + /** + * Simplified version of combineByKey that hash-partitions the output RDD. + */ + def combineByKey[C: ClassManifest](createCombiner: V => C, + mergeValue: (C, V) => C, + mergeCombiners: (C, C) => C, + numPartitions: Int): IndexedRDD[K, C] = { + combineByKey(createCombiner, mergeValue, mergeCombiners, new HashPartitioner(numPartitions)) + } + + + /** + * Simplified version of combineByKey that hash-partitions the resulting RDD using the + * existing partitioner/parallelism level. + */ + def combineByKey[C: ClassManifest](createCombiner: V => C, mergeValue: (C, V) => C, + mergeCombiners: (C, C) => C) + : IndexedRDD[K, C] = { + combineByKey(createCombiner, mergeValue, mergeCombiners, defaultPartitioner(index)) + } + + + /** + * Merge the values for each key using an associative function and a neutral "zero value" which may + * be added to the result an arbitrary number of times, and must not change the result (e.g., Nil for + * list concatenation, 0 for addition, or 1 for multiplication.). + */ + def foldByKey(zeroValue: V, partitioner: Partitioner)(func: (V, V) => V): IndexedRDD[K, V] = { + // Serialize the zero value to a byte array so that we can get a new clone of it on each key + val zeroBuffer = SparkEnv.get.closureSerializer.newInstance().serialize(zeroValue) + val zeroArray = new Array[Byte](zeroBuffer.limit) + zeroBuffer.get(zeroArray) + + // When deserializing, use a lazy val to create just one instance of the serializer per task + lazy val cachedSerializer = SparkEnv.get.closureSerializer.newInstance() + def createZero() = cachedSerializer.deserialize[V](ByteBuffer.wrap(zeroArray)) + combineByKey[V]((v: V) => func(createZero(), v), func, func, partitioner) + } + + /** + * Merge the values for each key using an associative function and a neutral "zero value" which may + * be added to the result an arbitrary number of times, and must not change the result (e.g., Nil for + * list concatenation, 0 for addition, or 1 for multiplication.). + */ + def foldByKey(zeroValue: V, numPartitions: Int)(func: (V, V) => V): IndexedRDD[K, V] = { + foldByKey(zeroValue, new HashPartitioner(numPartitions))(func) + } + + + /** + * Merge the values for each key using an associative function and a neutral "zero value" which may + * be added to the result an arbitrary number of times, and must not change the result (e.g., Nil for + * list concatenation, 0 for addition, or 1 for multiplication.). + */ + def foldByKey(zeroValue: V)(func: (V, V) => V): IndexedRDD[K, V] = { + foldByKey(zeroValue, defaultPartitioner(index))(func) + } + + + /** + * Merge the values for each key using an associative reduce function. This will also perform + * the merging locally on each mapper before sending results to a reducer, similarly to a + * "combiner" in MapReduce. + */ + def reduceByKey(partitioner: Partitioner, func: (V, V) => V): IndexedRDD[K, V] = { + combineByKey[V]((v: V) => v, func, func, partitioner) + } + + /** + * Merge the values for each key using an associative reduce function. This will also perform + * the merging locally on each mapper before sending results to a reducer, similarly to a + * "combiner" in MapReduce. Output will be hash-partitioned with numPartitions partitions. + */ + def reduceByKey(func: (V, V) => V, numPartitions: Int): IndexedRDD[K, V] = { + reduceByKey(new HashPartitioner(numPartitions), func) + } + + /** + * Merge the values for each key using an associative reduce function. This will also perform + * the merging locally on each mapper before sending results to a reducer, similarly to a + * "combiner" in MapReduce. Output will be hash-partitioned with the existing partitioner/ + * parallelism level. + */ + def reduceByKey(func: (V, V) => V): IndexedRDD[K, V] = { + reduceByKey(defaultPartitioner(index), func) + } + + + /** + * Group the values for each key in the RDD into a single sequence. Allows controlling the + * partitioning of the resulting key-value pair RDD by passing a Partitioner. + */ + def groupByKey(partitioner: Partitioner): IndexedRDD[K, Seq[V]] = { + val newValues = valuesRDD.mapPartitions(_.map{ar => ar.map{s => Seq(s)} }, true) + new IndexedRDD[K, Seq[V]](index, newValues) + } + + + /** + * Group the values for each key in the RDD into a single sequence. Hash-partitions the + * resulting RDD with into `numPartitions` partitions. + */ + def groupByKey(numPartitions: Int): IndexedRDD[K, Seq[V]] = { + groupByKey(new HashPartitioner(numPartitions)) + } + + + /** + * Group the values for each key in the RDD into a single sequence. Hash-partitions the + * resulting RDD with the existing partitioner/parallelism level. + */ + def groupByKey(): IndexedRDD[K, Seq[V]] = { + groupByKey(defaultPartitioner(index)) + } + /** * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the * list of values for that key in `this` as well as `other`. */ - def cogroup[W: ClassManifest](other: RDD[(K, W)], partitionerUnused: Partitioner): - IndexedRDD[K, (Seq[V], Seq[W])] = { + def cogroup[W: ClassManifest](other: RDD[(K, W)], partitioner: Partitioner): + IndexedRDD[K, (Seq[V], Seq[W])] = { //RDD[(K, (Seq[V], Seq[W]))] = { - assert(false) other match { case other: IndexedRDD[_, _] if other.index == index => { // if both RDDs share exactly the same index and therefore the same super set of keys @@ -174,7 +291,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( newIndex.putAll(otherIndex) // We need to rekey the index var ctr = 0 - for(e <- newIndex.entrySet) { + for (e <- newIndex.entrySet) { e.setValue(ctr) ctr += 1 } @@ -198,9 +315,9 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( // Preallocate the new Values array val newValues = new Array[Seq[(Seq[V],Seq[W])]](newIndex.size) // Lookup the sequences in both submaps - for((k,ind) <- newIndex) { - val thisSeq = if(thisIndex.contains(k)) thisValues(thisIndex.get(k)) else null - val otherSeq = if(otherIndex.contains(k)) otherValues(otherIndex.get(k)) else null + for ((k,ind) <- newIndex) { + val thisSeq = if (thisIndex.contains(k)) thisValues(thisIndex.get(k)) else null + val otherSeq = if (otherIndex.contains(k)) otherValues(otherIndex.get(k)) else null // if either of the sequences is not null then the key was in one of the two tables // and so the value should appear in the returned table newValues(ind) = (thisSeq, otherSeq) match { @@ -233,27 +350,27 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( // Get the corresponding indicies and values for this IndexedRDD val (thisIndex, thisValues) = thisTuplesIter.next() assert(!thisTuplesIter.hasNext()) + assert(thisIndex.size == thisValues.size) // Construct a new index val newIndex = thisIndex.clone().asInstanceOf[JHashMap[K, Int]] + assert(thisIndex.size == newIndex.size) // Construct a new array Buffer to store the values - val newValues = new ArrayBuffer[(Seq[V], ArrayBuffer[W])](thisValues.size) + val newValues = ArrayBuffer.fill[(Seq[V], ArrayBuffer[W])](thisValues.size)(null) // populate the newValues with the values in this IndexedRDD - for((k,i) <- thisIndex) { - if(thisValues(i) != null) { - newValues(i) = (thisValues(i), new ArrayBuffer[W]()) + for ((k,i) <- thisIndex) { + assert(i < thisValues.size) + if (thisValues(i) != null) { + newValues(i) = (thisValues(i), ArrayBuffer.empty[W]) } } // Now iterate through the other tuples updating the map - for((k,w) <- otherTuplesIter){ - if(!newIndex.contains(k)) { + for ((k,w) <- otherTuplesIter){ + if (!newIndex.contains(k)) { // update the index val ind = newIndex.size newIndex.put(k, ind) - // Create the buffer for w - val wbuffer = new ArrayBuffer[W]() - wbuffer.append(w) // Update the values - newValues.append( (Seq.empty[V], wbuffer) ) + newValues.append( (Seq.empty[V], ArrayBuffer(w) ) ) } else { val ind = newIndex.get(k) newValues(ind)._2.append(w) @@ -265,17 +382,167 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( List( (newIndex, newValuesArray) ).iterator }, otherShuffled).cache() - // Extract the index and values from the above RDD - val newIndex = groups.mapPartitions(_.map{ case (kMap,vAr) => kMap }, true) - val newValues = groups.mapPartitions(_.map{ case (kMap,vAr) => vAr }, true) + // Extract the index and values from the above RDD + val newIndex = groups.mapPartitions(_.map{ case (kMap,vAr) => kMap }, true) + val newValues = groups.mapPartitions(_.map{ case (kMap,vAr) => vAr }, true) - new IndexedRDD[K, (Seq[V], Seq[W])](newIndex, newValues) - - } + new IndexedRDD[K, (Seq[V], Seq[W])](newIndex, newValues) } } + } + /** + * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the + * list of values for that key in `this` as well as `other`. + */ + def cogroup[W: ClassManifest](other: RDD[(K, W)]): IndexedRDD[K, (Seq[V], Seq[W])] = { + cogroup(other, defaultPartitioner(this, other)) + } + + // /** + // * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a + // * tuple with the list of values for that key in `this`, `other1` and `other2`. + // */ + // def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)]) + // : IndexedRDD[K, (Seq[V], Seq[W1], Seq[W2])] = { + // cogroup(other1, other2, defaultPartitioner(this, other1, other2)) + // } + + /** + * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the + * list of values for that key in `this` as well as `other`. + */ + def cogroup[W: ClassManifest](other: RDD[(K, W)], numPartitions: Int): IndexedRDD[K, (Seq[V], Seq[W])] = { + cogroup(other, new HashPartitioner(numPartitions)) + } + + // /** + // * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a + // * tuple with the list of values for that key in `this`, `other1` and `other2`. + // */ + // def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)], numPartitions: Int) + // : RDD[(K, (Seq[V], Seq[W1], Seq[W2]))] = { + // cogroup(other1, other2, new HashPartitioner(numPartitions)) + // } + + /** Alias for cogroup. */ + def groupWith[W: ClassManifest](other: RDD[(K, W)]): IndexedRDD[K, (Seq[V], Seq[W])] = { + cogroup(other, defaultPartitioner(this, other)) + } + + // /** Alias for cogroup. */ + // def groupWith[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)]) + // : RDD[(K, (Seq[V], Seq[W1], Seq[W2]))] = { + // cogroup(other1, other2, defaultPartitioner(self, other1, other2)) + // } + + /** + * Return an RDD containing all pairs of elements with matching keys in `this` and `other`. Each + * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and + * (k, v2) is in `other`. Uses the given Partitioner to partition the output RDD. + */ + def join[W: ClassManifest](other: RDD[(K, W)], partitioner: Partitioner): IndexedRDD[K, (V, W)] = { + cogroup(other, partitioner).flatMapValues { + case (vs, ws) => + for (v <- vs.iterator; w <- ws.iterator) yield (v, w) + } + } + + /** + * Perform a left outer join of `this` and `other`. For each element (k, v) in `this`, the + * resulting RDD will either contain all pairs (k, (v, Some(w))) for w in `other`, or the + * pair (k, (v, None)) if no elements in `other` have key k. Uses the given Partitioner to + * partition the output RDD. + */ + def leftOuterJoin[W: ClassManifest](other: RDD[(K, W)], partitioner: Partitioner): IndexedRDD[K, (V, Option[W])] = { + cogroup(other, partitioner).flatMapValues { + case (vs, ws) => + if (ws.isEmpty) { + vs.iterator.map(v => (v, None)) + } else { + for (v <- vs.iterator; w <- ws.iterator) yield (v, Some(w)) + } + } + } + + /** + * Perform a right outer join of `this` and `other`. For each element (k, w) in `other`, the + * resulting RDD will either contain all pairs (k, (Some(v), w)) for v in `this`, or the + * pair (k, (None, w)) if no elements in `this` have key k. Uses the given Partitioner to + * partition the output RDD. + */ + def rightOuterJoin[W: ClassManifest](other: RDD[(K, W)], partitioner: Partitioner) + : IndexedRDD[K, (Option[V], W)] = { + cogroup(other, partitioner).flatMapValues { + case (vs, ws) => + if (vs.isEmpty) { + ws.iterator.map(w => (None, w)) + } else { + for (v <- vs.iterator; w <- ws.iterator) yield (Some(v), w) + } + } + } + + /** + * Return an RDD containing all pairs of elements with matching keys in `this` and `other`. Each + * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and + * (k, v2) is in `other`. Performs a hash join across the cluster. + */ + def join[W: ClassManifest](other: RDD[(K, W)]): IndexedRDD[K, (V, W)] = { + join(other, defaultPartitioner(this, other)) + } + + /** + * Return an RDD containing all pairs of elements with matching keys in `this` and `other`. Each + * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and + * (k, v2) is in `other`. Performs a hash join across the cluster. + */ + def join[W: ClassManifest](other: RDD[(K, W)], numPartitions: Int): IndexedRDD[K, (V, W)] = { + join(other, new HashPartitioner(numPartitions)) + } + + /** + * Perform a left outer join of `this` and `other`. For each element (k, v) in `this`, the + * resulting RDD will either contain all pairs (k, (v, Some(w))) for w in `other`, or the + * pair (k, (v, None)) if no elements in `other` have key k. Hash-partitions the output + * using the existing partitioner/parallelism level. + */ + def leftOuterJoin[W: ClassManifest](other: RDD[(K, W)]): IndexedRDD[K, (V, Option[W])] = { + leftOuterJoin(other, defaultPartitioner(this, other)) + } + + /** + * Perform a left outer join of `this` and `other`. For each element (k, v) in `this`, the + * resulting RDD will either contain all pairs (k, (v, Some(w))) for w in `other`, or the + * pair (k, (v, None)) if no elements in `other` have key k. Hash-partitions the output + * into `numPartitions` partitions. + */ + def leftOuterJoin[W: ClassManifest](other: RDD[(K, W)], numPartitions: Int): IndexedRDD[K, (V, Option[W])] = { + leftOuterJoin(other, new HashPartitioner(numPartitions)) + } + + /** + * Perform a right outer join of `this` and `other`. For each element (k, w) in `other`, the + * resulting RDD will either contain all pairs (k, (Some(v), w)) for v in `this`, or the + * pair (k, (None, w)) if no elements in `this` have key k. Hash-partitions the resulting + * RDD using the existing partitioner/parallelism level. + */ + def rightOuterJoin[W: ClassManifest](other: RDD[(K, W)]): IndexedRDD[K, (Option[V], W)] = { + rightOuterJoin(other, defaultPartitioner(this, other)) + } + + /** + * Perform a right outer join of `this` and `other`. For each element (k, w) in `other`, the + * resulting RDD will either contain all pairs (k, (Some(v), w)) for v in `this`, or the + * pair (k, (None, w)) if no elements in `this` have key k. Hash-partitions the resulting + * RDD into the given number of partitions. + */ + def rightOuterJoin[W: ClassManifest](other: RDD[(K, W)], numPartitions: Int): IndexedRDD[K, (Option[V], W)] = { + rightOuterJoin(other, new HashPartitioner(numPartitions)) + } + + /** @@ -304,7 +571,7 @@ object IndexedRDD { tbl: RDD[(K,V)], existingIndex: RDD[JHashMap[K,Int]] = null ): IndexedRDD[K, V] = { - if(existingIndex == null) { + if (existingIndex == null) { // Shuffle the table (if necessary) val shuffledTbl = if (tbl.partitioner.isEmpty) { @@ -314,11 +581,11 @@ object IndexedRDD { val groups = shuffledTbl.mapPartitions( iter => { val indexMap = new JHashMap[K, Int]() val values = new ArrayBuffer[Seq[V]]() - for((k,v) <- iter){ + for ((k,v) <- iter){ if(!indexMap.contains(k)) { val ind = indexMap.size indexMap.put(k, ind) - values.append(new ArrayBuffer[V]()) + values.append(ArrayBuffer.empty[V]) } val ind = indexMap.get(k) values(ind).asInstanceOf[ArrayBuffer[V]].append(v) @@ -351,11 +618,11 @@ object IndexedRDD { val index: JHashMap[K,Int] = indexIter.next() assert(!indexIter.hasNext()) val values = new Array[Seq[V]](index.size) - for((k,v) <- tblIter) { + for ((k,v) <- tblIter) { assert(index.contains(k)) val ind = index(k) - if(values(ind) == null){ - values(ind) = new ArrayBuffer[V]() + if (values(ind) == null) { + values(ind) = ArrayBuffer.empty[V] } values(ind).asInstanceOf[ArrayBuffer[V]].append(v) } diff --git a/core/src/test/scala/spark/IndexedRDDSuite.scala b/core/src/test/scala/spark/IndexedRDDSuite.scala index aacb6423ee17f..f30e1f57fa112 100644 --- a/core/src/test/scala/spark/IndexedRDDSuite.scala +++ b/core/src/test/scala/spark/IndexedRDDSuite.scala @@ -31,6 +31,8 @@ import scala.collection.mutable.HashSet import spark.rdd.ShuffledRDD import spark.SparkContext._ +import spark._ + class IndexedRDDSuite extends FunSuite with SharedSparkContext { @@ -165,6 +167,7 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1), (4,-4), (4, 4) )).index() val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).index(rdd1.index) val joined = rdd1.join(rdd2).collect() + assert(joined.size === 6) assert(joined.toSet === Set( (1, (1, 'x')), (1, (2, 'x')), @@ -177,8 +180,8 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { test("join all-to-all") { - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (1, 3))) - val rdd2 = sc.parallelize(Array((1, 'x'), (1, 'y'))) + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (1, 3))).index() + val rdd2 = sc.parallelize(Array((1, 'x'), (1, 'y'))).index(rdd1.index) val joined = rdd1.join(rdd2).collect() assert(joined.size === 6) assert(joined.toSet === Set( @@ -191,8 +194,8 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { )) } - test("leftOuterJoin") { - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) + test("leftOuterJoinIndex") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).index() val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) val joined = rdd1.leftOuterJoin(rdd2).collect() assert(joined.size === 5) @@ -205,6 +208,35 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { )) } + test("leftOuterJoinIndextoIndex") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).index() + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).index() + val joined = rdd1.leftOuterJoin(rdd2).collect() + assert(joined.size === 5) + assert(joined.toSet === Set( + (1, (1, Some('x'))), + (1, (2, Some('x'))), + (2, (1, Some('y'))), + (2, (1, Some('z'))), + (3, (1, None)) + )) + } + + test("leftOuterJoinIndextoSharedIndex") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1), (4, -4))).index() + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).index(rdd1.index) + val joined = rdd1.leftOuterJoin(rdd2).collect() + assert(joined.size === 6) + assert(joined.toSet === Set( + (1, (1, Some('x'))), + (1, (2, Some('x'))), + (2, (1, Some('y'))), + (2, (1, Some('z'))), + (4, (-4, Some('w'))), + (3, (1, None)) + )) + } + test("rightOuterJoin") { val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) From 3bb6e019d4eee5cc80b6b506951ce9efc4f21ed2 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Thu, 15 Aug 2013 14:29:48 -0700 Subject: [PATCH 059/531] adding better error handling when indexing an RDD --- core/src/main/scala/spark/rdd/IndexedRDD.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/spark/rdd/IndexedRDD.scala b/core/src/main/scala/spark/rdd/IndexedRDD.scala index b65efa4447b96..7d8e0d3938ec2 100644 --- a/core/src/main/scala/spark/rdd/IndexedRDD.scala +++ b/core/src/main/scala/spark/rdd/IndexedRDD.scala @@ -350,15 +350,12 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( // Get the corresponding indicies and values for this IndexedRDD val (thisIndex, thisValues) = thisTuplesIter.next() assert(!thisTuplesIter.hasNext()) - assert(thisIndex.size == thisValues.size) // Construct a new index val newIndex = thisIndex.clone().asInstanceOf[JHashMap[K, Int]] - assert(thisIndex.size == newIndex.size) // Construct a new array Buffer to store the values val newValues = ArrayBuffer.fill[(Seq[V], ArrayBuffer[W])](thisValues.size)(null) // populate the newValues with the values in this IndexedRDD for ((k,i) <- thisIndex) { - assert(i < thisValues.size) if (thisValues(i) != null) { newValues(i) = (thisValues(i), ArrayBuffer.empty[W]) } @@ -619,7 +616,10 @@ object IndexedRDD { assert(!indexIter.hasNext()) val values = new Array[Seq[V]](index.size) for ((k,v) <- tblIter) { - assert(index.contains(k)) + if (!index.contains(k)) { + throw new SparkException("Error: Try to bind an external index " + + "to an RDD which contains keys that are not in the index.") + } val ind = index(k) if (values(ind) == null) { values(ind) = ArrayBuffer.empty[V] From 327a4db9f7546c58d93e0ccd38a6d131aee98c78 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Thu, 15 Aug 2013 16:36:26 -0700 Subject: [PATCH 060/531] changing caching behavior on indexedrdds --- core/src/main/scala/spark/rdd/IndexedRDD.scala | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/core/src/main/scala/spark/rdd/IndexedRDD.scala b/core/src/main/scala/spark/rdd/IndexedRDD.scala index 7d8e0d3938ec2..6b1cb7608bba4 100644 --- a/core/src/main/scala/spark/rdd/IndexedRDD.scala +++ b/core/src/main/scala/spark/rdd/IndexedRDD.scala @@ -58,6 +58,12 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( override def getPartitions: Array[Partition] = tuples.getPartitions override def getPreferredLocations(s: Partition): Seq[String] = tuples.getPreferredLocations(s) + override def cache: IndexedRDD[K,V] = { + index.cache + valuesRDD.cache + return this + } + /** * Construct a new IndexedRDD that is indexed by only the keys in the RDD From 8fd37adf832a7cd234dca64ea7679668b89fafa8 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Sun, 18 Aug 2013 10:57:35 -0700 Subject: [PATCH 061/531] Merged with changes to zipPartitions --- .../src/main/scala/spark/rdd/IndexedRDD.scala | 25 +++++++++---------- 1 file changed, 12 insertions(+), 13 deletions(-) diff --git a/core/src/main/scala/spark/rdd/IndexedRDD.scala b/core/src/main/scala/spark/rdd/IndexedRDD.scala index 6b1cb7608bba4..37ff11e6396b9 100644 --- a/core/src/main/scala/spark/rdd/IndexedRDD.scala +++ b/core/src/main/scala/spark/rdd/IndexedRDD.scala @@ -262,7 +262,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( // However it is possible that both RDDs are missing a value for a given key in // which case the returned RDD should have a null value val newValues = - valuesRDD.zipPartitions[ Seq[Seq[W]], Seq[Seq[(Seq[V], Seq[W])]] ]( + valuesRDD.zipPartitions(other.valuesRDD)( (thisIter, otherIter) => { val thisValues: Seq[Seq[V]] = thisIter.next() assert(!thisIter.hasNext()) @@ -277,7 +277,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( case (a, b) => Seq((a,b)) }.toSeq List(tmp).iterator - }, other.valuesRDD) + }) new IndexedRDD[K, (Seq[V], Seq[W])](index, newValues) } case other: IndexedRDD[_, _] if other.index.partitioner == index.partitioner => { @@ -285,7 +285,8 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( // then we we need to first merge the indicies and then use the merged index to // merge the values. val newIndex = - index.zipPartitions[JHashMap[K,Int], JHashMap[K,Int]]( (thisIter, otherIter) => { + index.zipPartitions(other.index)( + (thisIter, otherIter) => { val thisIndex = thisIter.next() assert(!thisIter.hasNext()) val otherIndex = otherIter.next() @@ -302,13 +303,10 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( ctr += 1 } List(newIndex).iterator - }, other.index).cache() + }).cache() // Use the new index along with the this and the other indices to merge the values val newValues = - newIndex.zipPartitions[ - (JHashMap[K, Int], Seq[Seq[V]]), - (JHashMap[K, Int], Seq[Seq[W]]), - Seq[Seq[(Seq[V],Seq[W])]] ]( + newIndex.zipPartitions(tuples, other.tuples)( (newIndexIter, thisTuplesIter, otherTuplesIter) => { // Get the new index for this partition val newIndex = newIndexIter.next() @@ -334,7 +332,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( } } List(newValues.toSeq).iterator - }, tuples, other.tuples) + }) new IndexedRDD(newIndex, newValues) } case _ => { @@ -351,7 +349,8 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( new ShuffledRDD[K,W](other, partitioner) } // Join the other RDD with this RDD building a new valueset and new index on the fly - val groups = tuples.zipPartitions[(K, W), (JHashMap[K, Int], Seq[Seq[(Seq[V],Seq[W])]]) ]( + val groups = + tuples.zipPartitions(otherShuffled)( (thisTuplesIter, otherTuplesIter) => { // Get the corresponding indicies and values for this IndexedRDD val (thisIndex, thisValues) = thisTuplesIter.next() @@ -383,7 +382,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( val newValuesArray: Seq[Seq[(Seq[V],Seq[W])]] = newValues.view.map{ case (s, ab) => Seq((s, ab.toSeq)) }.toSeq List( (newIndex, newValuesArray) ).iterator - }, otherShuffled).cache() + }).cache() // Extract the index and values from the above RDD val newIndex = groups.mapPartitions(_.map{ case (kMap,vAr) => kMap }, true) @@ -615,7 +614,7 @@ object IndexedRDD { } // Use the index to build the new values table - val values = index.zipPartitions[ (K, V), Seq[Seq[V]] ]( + val values = index.zipPartitions(shuffledTbl)( (indexIter, tblIter) => { // There is only one map val index: JHashMap[K,Int] = indexIter.next() @@ -633,7 +632,7 @@ object IndexedRDD { values(ind).asInstanceOf[ArrayBuffer[V]].append(v) } List(values.toSeq).iterator - }, shuffledTbl) + }) new IndexedRDD[K,V](index, values) } From 630281bf7671ffec65a4672361d3d9570cfe7a39 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Sun, 18 Aug 2013 17:16:45 -0700 Subject: [PATCH 062/531] Corrected all indexed RDD tests. There appears to be an issue with subtract by key tests that needs to be investigated further. --- .../main/scala/spark/PairRDDFunctions.scala | 9 +- core/src/main/scala/spark/RDD.scala | 9 + .../src/main/scala/spark/rdd/IndexedRDD.scala | 302 ++++++------------ .../test/scala/spark/IndexedRDDSuite.scala | 206 ++++++++---- 4 files changed, 259 insertions(+), 267 deletions(-) diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index 469e870409763..6751da72af701 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -699,7 +699,14 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( def values: RDD[V] = self.map(_._2) - def index(existingIndex: RDD[JHashMap[K,Int]] = null): IndexedRDD[K,V] = + def indexed(numPartitions: Int): IndexedRDD[K,V] = + IndexedRDD(self.partitionBy(new HashPartitioner(numPartitions))) + + def indexed(partitioner: Partitioner): IndexedRDD[K,V] = + IndexedRDD(self.partitionBy(partitioner)) + + + def indexed(existingIndex: RDD[BlockIndex[K]] = null): IndexedRDD[K,V] = IndexedRDD(self, existingIndex) private[spark] def getKeyClass() = implicitly[ClassManifest[K]].erasure diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index 503ea6ccbf47b..1c5095fa865fe 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -54,6 +54,9 @@ import spark.rdd.ZippedRDD import spark.rdd.ZippedPartitionsRDD2 import spark.rdd.ZippedPartitionsRDD3 import spark.rdd.ZippedPartitionsRDD4 + +import spark.rdd.{IndexedRDD, BlockIndex} + import spark.storage.StorageLevel import spark.util.BoundedPriorityQueue @@ -770,6 +773,12 @@ abstract class RDD[T: ClassManifest]( return buf.toArray } + + def makeIndex(partitioner: Option[Partitioner] = None): RDD[BlockIndex[T]] = + IndexedRDD.makeIndex(this, partitioner) + + + /** * Return the first element in this RDD. */ diff --git a/core/src/main/scala/spark/rdd/IndexedRDD.scala b/core/src/main/scala/spark/rdd/IndexedRDD.scala index 37ff11e6396b9..b2a2180b8f148 100644 --- a/core/src/main/scala/spark/rdd/IndexedRDD.scala +++ b/core/src/main/scala/spark/rdd/IndexedRDD.scala @@ -34,7 +34,13 @@ import spark.Partitioner._ -// import java.io.{ObjectOutputStream, IOException} +/** + * And index RDD + */ +class BlockIndex[@specialized K: ClassManifest] extends JHashMap[K,Int] + +//type BlockIndex[@specialized K: ClassManifest] = JHashMap[K,Int] + /** * An IndexedRDD is an RDD[(K,V)] where each K is unique. @@ -43,20 +49,26 @@ import spark.Partitioner._ * be used to accelerate join and aggregation operations. */ class IndexedRDD[K: ClassManifest, V: ClassManifest]( - val index: RDD[ JHashMap[K, Int] ], + val index: RDD[ BlockIndex[K] ], val valuesRDD: RDD[ Seq[Seq[V]] ]) extends RDD[(K, V)](index.context, List(new OneToOneDependency(index), new OneToOneDependency(valuesRDD)) ) { - //with PairRDDFunctions[K,V] { - - - val tuples = new ZippedRDD[JHashMap[K, Int], Seq[Seq[V]]](index.context, index, valuesRDD) + /** + * An internal representation of the maps and block managers + */ + protected val tuples = new ZippedRDD(index.context, index, valuesRDD) override val partitioner = index.partitioner + + override def getPartitions: Array[Partition] = tuples.getPartitions - override def getPreferredLocations(s: Partition): Seq[String] = tuples.getPreferredLocations(s) + + + override def getPreferredLocations(s: Partition): Seq[String] = + tuples.getPreferredLocations(s) + override def cache: IndexedRDD[K,V] = { index.cache @@ -77,9 +89,10 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( */ def mapValues[U: ClassManifest](f: V => U): IndexedRDD[K, U] = { val cleanF = index.context.clean(f) - val newValues = valuesRDD.mapPartitions(_.map{ values => - values.map{_.map(x => f(x))} - }, true) + val newValues = valuesRDD.mapPartitions(_.map(values => values.map{ + case null => null + case row => row.map(x => f(x)) + }), true) new IndexedRDD[K,U](index, newValues) } @@ -90,9 +103,10 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( */ def flatMapValues[U: ClassManifest](f: V => TraversableOnce[U]): IndexedRDD[K,U] = { val cleanF = index.context.clean(f) - val newValues = valuesRDD.mapPartitions(_.map{ values => - values.map{_.flatMap(x => f(x))} - }, true) + val newValues = valuesRDD.mapPartitions(_.map(values => values.map{ + case null => null + case row => row.flatMap(x => f(x)) + }), true) new IndexedRDD[K,U](index, newValues) } @@ -106,15 +120,10 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( * - `createCombiner`, which turns a V into a C (e.g., creates a one-element list) * - `mergeValue`, to merge a V into a C (e.g., adds it to the end of a list) * - `mergeCombiners`, to combine two C's into a single one. - * - * In addition, users can control the partitioning of the output RDD, and whether to perform - * map-side aggregation (if a mapper can produce multiple items with the same key). */ def combineByKey[C: ClassManifest](createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => C, - partitioner: Partitioner, - mapSideCombine: Boolean = true, serializerClass: String = null): IndexedRDD[K, C] = { val newValues = valuesRDD.mapPartitions( _.map{ groups: Seq[Seq[V]] => @@ -131,35 +140,12 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( new IndexedRDD[K,C](index, newValues) } - - /** - * Simplified version of combineByKey that hash-partitions the output RDD. - */ - def combineByKey[C: ClassManifest](createCombiner: V => C, - mergeValue: (C, V) => C, - mergeCombiners: (C, C) => C, - numPartitions: Int): IndexedRDD[K, C] = { - combineByKey(createCombiner, mergeValue, mergeCombiners, new HashPartitioner(numPartitions)) - } - - - /** - * Simplified version of combineByKey that hash-partitions the resulting RDD using the - * existing partitioner/parallelism level. - */ - def combineByKey[C: ClassManifest](createCombiner: V => C, mergeValue: (C, V) => C, - mergeCombiners: (C, C) => C) - : IndexedRDD[K, C] = { - combineByKey(createCombiner, mergeValue, mergeCombiners, defaultPartitioner(index)) - } - - /** * Merge the values for each key using an associative function and a neutral "zero value" which may * be added to the result an arbitrary number of times, and must not change the result (e.g., Nil for * list concatenation, 0 for addition, or 1 for multiplication.). */ - def foldByKey(zeroValue: V, partitioner: Partitioner)(func: (V, V) => V): IndexedRDD[K, V] = { + def foldByKey(zeroValue: V)(func: (V, V) => V): IndexedRDD[K, V] = { // Serialize the zero value to a byte array so that we can get a new clone of it on each key val zeroBuffer = SparkEnv.get.closureSerializer.newInstance().serialize(zeroValue) val zeroArray = new Array[Byte](zeroBuffer.limit) @@ -168,45 +154,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( // When deserializing, use a lazy val to create just one instance of the serializer per task lazy val cachedSerializer = SparkEnv.get.closureSerializer.newInstance() def createZero() = cachedSerializer.deserialize[V](ByteBuffer.wrap(zeroArray)) - combineByKey[V]((v: V) => func(createZero(), v), func, func, partitioner) - } - - /** - * Merge the values for each key using an associative function and a neutral "zero value" which may - * be added to the result an arbitrary number of times, and must not change the result (e.g., Nil for - * list concatenation, 0 for addition, or 1 for multiplication.). - */ - def foldByKey(zeroValue: V, numPartitions: Int)(func: (V, V) => V): IndexedRDD[K, V] = { - foldByKey(zeroValue, new HashPartitioner(numPartitions))(func) - } - - - /** - * Merge the values for each key using an associative function and a neutral "zero value" which may - * be added to the result an arbitrary number of times, and must not change the result (e.g., Nil for - * list concatenation, 0 for addition, or 1 for multiplication.). - */ - def foldByKey(zeroValue: V)(func: (V, V) => V): IndexedRDD[K, V] = { - foldByKey(zeroValue, defaultPartitioner(index))(func) - } - - - /** - * Merge the values for each key using an associative reduce function. This will also perform - * the merging locally on each mapper before sending results to a reducer, similarly to a - * "combiner" in MapReduce. - */ - def reduceByKey(partitioner: Partitioner, func: (V, V) => V): IndexedRDD[K, V] = { - combineByKey[V]((v: V) => v, func, func, partitioner) - } - - /** - * Merge the values for each key using an associative reduce function. This will also perform - * the merging locally on each mapper before sending results to a reducer, similarly to a - * "combiner" in MapReduce. Output will be hash-partitioned with numPartitions partitions. - */ - def reduceByKey(func: (V, V) => V, numPartitions: Int): IndexedRDD[K, V] = { - reduceByKey(new HashPartitioner(numPartitions), func) + combineByKey[V]((v: V) => func(createZero(), v), func, func) } /** @@ -216,26 +164,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( * parallelism level. */ def reduceByKey(func: (V, V) => V): IndexedRDD[K, V] = { - reduceByKey(defaultPartitioner(index), func) - } - - - /** - * Group the values for each key in the RDD into a single sequence. Allows controlling the - * partitioning of the resulting key-value pair RDD by passing a Partitioner. - */ - def groupByKey(partitioner: Partitioner): IndexedRDD[K, Seq[V]] = { - val newValues = valuesRDD.mapPartitions(_.map{ar => ar.map{s => Seq(s)} }, true) - new IndexedRDD[K, Seq[V]](index, newValues) - } - - - /** - * Group the values for each key in the RDD into a single sequence. Hash-partitions the - * resulting RDD with into `numPartitions` partitions. - */ - def groupByKey(numPartitions: Int): IndexedRDD[K, Seq[V]] = { - groupByKey(new HashPartitioner(numPartitions)) + combineByKey[V]((v: V) => v, func, func) } @@ -244,7 +173,8 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( * resulting RDD with the existing partitioner/parallelism level. */ def groupByKey(): IndexedRDD[K, Seq[V]] = { - groupByKey(defaultPartitioner(index)) + val newValues = valuesRDD.mapPartitions(_.map{ar => ar.map{s => Seq(s)} }, true) + new IndexedRDD[K, Seq[V]](index, newValues) } @@ -252,8 +182,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the * list of values for that key in `this` as well as `other`. */ - def cogroup[W: ClassManifest](other: RDD[(K, W)], partitioner: Partitioner): - IndexedRDD[K, (Seq[V], Seq[W])] = { + def cogroup[W: ClassManifest](other: RDD[(K, W)]): IndexedRDD[K, (Seq[V], Seq[W])] = { //RDD[(K, (Seq[V], Seq[W]))] = { other match { case other: IndexedRDD[_, _] if other.index == index => { @@ -291,7 +220,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( assert(!thisIter.hasNext()) val otherIndex = otherIter.next() assert(!otherIter.hasNext()) - val newIndex = new JHashMap[K, Int]() + val newIndex = new BlockIndex[K]() // @todo Merge only the keys that correspond to non-null values // Merge the keys newIndex.putAll(thisIndex) @@ -356,9 +285,9 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( val (thisIndex, thisValues) = thisTuplesIter.next() assert(!thisTuplesIter.hasNext()) // Construct a new index - val newIndex = thisIndex.clone().asInstanceOf[JHashMap[K, Int]] + val newIndex = thisIndex.clone().asInstanceOf[BlockIndex[K]] // Construct a new array Buffer to store the values - val newValues = ArrayBuffer.fill[(Seq[V], ArrayBuffer[W])](thisValues.size)(null) + val newValues = ArrayBuffer.fill[(Seq[V], Seq[W])](thisValues.size)(null) // populate the newValues with the values in this IndexedRDD for ((k,i) <- thisIndex) { if (thisValues(i) != null) { @@ -375,12 +304,21 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( newValues.append( (Seq.empty[V], ArrayBuffer(w) ) ) } else { val ind = newIndex.get(k) - newValues(ind)._2.append(w) + if(newValues(ind) == null) { + // If the other key was in the index but not in the values + // of this indexed RDD then create a new values entry for it + newValues(ind) = (Seq.empty[V], ArrayBuffer(w)) + } else { + newValues(ind)._2.asInstanceOf[ArrayBuffer[W]].append(w) + } } } // Finalize the new values array val newValuesArray: Seq[Seq[(Seq[V],Seq[W])]] = - newValues.view.map{ case (s, ab) => Seq((s, ab.toSeq)) }.toSeq + newValues.view.map{ + case null => null + case (s, ab) => Seq((s, ab.toSeq)) + }.toSeq List( (newIndex, newValuesArray) ).iterator }).cache() @@ -394,13 +332,6 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( } - /** - * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the - * list of values for that key in `this` as well as `other`. - */ - def cogroup[W: ClassManifest](other: RDD[(K, W)]): IndexedRDD[K, (Seq[V], Seq[W])] = { - cogroup(other, defaultPartitioner(this, other)) - } // /** // * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a @@ -411,14 +342,6 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( // cogroup(other1, other2, defaultPartitioner(this, other1, other2)) // } - /** - * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the - * list of values for that key in `this` as well as `other`. - */ - def cogroup[W: ClassManifest](other: RDD[(K, W)], numPartitions: Int): IndexedRDD[K, (Seq[V], Seq[W])] = { - cogroup(other, new HashPartitioner(numPartitions)) - } - // /** // * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a // * tuple with the list of values for that key in `this`, `other1` and `other2`. @@ -430,7 +353,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( /** Alias for cogroup. */ def groupWith[W: ClassManifest](other: RDD[(K, W)]): IndexedRDD[K, (Seq[V], Seq[W])] = { - cogroup(other, defaultPartitioner(this, other)) + cogroup(other) } // /** Alias for cogroup. */ @@ -439,26 +362,28 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( // cogroup(other1, other2, defaultPartitioner(self, other1, other2)) // } + /** * Return an RDD containing all pairs of elements with matching keys in `this` and `other`. Each * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and - * (k, v2) is in `other`. Uses the given Partitioner to partition the output RDD. + * (k, v2) is in `other`. Performs a hash join across the cluster. */ - def join[W: ClassManifest](other: RDD[(K, W)], partitioner: Partitioner): IndexedRDD[K, (V, W)] = { - cogroup(other, partitioner).flatMapValues { + def join[W: ClassManifest](other: RDD[(K, W)]): IndexedRDD[K, (V, W)] = { + cogroup(other).flatMapValues { case (vs, ws) => for (v <- vs.iterator; w <- ws.iterator) yield (v, w) } } + /** * Perform a left outer join of `this` and `other`. For each element (k, v) in `this`, the * resulting RDD will either contain all pairs (k, (v, Some(w))) for w in `other`, or the - * pair (k, (v, None)) if no elements in `other` have key k. Uses the given Partitioner to - * partition the output RDD. + * pair (k, (v, None)) if no elements in `other` have key k. Hash-partitions the output + * using the existing partitioner/parallelism level. */ - def leftOuterJoin[W: ClassManifest](other: RDD[(K, W)], partitioner: Partitioner): IndexedRDD[K, (V, Option[W])] = { - cogroup(other, partitioner).flatMapValues { + def leftOuterJoin[W: ClassManifest](other: RDD[(K, W)]): IndexedRDD[K, (V, Option[W])] = { + cogroup(other).flatMapValues { case (vs, ws) => if (ws.isEmpty) { vs.iterator.map(v => (v, None)) @@ -466,17 +391,18 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( for (v <- vs.iterator; w <- ws.iterator) yield (v, Some(w)) } } + } + /** * Perform a right outer join of `this` and `other`. For each element (k, w) in `other`, the * resulting RDD will either contain all pairs (k, (Some(v), w)) for v in `this`, or the - * pair (k, (None, w)) if no elements in `this` have key k. Uses the given Partitioner to - * partition the output RDD. + * pair (k, (None, w)) if no elements in `this` have key k. Hash-partitions the resulting + * RDD using the existing partitioner/parallelism level. */ - def rightOuterJoin[W: ClassManifest](other: RDD[(K, W)], partitioner: Partitioner) - : IndexedRDD[K, (Option[V], W)] = { - cogroup(other, partitioner).flatMapValues { + def rightOuterJoin[W: ClassManifest](other: RDD[(K, W)]): IndexedRDD[K, (Option[V], W)] = { + cogroup(other).flatMapValues { case (vs, ws) => if (vs.isEmpty) { ws.iterator.map(w => (None, w)) @@ -484,69 +410,10 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( for (v <- vs.iterator; w <- ws.iterator) yield (Some(v), w) } } - } - - /** - * Return an RDD containing all pairs of elements with matching keys in `this` and `other`. Each - * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and - * (k, v2) is in `other`. Performs a hash join across the cluster. - */ - def join[W: ClassManifest](other: RDD[(K, W)]): IndexedRDD[K, (V, W)] = { - join(other, defaultPartitioner(this, other)) - } - - /** - * Return an RDD containing all pairs of elements with matching keys in `this` and `other`. Each - * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and - * (k, v2) is in `other`. Performs a hash join across the cluster. - */ - def join[W: ClassManifest](other: RDD[(K, W)], numPartitions: Int): IndexedRDD[K, (V, W)] = { - join(other, new HashPartitioner(numPartitions)) - } - - /** - * Perform a left outer join of `this` and `other`. For each element (k, v) in `this`, the - * resulting RDD will either contain all pairs (k, (v, Some(w))) for w in `other`, or the - * pair (k, (v, None)) if no elements in `other` have key k. Hash-partitions the output - * using the existing partitioner/parallelism level. - */ - def leftOuterJoin[W: ClassManifest](other: RDD[(K, W)]): IndexedRDD[K, (V, Option[W])] = { - leftOuterJoin(other, defaultPartitioner(this, other)) - } - - /** - * Perform a left outer join of `this` and `other`. For each element (k, v) in `this`, the - * resulting RDD will either contain all pairs (k, (v, Some(w))) for w in `other`, or the - * pair (k, (v, None)) if no elements in `other` have key k. Hash-partitions the output - * into `numPartitions` partitions. - */ - def leftOuterJoin[W: ClassManifest](other: RDD[(K, W)], numPartitions: Int): IndexedRDD[K, (V, Option[W])] = { - leftOuterJoin(other, new HashPartitioner(numPartitions)) - } - - /** - * Perform a right outer join of `this` and `other`. For each element (k, w) in `other`, the - * resulting RDD will either contain all pairs (k, (Some(v), w)) for v in `this`, or the - * pair (k, (None, w)) if no elements in `this` have key k. Hash-partitions the resulting - * RDD using the existing partitioner/parallelism level. - */ - def rightOuterJoin[W: ClassManifest](other: RDD[(K, W)]): IndexedRDD[K, (Option[V], W)] = { - rightOuterJoin(other, defaultPartitioner(this, other)) - } - /** - * Perform a right outer join of `this` and `other`. For each element (k, w) in `other`, the - * resulting RDD will either contain all pairs (k, (Some(v), w)) for v in `this`, or the - * pair (k, (None, w)) if no elements in `this` have key k. Hash-partitions the resulting - * RDD into the given number of partitions. - */ - def rightOuterJoin[W: ClassManifest](other: RDD[(K, W)], numPartitions: Int): IndexedRDD[K, (Option[V], W)] = { - rightOuterJoin(other, new HashPartitioner(numPartitions)) } - - /** * Provide the RDD[(K,V)] equivalent output. */ @@ -571,7 +438,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( object IndexedRDD { def apply[K: ClassManifest, V: ClassManifest]( tbl: RDD[(K,V)], - existingIndex: RDD[JHashMap[K,Int]] = null ): IndexedRDD[K, V] = { + existingIndex: RDD[BlockIndex[K]] = null ): IndexedRDD[K, V] = { if (existingIndex == null) { // Shuffle the table (if necessary) @@ -581,7 +448,7 @@ object IndexedRDD { } else { tbl } val groups = shuffledTbl.mapPartitions( iter => { - val indexMap = new JHashMap[K, Int]() + val indexMap = new BlockIndex[K]() val values = new ArrayBuffer[Seq[V]]() for ((k,v) <- iter){ if(!indexMap.contains(k)) { @@ -617,12 +484,12 @@ object IndexedRDD { val values = index.zipPartitions(shuffledTbl)( (indexIter, tblIter) => { // There is only one map - val index: JHashMap[K,Int] = indexIter.next() + val index = indexIter.next() assert(!indexIter.hasNext()) val values = new Array[Seq[V]](index.size) for ((k,v) <- tblIter) { if (!index.contains(k)) { - throw new SparkException("Error: Try to bind an external index " + + throw new SparkException("Error: Trying to bind an external index " + "to an RDD which contains keys that are not in the index.") } val ind = index(k) @@ -638,6 +505,41 @@ object IndexedRDD { } } + /** + * Construct and index of the unique values in a given RDD. + */ + def makeIndex[K: ClassManifest](keys: RDD[K], + partitioner: Option[Partitioner] = None): RDD[BlockIndex[K]] = { + + + // Ugly hack :-(. In order to partition the keys they must have values. + val tbl = keys.mapPartitions(_.map(k => (k, false)), true) + // Shuffle the table (if necessary) + val shuffledTbl = partitioner match { + case None => { + if (tbl.partitioner.isEmpty) { + new ShuffledRDD[K, Boolean](tbl, Partitioner.defaultPartitioner(tbl)) + } else { tbl } + } + case Some(partitioner) => + tbl.partitionBy(partitioner) +// new ShuffledRDD[K, Boolean](tbl, partitioner) + } + + + val index = shuffledTbl.mapPartitions( iter => { + val indexMap = new BlockIndex[K]() + for ( (k,_) <- iter ){ + if(!indexMap.contains(k)){ + val ind = indexMap.size + indexMap.put(k, ind) + } + } + List(indexMap).iterator + }, true).cache + index + } + } diff --git a/core/src/test/scala/spark/IndexedRDDSuite.scala b/core/src/test/scala/spark/IndexedRDDSuite.scala index f30e1f57fa112..c1433c50950ca 100644 --- a/core/src/test/scala/spark/IndexedRDDSuite.scala +++ b/core/src/test/scala/spark/IndexedRDDSuite.scala @@ -30,10 +30,13 @@ import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashSet import spark.rdd.ShuffledRDD +import spark.rdd.IndexedRDD + import spark.SparkContext._ import spark._ + class IndexedRDDSuite extends FunSuite with SharedSparkContext { def lineage(rdd: RDD[_]): collection.mutable.HashSet[RDD[_]] = { @@ -49,7 +52,7 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { } test("groupByKey") { - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1))).index() + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1))).indexed() val groups = pairs.groupByKey().collect() assert(groups.size === 2) val valuesFor1 = groups.find(_._1 == 1).get._2 @@ -59,7 +62,7 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { } test("groupByKey with duplicates") { - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))).index() + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))).indexed() val groups = pairs.groupByKey().collect() assert(groups.size === 2) val valuesFor1 = groups.find(_._1 == 1).get._2 @@ -69,7 +72,7 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { } test("groupByKey with negative key hash codes") { - val pairs = sc.parallelize(Array((-1, 1), (-1, 2), (-1, 3), (2, 1))).index() + val pairs = sc.parallelize(Array((-1, 1), (-1, 2), (-1, 3), (2, 1))).indexed() val groups = pairs.groupByKey().collect() assert(groups.size === 2) val valuesForMinus1 = groups.find(_._1 == -1).get._2 @@ -79,8 +82,8 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { } test("groupByKey with many output partitions") { - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1))).index() - val groups = pairs.groupByKey(10).collect() + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1))).indexed(10) + val groups = pairs.groupByKey().collect() assert(groups.size === 2) val valuesFor1 = groups.find(_._1 == 1).get._2 assert(valuesFor1.toList.sorted === List(1, 2, 3)) @@ -89,13 +92,13 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { } test("reduceByKey") { - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))).index() + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))).indexed() val sums = pairs.reduceByKey(_+_).collect() assert(sums.toSet === Set((1, 7), (2, 1))) } test("reduceByKey with collectAsMap") { - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))).index() + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))).indexed() val sums = pairs.reduceByKey(_+_).collectAsMap() assert(sums.size === 2) assert(sums(1) === 7) @@ -103,8 +106,8 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { } test("reduceByKey with many output partitons") { - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))).index() - val sums = pairs.reduceByKey(_+_, 10).collect() + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))).indexed(10) + val sums = pairs.reduceByKey(_+_).collect() assert(sums.toSet === Set((1, 7), (2, 1))) } @@ -113,7 +116,7 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { def numPartitions = 2 def getPartition(key: Any) = key.asInstanceOf[Int] } - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 1), (0, 1))).partitionBy(p).index() + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 1), (0, 1))).indexed(p) val sums = pairs.reduceByKey(_+_) assert(sums.collect().toSet === Set((1, 4), (0, 1))) assert(sums.partitioner === Some(p)) @@ -123,22 +126,10 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { assert(deps.filter(_.isInstanceOf[ShuffledRDD[_,_]]).size === 1) // ShuffledRDD, ParallelCollection } - test("join") { - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) - val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) - val joined = rdd1.join(rdd2).collect() - assert(joined.size === 4) - assert(joined.toSet === Set( - (1, (1, 'x')), - (1, (2, 'x')), - (2, (1, 'y')), - (2, (1, 'z')) - )) - } test("joinIndexVsPair") { - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).index() + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed() val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) val joined = rdd1.join(rdd2).collect() assert(joined.size === 4) @@ -151,8 +142,8 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { } test("joinIndexVsIndex") { - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).index() - val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).index() + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed() + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).indexed() val joined = rdd1.join(rdd2).collect() assert(joined.size === 4) assert(joined.toSet === Set( @@ -164,8 +155,8 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { } test("joinSharedIndex") { - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1), (4,-4), (4, 4) )).index() - val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).index(rdd1.index) + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1), (4,-4), (4, 4) )).indexed() + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).indexed(rdd1.index) val joined = rdd1.join(rdd2).collect() assert(joined.size === 6) assert(joined.toSet === Set( @@ -180,8 +171,8 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { test("join all-to-all") { - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (1, 3))).index() - val rdd2 = sc.parallelize(Array((1, 'x'), (1, 'y'))).index(rdd1.index) + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (1, 3))).indexed() + val rdd2 = sc.parallelize(Array((1, 'x'), (1, 'y'))).indexed(rdd1.index) val joined = rdd1.join(rdd2).collect() assert(joined.size === 6) assert(joined.toSet === Set( @@ -195,7 +186,8 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { } test("leftOuterJoinIndex") { - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).index() + val index = sc.parallelize( 1 to 6 ).makeIndex() + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed(index) val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) val joined = rdd1.leftOuterJoin(rdd2).collect() assert(joined.size === 5) @@ -209,8 +201,8 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { } test("leftOuterJoinIndextoIndex") { - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).index() - val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).index() + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed() + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).indexed() val joined = rdd1.leftOuterJoin(rdd2).collect() assert(joined.size === 5) assert(joined.toSet === Set( @@ -223,8 +215,8 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { } test("leftOuterJoinIndextoSharedIndex") { - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1), (4, -4))).index() - val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).index(rdd1.index) + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1), (4, -4))).indexed() + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).indexed(rdd1.index) val joined = rdd1.leftOuterJoin(rdd2).collect() assert(joined.size === 6) assert(joined.toSet === Set( @@ -237,8 +229,25 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { )) } +test("leftOuterJoinIndextoIndexExternal") { + val index = sc.parallelize( 1 to 6 ).makeIndex() + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed(index) + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).indexed(index) + val joined = rdd1.leftOuterJoin(rdd2).collect() + assert(joined.size === 5) + assert(joined.toSet === Set( + (1, (1, Some('x'))), + (1, (2, Some('x'))), + (2, (1, Some('y'))), + (2, (1, Some('z'))), + (3, (1, None)) + )) + } + + test("rightOuterJoin") { - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) + val index = sc.parallelize( 1 to 6 ).makeIndex() + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed(index) val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) val joined = rdd1.rightOuterJoin(rdd2).collect() assert(joined.size === 5) @@ -251,17 +260,58 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { )) } - test("join with no matches") { - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) + test("rightOuterJoinIndex2Index") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed() + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).indexed() + val joined = rdd1.rightOuterJoin(rdd2).collect() + assert(joined.size === 5) + assert(joined.toSet === Set( + (1, (Some(1), 'x')), + (1, (Some(2), 'x')), + (2, (Some(1), 'y')), + (2, (Some(1), 'z')), + (4, (None, 'w')) + )) + } + + + test("rightOuterJoinIndex2Indexshared") { + val index = sc.parallelize( 1 to 6 ).makeIndex() + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed(index) + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).indexed(index) + val joined = rdd1.rightOuterJoin(rdd2).collect() + assert(joined.size === 5) + assert(joined.toSet === Set( + (1, (Some(1), 'x')), + (1, (Some(2), 'x')), + (2, (Some(1), 'y')), + (2, (Some(1), 'z')), + (4, (None, 'w')) + )) + } + + + test("join with no matches index") { + val index = IndexedRDD.makeIndex( sc.parallelize( 1 to 6 ) ) + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed(index) val rdd2 = sc.parallelize(Array((4, 'x'), (5, 'y'), (5, 'z'), (6, 'w'))) val joined = rdd1.join(rdd2).collect() assert(joined.size === 0) } + test("join with no matches shared index") { + val index = IndexedRDD.makeIndex( sc.parallelize( 1 to 6 ) ) + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed(index) + val rdd2 = sc.parallelize(Array((4, 'x'), (5, 'y'), (5, 'z'), (6, 'w'))).indexed(index) + val joined = rdd1.join(rdd2).collect() + assert(joined.size === 0) + } + + test("join with many output partitions") { - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed(10) val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) - val joined = rdd1.join(rdd2, 10).collect() + val joined = rdd1.join(rdd2).collect() assert(joined.size === 4) assert(joined.toSet === Set( (1, (1, 'x')), @@ -271,9 +321,25 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { )) } + test("join with many output partitions and two indices") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed(10) + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).indexed(20) + val joined = rdd1.join(rdd2).collect() + assert(joined.size === 4) + assert(joined.toSet === Set( + (1, (1, 'x')), + (1, (2, 'x')), + (2, (1, 'y')), + (2, (1, 'z')) + )) + } + + test("groupWith") { - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) - val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) + val index = IndexedRDD.makeIndex( sc.parallelize( 1 to 6 ) ) + + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed(index) + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).indexed(index) val joined = rdd1.groupWith(rdd2).collect() assert(joined.size === 4) assert(joined.toSet === Set( @@ -294,7 +360,7 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { } test("keys and values") { - val rdd = sc.parallelize(Array((1, "a"), (2, "b"))) + val rdd = sc.parallelize(Array((1, "a"), (2, "b"))).indexed() assert(rdd.keys.collect().toList === List(1, 2)) assert(rdd.values.collect().toList === List("a", "b")) } @@ -309,12 +375,14 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { assert(c.partitions.size === 2000) } - test("default partitioner uses largest partitioner") { - val a = sc.makeRDD(Array((1, "a"), (2, "b")), 2) - val b = sc.makeRDD(Array((1, "a"), (2, "b")), 2000) - val c = a.join(b) - assert(c.partitions.size === 2000) - } + // test("default partitioner uses largest partitioner indexed to indexed") { + // val a = sc.makeRDD(Array((1, "a"), (2, "b")), 2).indexed() + // val b = sc.makeRDD(Array((1, "a"), (2, "b")), 2000).indexed() + // val c = a.join(b) + // assert(c.partitions.size === 2000) + // } + + test("subtract") { val a = sc.parallelize(Array(1, 2, 3), 2) @@ -331,7 +399,7 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { def getPartition(key: Any) = key.asInstanceOf[Int] } // partitionBy so we have a narrow dependency - val a = sc.parallelize(Array((1, "a"), (2, "b"), (3, "c"))).partitionBy(p) + val a = sc.parallelize(Array((1, "a"), (2, "b"), (3, "c"))).indexed(p) // more partitions/no partitioner so a shuffle dependency val b = sc.parallelize(Array((2, "b"), (3, "cc"), (4, "d")), 4) val c = a.subtract(b) @@ -341,36 +409,42 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { } test("subtractByKey") { - val a = sc.parallelize(Array((1, "a"), (1, "a"), (2, "b"), (3, "c")), 2) + + val a = sc.parallelize(Array((1, "a"), (1, "a"), (2, "b"), (3, "c")), 2).indexed() val b = sc.parallelize(Array((2, 20), (3, 30), (4, 40)), 4) val c = a.subtractByKey(b) assert(c.collect().toSet === Set((1, "a"), (1, "a"))) assert(c.partitions.size === a.partitions.size) } - test("subtractByKey with narrow dependency") { - // use a deterministic partitioner - val p = new Partitioner() { - def numPartitions = 5 - def getPartition(key: Any) = key.asInstanceOf[Int] - } - // partitionBy so we have a narrow dependency - val a = sc.parallelize(Array((1, "a"), (1, "a"), (2, "b"), (3, "c"))).partitionBy(p) - // more partitions/no partitioner so a shuffle dependency - val b = sc.parallelize(Array((2, "b"), (3, "cc"), (4, "d")), 4) - val c = a.subtractByKey(b) - assert(c.collect().toSet === Set((1, "a"), (1, "a"))) - assert(c.partitioner.get === p) - } + // test("subtractByKey with narrow dependency") { + // // use a deterministic partitioner + // val p = new Partitioner() { + // def numPartitions = 5 + // def getPartition(key: Any) = key.asInstanceOf[Int] + // } + + // val index = sc.parallelize( 1 to 6 ).makeIndex(Some(p)) + // // partitionBy so we have a narrow dependency + // val a = sc.parallelize(Array((1, "a"), (1, "a"), (2, "b"), (3, "c"))).indexed(index) + // // more partitions/no partitioner so a shuffle dependency + // val b = sc.parallelize(Array((2, "b"), (3, "cc"), (4, "d")), 4).indexed(index) + // val c = a.subtractByKey(b) + // assert(c.collect().toSet === Set((1, "a"), (1, "a"))) + // assert(c.partitioner.get === p) + // } test("foldByKey") { - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) + val index = IndexedRDD.makeIndex( sc.parallelize( 1 to 6 ) ) + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))).indexed(index) val sums = pairs.foldByKey(0)(_+_).collect() assert(sums.toSet === Set((1, 7), (2, 1))) } test("foldByKey with mutable result type") { - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) + val index = IndexedRDD.makeIndex( sc.parallelize( 1 to 6 ) ) + + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))).indexed(index) val bufs = pairs.mapValues(v => ArrayBuffer(v)).cache() // Fold the values using in-place mutation val sums = bufs.foldByKey(new ArrayBuffer[Int])(_ ++= _).collect() From 023702c90da7c421e1e339e14878bccebbb26e24 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Mon, 19 Aug 2013 13:32:54 -0700 Subject: [PATCH 063/531] Updating documentation. --- core/src/main/scala/spark/rdd/IndexedRDD.scala | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/spark/rdd/IndexedRDD.scala b/core/src/main/scala/spark/rdd/IndexedRDD.scala index b2a2180b8f148..d19d220ec24d3 100644 --- a/core/src/main/scala/spark/rdd/IndexedRDD.scala +++ b/core/src/main/scala/spark/rdd/IndexedRDD.scala @@ -35,18 +35,20 @@ import spark.Partitioner._ /** - * And index RDD + * The BlockIndex is the internal map structure used inside the index + * of the IndexedRDD. */ class BlockIndex[@specialized K: ClassManifest] extends JHashMap[K,Int] -//type BlockIndex[@specialized K: ClassManifest] = JHashMap[K,Int] /** - * An IndexedRDD is an RDD[(K,V)] where each K is unique. + * An IndexedRDD[K,V] extends the RDD[(K,V)] by pre-indexing the keys and + * organizing the values to enable faster join operations. + * + * In addition to providing the basic RDD[(K,V)] functionality the IndexedRDD + * exposes an index member which can be used to "key" other IndexedRDDs * - * The IndexedRDD contains an index datastructure that can - * be used to accelerate join and aggregation operations. */ class IndexedRDD[K: ClassManifest, V: ClassManifest]( val index: RDD[ BlockIndex[K] ], From 55c6e73bfbd29358226d7fb6b7f753d1b05cd9bc Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Mon, 19 Aug 2013 17:35:51 -0700 Subject: [PATCH 064/531] Wrapping the index in an opaque type. --- .../main/scala/spark/PairRDDFunctions.scala | 2 +- core/src/main/scala/spark/RDD.scala | 4 +- .../src/main/scala/spark/rdd/IndexedRDD.scala | 75 +++++++++++++------ 3 files changed, 55 insertions(+), 26 deletions(-) diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index 6751da72af701..03f25040a114a 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -706,7 +706,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( IndexedRDD(self.partitionBy(partitioner)) - def indexed(existingIndex: RDD[BlockIndex[K]] = null): IndexedRDD[K,V] = + def indexed(existingIndex: RDDIndex[K] = null): IndexedRDD[K,V] = IndexedRDD(self, existingIndex) private[spark] def getKeyClass() = implicitly[ClassManifest[K]].erasure diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index 1c5095fa865fe..16644715246cf 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -55,7 +55,7 @@ import spark.rdd.ZippedPartitionsRDD2 import spark.rdd.ZippedPartitionsRDD3 import spark.rdd.ZippedPartitionsRDD4 -import spark.rdd.{IndexedRDD, BlockIndex} +import spark.rdd.{IndexedRDD, RDDIndex} import spark.storage.StorageLevel import spark.util.BoundedPriorityQueue @@ -774,7 +774,7 @@ abstract class RDD[T: ClassManifest]( } - def makeIndex(partitioner: Option[Partitioner] = None): RDD[BlockIndex[T]] = + def makeIndex(partitioner: Option[Partitioner] = None): RDDIndex[T] = IndexedRDD.makeIndex(this, partitioner) diff --git a/core/src/main/scala/spark/rdd/IndexedRDD.scala b/core/src/main/scala/spark/rdd/IndexedRDD.scala index d19d220ec24d3..ab1e460aeb651 100644 --- a/core/src/main/scala/spark/rdd/IndexedRDD.scala +++ b/core/src/main/scala/spark/rdd/IndexedRDD.scala @@ -31,6 +31,10 @@ import spark.rdd._ import spark.SparkContext._ import spark.Partitioner._ +import spark.storage.StorageLevel + + + @@ -41,6 +45,20 @@ import spark.Partitioner._ class BlockIndex[@specialized K: ClassManifest] extends JHashMap[K,Int] +/** + * The RDDIndex is an opaque type used to represent the organization + * of values in an RDD + */ +class RDDIndex[@specialized K: ClassManifest](private[spark] val rdd: RDD[BlockIndex[K]]) { + def persist(newLevel: StorageLevel): RDDIndex[K] = { + rdd.persist(newLevel) + return this + } +} + + + + /** * An IndexedRDD[K,V] extends the RDD[(K,V)] by pre-indexing the keys and @@ -51,30 +69,41 @@ class BlockIndex[@specialized K: ClassManifest] extends JHashMap[K,Int] * */ class IndexedRDD[K: ClassManifest, V: ClassManifest]( - val index: RDD[ BlockIndex[K] ], + val index: RDDIndex[K], val valuesRDD: RDD[ Seq[Seq[V]] ]) - extends RDD[(K, V)](index.context, - List(new OneToOneDependency(index), new OneToOneDependency(valuesRDD)) ) { + extends RDD[(K, V)](index.rdd.context, + List(new OneToOneDependency(index.rdd), new OneToOneDependency(valuesRDD)) ) { /** - * An internal representation of the maps and block managers + * An internal representation which joins the block indices with the values */ - protected val tuples = new ZippedRDD(index.context, index, valuesRDD) + protected val tuples = new ZippedRDD(index.rdd.context, index.rdd, valuesRDD) - override val partitioner = index.partitioner + /** + * The partitioner is defined by the index + */ + override val partitioner = index.rdd.partitioner + /** + * The actual partitions are defined by the tuples. + */ override def getPartitions: Array[Partition] = tuples.getPartitions - + /** + * The preferred locations are computed based on the preferred locations of the tuples. + */ override def getPreferredLocations(s: Partition): Seq[String] = tuples.getPreferredLocations(s) - override def cache: IndexedRDD[K,V] = { - index.cache - valuesRDD.cache + /** + * Caching an IndexedRDD causes the index and values to be cached separately. + */ + override def persist(newLevel: StorageLevel): RDD[(K,V)] = { + index.persist(newLevel) + valuesRDD.persist(newLevel) return this } @@ -90,7 +119,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( * this also retains the original RDD's partitioning. */ def mapValues[U: ClassManifest](f: V => U): IndexedRDD[K, U] = { - val cleanF = index.context.clean(f) + val cleanF = index.rdd.context.clean(f) val newValues = valuesRDD.mapPartitions(_.map(values => values.map{ case null => null case row => row.map(x => f(x)) @@ -104,7 +133,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( * keys; this also retains the original RDD's partitioning. */ def flatMapValues[U: ClassManifest](f: V => TraversableOnce[U]): IndexedRDD[K,U] = { - val cleanF = index.context.clean(f) + val cleanF = index.rdd.context.clean(f) val newValues = valuesRDD.mapPartitions(_.map(values => values.map{ case null => null case row => row.flatMap(x => f(x)) @@ -211,12 +240,12 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( }) new IndexedRDD[K, (Seq[V], Seq[W])](index, newValues) } - case other: IndexedRDD[_, _] if other.index.partitioner == index.partitioner => { + case other: IndexedRDD[_, _] if other.index.rdd.partitioner == index.rdd.partitioner => { // If both RDDs are indexed using different indices but with the same partitioners // then we we need to first merge the indicies and then use the merged index to // merge the values. val newIndex = - index.zipPartitions(other.index)( + index.rdd.zipPartitions(other.index.rdd)( (thisIter, otherIter) => { val thisIndex = thisIter.next() assert(!thisIter.hasNext()) @@ -264,11 +293,11 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( } List(newValues.toSeq).iterator }) - new IndexedRDD(newIndex, newValues) + new IndexedRDD(new RDDIndex(newIndex), newValues) } case _ => { // Get the partitioner from the index - val partitioner = index.partitioner match { + val partitioner = index.rdd.partitioner match { case Some(p) => p case None => throw new SparkException("An index must have a partitioner.") } @@ -328,7 +357,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( val newIndex = groups.mapPartitions(_.map{ case (kMap,vAr) => kMap }, true) val newValues = groups.mapPartitions(_.map{ case (kMap,vAr) => vAr }, true) - new IndexedRDD[K, (Seq[V], Seq[W])](newIndex, newValues) + new IndexedRDD[K, (Seq[V], Seq[W])](new RDDIndex(newIndex), newValues) } } } @@ -440,7 +469,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( object IndexedRDD { def apply[K: ClassManifest, V: ClassManifest]( tbl: RDD[(K,V)], - existingIndex: RDD[BlockIndex[K]] = null ): IndexedRDD[K, V] = { + existingIndex: RDDIndex[K] = null ): IndexedRDD[K, V] = { if (existingIndex == null) { // Shuffle the table (if necessary) @@ -466,10 +495,10 @@ object IndexedRDD { // extract the index and the values val index = groups.mapPartitions(_.map{ case (kMap,vAr) => kMap }, true) val values = groups.mapPartitions(_.map{ case (kMap,vAr) => vAr }, true) - new IndexedRDD[K,V](index, values) + new IndexedRDD[K,V](new RDDIndex(index), values) } else { val index = existingIndex - val partitioner = index.partitioner match { + val partitioner = index.rdd.partitioner match { case Some(p) => p case None => throw new SparkException("An index must have a partitioner.") } @@ -483,7 +512,7 @@ object IndexedRDD { } // Use the index to build the new values table - val values = index.zipPartitions(shuffledTbl)( + val values = index.rdd.zipPartitions(shuffledTbl)( (indexIter, tblIter) => { // There is only one map val index = indexIter.next() @@ -511,7 +540,7 @@ object IndexedRDD { * Construct and index of the unique values in a given RDD. */ def makeIndex[K: ClassManifest](keys: RDD[K], - partitioner: Option[Partitioner] = None): RDD[BlockIndex[K]] = { + partitioner: Option[Partitioner] = None): RDDIndex[K] = { // Ugly hack :-(. In order to partition the keys they must have values. @@ -539,7 +568,7 @@ object IndexedRDD { } List(indexMap).iterator }, true).cache - index + new RDDIndex(index) } } From 93503a7054e07cec71d8cac006e8585acb0eceb0 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Tue, 27 Aug 2013 18:16:19 -0700 Subject: [PATCH 065/531] Allowing RDD to select its implementation of PairRDDFunctions --- .../scala/spark/IndexedRDDFunctions.scala | 273 ++++++++++++++ .../main/scala/spark/PairRDDFunctions.scala | 44 +-- core/src/main/scala/spark/RDD.scala | 13 + core/src/main/scala/spark/SparkContext.scala | 2 +- .../scala/spark/api/java/JavaPairRDD.scala | 78 +++- .../src/main/scala/spark/rdd/IndexedRDD.scala | 340 +----------------- 6 files changed, 380 insertions(+), 370 deletions(-) create mode 100644 core/src/main/scala/spark/IndexedRDDFunctions.scala diff --git a/core/src/main/scala/spark/IndexedRDDFunctions.scala b/core/src/main/scala/spark/IndexedRDDFunctions.scala new file mode 100644 index 0000000000000..8bfe9d75e1ed2 --- /dev/null +++ b/core/src/main/scala/spark/IndexedRDDFunctions.scala @@ -0,0 +1,273 @@ +/* + * 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 spark + +import java.util.{HashMap => JHashMap, BitSet => JBitSet, HashSet => JHashSet} + +import scala.collection.JavaConversions._ +import scala.collection.mutable.ArrayBuffer + +import spark._ + +import spark.rdd.ShuffledRDD +import spark.rdd.IndexedRDD +import spark.rdd.BlockIndex +import spark.rdd.RDDIndex + + +class IndexedRDDFunctions[K: ClassManifest, V: ClassManifest](self: IndexedRDD[K,V]) + extends PairRDDFunctions[K,V](self) { + + /** + * Construct a new IndexedRDD that is indexed by only the keys in the RDD + */ + def reindex(): IndexedRDD[K,V] = IndexedRDD(self) + + + /** + * Pass each value in the key-value pair RDD through a map function without changing the keys; + * this also retains the original RDD's partitioning. + */ + override def mapValues[U: ClassManifest](f: V => U): RDD[(K, U)] = { + val cleanF = self.index.rdd.context.clean(f) + val newValues = self.valuesRDD.mapPartitions(_.map(values => values.map{ + case null => null + case row => row.map(x => f(x)) + }), true) + new IndexedRDD[K,U](self.index, newValues) + } + + + /** + * Pass each value in the key-value pair RDD through a flatMap function without changing the + * keys; this also retains the original RDD's partitioning. + */ + override def flatMapValues[U: ClassManifest](f: V => TraversableOnce[U]): RDD[(K,U)] = { + val cleanF = self.index.rdd.context.clean(f) + val newValues = self.valuesRDD.mapPartitions(_.map(values => values.map{ + case null => null + case row => row.flatMap(x => f(x)) + }), true) + new IndexedRDD[K,U](self.index, newValues) + } + + + /** + * Generic function to combine the elements for each key using a custom set of aggregation + * functions. Turns an RDD[(K, V)] into a result of type RDD[(K, C)], for a "combined type" C + * Note that V and C can be different -- for example, one might group an RDD of type + * (Int, Int) into an RDD of type (Int, Seq[Int]). Users provide three functions: + * + * - `createCombiner`, which turns a V into a C (e.g., creates a one-element list) + * - `mergeValue`, to merge a V into a C (e.g., adds it to the end of a list) + * - `mergeCombiners`, to combine two C's into a single one. + */ + override def combineByKey[C: ClassManifest](createCombiner: V => C, + mergeValue: (C, V) => C, + mergeCombiners: (C, C) => C, + partitioner: Partitioner, + mapSideCombine: Boolean = true, + serializerClass: String = null): RDD[(K, C)] = { + val newValues = self.valuesRDD.mapPartitions( + _.map{ groups: Seq[Seq[V]] => + groups.map{ group: Seq[V] => + if (group != null && !group.isEmpty) { + val c: C = createCombiner(group.head) + val sum: C = group.tail.foldLeft(c)(mergeValue) + Seq(sum) + } else { + null + } + } + }, true) + new IndexedRDD[K,C](self.index, newValues) + } + + + + /** + * Group the values for each key in the RDD into a single sequence. Hash-partitions the + * resulting RDD with the existing partitioner/parallelism level. + */ + override def groupByKey(partitioner: Partitioner): RDD[(K, Seq[V])] = { + val newValues = self.valuesRDD.mapPartitions(_.map{ar => ar.map{s => Seq(s)} }, true) + new IndexedRDD[K, Seq[V]](self.index, newValues) + } + + + /** + * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the + * list of values for that key in `this` as well as `other`. + */ + override def cogroup[W: ClassManifest](other: RDD[(K, W)], partitioner: Partitioner): + IndexedRDD[K, (Seq[V], Seq[W])] = { + //RDD[(K, (Seq[V], Seq[W]))] = { + other match { + case other: IndexedRDD[_, _] if self.index == other.index => { + // if both RDDs share exactly the same index and therefore the same super set of keys + // then we simply merge the value RDDs. + // However it is possible that both RDDs are missing a value for a given key in + // which case the returned RDD should have a null value + val newValues = + self.valuesRDD.zipPartitions(other.valuesRDD)( + (thisIter, otherIter) => { + val thisValues: Seq[Seq[V]] = thisIter.next() + assert(!thisIter.hasNext) + val otherValues: Seq[Seq[W]] = otherIter.next() + assert(!otherIter.hasNext) + // Zip the values and if both arrays are null then the key is not present and + // so the resulting value must be null (not a tuple of empty sequences) + val tmp: Seq[Seq[(Seq[V], Seq[W])]] = thisValues.view.zip(otherValues).map{ + case (null, null) => null // The key is not present in either RDD + case (a, null) => Seq((a, Seq.empty[W])) + case (null, b) => Seq((Seq.empty[V], b)) + case (a, b) => Seq((a,b)) + }.toSeq + List(tmp).iterator + }) + new IndexedRDD[K, (Seq[V], Seq[W])](self.index, newValues) + } + case other: IndexedRDD[_, _] + if self.index.rdd.partitioner == other.index.rdd.partitioner => { + // If both RDDs are indexed using different indices but with the same partitioners + // then we we need to first merge the indicies and then use the merged index to + // merge the values. + val newIndex = + self.index.rdd.zipPartitions(other.index.rdd)( + (thisIter, otherIter) => { + val thisIndex = thisIter.next() + assert(!thisIter.hasNext) + val otherIndex = otherIter.next() + assert(!otherIter.hasNext) + val newIndex = new BlockIndex[K]() + // @todo Merge only the keys that correspond to non-null values + // Merge the keys + newIndex.putAll(thisIndex) + newIndex.putAll(otherIndex) + // We need to rekey the index + var ctr = 0 + for (e <- newIndex.entrySet) { + e.setValue(ctr) + ctr += 1 + } + List(newIndex).iterator + }).cache() + // Use the new index along with the this and the other indices to merge the values + val newValues = + newIndex.zipPartitions(self.tuples, other.tuples)( + (newIndexIter, thisTuplesIter, otherTuplesIter) => { + // Get the new index for this partition + val newIndex = newIndexIter.next() + assert(!newIndexIter.hasNext) + // Get the corresponding indicies and values for this and the other IndexedRDD + val (thisIndex, thisValues) = thisTuplesIter.next() + assert(!thisTuplesIter.hasNext) + val (otherIndex, otherValues) = otherTuplesIter.next() + assert(!otherTuplesIter.hasNext) + // Preallocate the new Values array + val newValues = new Array[Seq[(Seq[V],Seq[W])]](newIndex.size) + // Lookup the sequences in both submaps + for ((k,ind) <- newIndex) { + val thisSeq = if (thisIndex.contains(k)) thisValues(thisIndex.get(k)) else null + val otherSeq = if (otherIndex.contains(k)) otherValues(otherIndex.get(k)) else null + // if either of the sequences is not null then the key was in one of the two tables + // and so the value should appear in the returned table + newValues(ind) = (thisSeq, otherSeq) match { + case (null, null) => null + case (a, null) => Seq( (a, Seq.empty[W]) ) + case (null, b) => Seq( (Seq.empty[V], b) ) + case (a, b) => Seq( (a,b) ) + } + } + List(newValues.toSeq).iterator + }) + new IndexedRDD(new RDDIndex(newIndex), newValues) + } + case _ => { + // Get the partitioner from the index + val partitioner = self.index.rdd.partitioner match { + case Some(p) => p + case None => throw new SparkException("An index must have a partitioner.") + } + // Shuffle the other RDD using the partitioner for this index + val otherShuffled = + if (other.partitioner == Some(partitioner)) { + other + } else { + new ShuffledRDD[K,W](other, partitioner) + } + // Join the other RDD with this RDD building a new valueset and new index on the fly + val groups = + self.tuples.zipPartitions(otherShuffled)( + (thisTuplesIter, otherTuplesIter) => { + // Get the corresponding indicies and values for this IndexedRDD + val (thisIndex, thisValues) = thisTuplesIter.next() + assert(!thisTuplesIter.hasNext()) + // Construct a new index + val newIndex = thisIndex.clone().asInstanceOf[BlockIndex[K]] + // Construct a new array Buffer to store the values + val newValues = ArrayBuffer.fill[(Seq[V], Seq[W])](thisValues.size)(null) + // populate the newValues with the values in this IndexedRDD + for ((k,i) <- thisIndex) { + if (thisValues(i) != null) { + newValues(i) = (thisValues(i), ArrayBuffer.empty[W]) + } + } + // Now iterate through the other tuples updating the map + for ((k,w) <- otherTuplesIter){ + if (!newIndex.contains(k)) { + // update the index + val ind = newIndex.size + newIndex.put(k, ind) + // Update the values + newValues.append( (Seq.empty[V], ArrayBuffer(w) ) ) + } else { + val ind = newIndex.get(k) + if(newValues(ind) == null) { + // If the other key was in the index but not in the values + // of this indexed RDD then create a new values entry for it + newValues(ind) = (Seq.empty[V], ArrayBuffer(w)) + } else { + newValues(ind)._2.asInstanceOf[ArrayBuffer[W]].append(w) + } + } + } + // Finalize the new values array + val newValuesArray: Seq[Seq[(Seq[V],Seq[W])]] = + newValues.view.map{ + case null => null + case (s, ab) => Seq((s, ab.toSeq)) + }.toSeq + List( (newIndex, newValuesArray) ).iterator + }).cache() + + // Extract the index and values from the above RDD + val newIndex = groups.mapPartitions(_.map{ case (kMap,vAr) => kMap }, true) + val newValues = groups.mapPartitions(_.map{ case (kMap,vAr) => vAr }, true) + + new IndexedRDD[K, (Seq[V], Seq[W])](new RDDIndex(newIndex), newValues) + } + } + } + + +} + +//(self: IndexedRDD[K, V]) extends PairRDDFunctions(self) { } + + diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index 03f25040a114a..ca42980b468f6 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -69,7 +69,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * In addition, users can control the partitioning of the output RDD, and whether to perform * map-side aggregation (if a mapper can produce multiple items with the same key). */ - def combineByKey[C](createCombiner: V => C, + def combineByKey[C: ClassManifest](createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => C, partitioner: Partitioner, @@ -103,7 +103,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( /** * Simplified version of combineByKey that hash-partitions the output RDD. */ - def combineByKey[C](createCombiner: V => C, + def combineByKey[C: ClassManifest](createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => C, numPartitions: Int): RDD[(K, C)] = { @@ -248,7 +248,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and * (k, v2) is in `other`. Uses the given Partitioner to partition the output RDD. */ - def join[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, W))] = { + def join[W: ClassManifest](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, W))] = { this.cogroup(other, partitioner).flatMapValues { case (vs, ws) => for (v <- vs.iterator; w <- ws.iterator) yield (v, w) @@ -261,7 +261,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * pair (k, (v, None)) if no elements in `other` have key k. Uses the given Partitioner to * partition the output RDD. */ - def leftOuterJoin[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, Option[W]))] = { + def leftOuterJoin[W: ClassManifest](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, Option[W]))] = { this.cogroup(other, partitioner).flatMapValues { case (vs, ws) => if (ws.isEmpty) { @@ -278,7 +278,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * pair (k, (None, w)) if no elements in `this` have key k. Uses the given Partitioner to * partition the output RDD. */ - def rightOuterJoin[W](other: RDD[(K, W)], partitioner: Partitioner) + def rightOuterJoin[W: ClassManifest](other: RDD[(K, W)], partitioner: Partitioner) : RDD[(K, (Option[V], W))] = { this.cogroup(other, partitioner).flatMapValues { case (vs, ws) => @@ -294,7 +294,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * Simplified version of combineByKey that hash-partitions the resulting RDD using the * existing partitioner/parallelism level. */ - def combineByKey[C](createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => C) + def combineByKey[C: ClassManifest](createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => C) : RDD[(K, C)] = { combineByKey(createCombiner, mergeValue, mergeCombiners, defaultPartitioner(self)) } @@ -322,7 +322,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and * (k, v2) is in `other`. Performs a hash join across the cluster. */ - def join[W](other: RDD[(K, W)]): RDD[(K, (V, W))] = { + def join[W: ClassManifest](other: RDD[(K, W)]): RDD[(K, (V, W))] = { join(other, defaultPartitioner(self, other)) } @@ -331,7 +331,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and * (k, v2) is in `other`. Performs a hash join across the cluster. */ - def join[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (V, W))] = { + def join[W: ClassManifest](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (V, W))] = { join(other, new HashPartitioner(numPartitions)) } @@ -341,7 +341,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * pair (k, (v, None)) if no elements in `other` have key k. Hash-partitions the output * using the existing partitioner/parallelism level. */ - def leftOuterJoin[W](other: RDD[(K, W)]): RDD[(K, (V, Option[W]))] = { + def leftOuterJoin[W: ClassManifest](other: RDD[(K, W)]): RDD[(K, (V, Option[W]))] = { leftOuterJoin(other, defaultPartitioner(self, other)) } @@ -351,7 +351,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * pair (k, (v, None)) if no elements in `other` have key k. Hash-partitions the output * into `numPartitions` partitions. */ - def leftOuterJoin[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (V, Option[W]))] = { + def leftOuterJoin[W: ClassManifest](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (V, Option[W]))] = { leftOuterJoin(other, new HashPartitioner(numPartitions)) } @@ -361,7 +361,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * pair (k, (None, w)) if no elements in `this` have key k. Hash-partitions the resulting * RDD using the existing partitioner/parallelism level. */ - def rightOuterJoin[W](other: RDD[(K, W)]): RDD[(K, (Option[V], W))] = { + def rightOuterJoin[W: ClassManifest](other: RDD[(K, W)]): RDD[(K, (Option[V], W))] = { rightOuterJoin(other, defaultPartitioner(self, other)) } @@ -371,7 +371,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * pair (k, (None, w)) if no elements in `this` have key k. Hash-partitions the resulting * RDD into the given number of partitions. */ - def rightOuterJoin[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (Option[V], W))] = { + def rightOuterJoin[W: ClassManifest](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (Option[V], W))] = { rightOuterJoin(other, new HashPartitioner(numPartitions)) } @@ -384,7 +384,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * Pass each value in the key-value pair RDD through a map function without changing the keys; * this also retains the original RDD's partitioning. */ - def mapValues[U](f: V => U): RDD[(K, U)] = { + def mapValues[U: ClassManifest](f: V => U): RDD[(K, U)] = { val cleanF = self.context.clean(f) new MappedValuesRDD(self, cleanF) } @@ -393,7 +393,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * Pass each value in the key-value pair RDD through a flatMap function without changing the * keys; this also retains the original RDD's partitioning. */ - def flatMapValues[U](f: V => TraversableOnce[U]): RDD[(K, U)] = { + def flatMapValues[U: ClassManifest](f: V => TraversableOnce[U]): RDD[(K, U)] = { val cleanF = self.context.clean(f) new FlatMappedValuesRDD(self, cleanF) } @@ -402,7 +402,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the * list of values for that key in `this` as well as `other`. */ - def cogroup[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (Seq[V], Seq[W]))] = { + def cogroup[W: ClassManifest](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (Seq[V], Seq[W]))] = { if (partitioner.isInstanceOf[HashPartitioner] && getKeyClass().isArray) { throw new SparkException("Default partitioner cannot partition array keys.") } @@ -420,7 +420,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a * tuple with the list of values for that key in `this`, `other1` and `other2`. */ - def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)], partitioner: Partitioner) + def cogroup[W1: ClassManifest, W2: ClassManifest](other1: RDD[(K, W1)], other2: RDD[(K, W2)], partitioner: Partitioner) : RDD[(K, (Seq[V], Seq[W1], Seq[W2]))] = { if (partitioner.isInstanceOf[HashPartitioner] && getKeyClass().isArray) { throw new SparkException("Default partitioner cannot partition array keys.") @@ -441,7 +441,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the * list of values for that key in `this` as well as `other`. */ - def cogroup[W](other: RDD[(K, W)]): RDD[(K, (Seq[V], Seq[W]))] = { + def cogroup[W: ClassManifest](other: RDD[(K, W)]): RDD[(K, (Seq[V], Seq[W]))] = { cogroup(other, defaultPartitioner(self, other)) } @@ -449,7 +449,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a * tuple with the list of values for that key in `this`, `other1` and `other2`. */ - def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)]) + def cogroup[W1: ClassManifest, W2: ClassManifest](other1: RDD[(K, W1)], other2: RDD[(K, W2)]) : RDD[(K, (Seq[V], Seq[W1], Seq[W2]))] = { cogroup(other1, other2, defaultPartitioner(self, other1, other2)) } @@ -458,7 +458,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the * list of values for that key in `this` as well as `other`. */ - def cogroup[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (Seq[V], Seq[W]))] = { + def cogroup[W: ClassManifest](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (Seq[V], Seq[W]))] = { cogroup(other, new HashPartitioner(numPartitions)) } @@ -466,18 +466,18 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a * tuple with the list of values for that key in `this`, `other1` and `other2`. */ - def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)], numPartitions: Int) + def cogroup[W1: ClassManifest, W2: ClassManifest](other1: RDD[(K, W1)], other2: RDD[(K, W2)], numPartitions: Int) : RDD[(K, (Seq[V], Seq[W1], Seq[W2]))] = { cogroup(other1, other2, new HashPartitioner(numPartitions)) } /** Alias for cogroup. */ - def groupWith[W](other: RDD[(K, W)]): RDD[(K, (Seq[V], Seq[W]))] = { + def groupWith[W: ClassManifest](other: RDD[(K, W)]): RDD[(K, (Seq[V], Seq[W]))] = { cogroup(other, defaultPartitioner(self, other)) } /** Alias for cogroup. */ - def groupWith[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)]) + def groupWith[W1: ClassManifest, W2: ClassManifest](other1: RDD[(K, W1)], other2: RDD[(K, W2)]) : RDD[(K, (Seq[V], Seq[W1], Seq[W2]))] = { cogroup(other1, other2, defaultPartitioner(self, other1, other2)) } diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index 16644715246cf..95adbda43f40d 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -774,6 +774,19 @@ abstract class RDD[T: ClassManifest]( } + + // def pairRDDFunctions[K: ClassManifest, V,](implicit t: T <:< (K, V), k: ClassManifest[K], v: ClassManifest[V]): + // PairRDDFunctions[K, V] = { + // new PairRDDFunctions(this.asInstanceOf[RDD[(K,V)]]) + // } + + def pairRDDFunctions[K, V]( + implicit t: T <:< (K, V), k: ClassManifest[K], v: ClassManifest[V]): + PairRDDFunctions[K, V] = { + new PairRDDFunctions(this.asInstanceOf[RDD[(K,V)]]) + } + + def makeIndex(partitioner: Option[Partitioner] = None): RDDIndex[T] = IndexedRDD.makeIndex(this, partitioner) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 80c65dfebd2ad..739473078600c 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -878,7 +878,7 @@ object SparkContext { // TODO: Add AccumulatorParams for other types, e.g. lists and strings implicit def rddToPairRDDFunctions[K: ClassManifest, V: ClassManifest](rdd: RDD[(K, V)]) = - new PairRDDFunctions(rdd) + rdd.pairRDDFunctions implicit def rddToSequenceFileRDDFunctions[K <% Writable: ClassManifest, V <% Writable: ClassManifest]( rdd: RDD[(K, V)]) = diff --git a/core/src/main/scala/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/spark/api/java/JavaPairRDD.scala index c2995b836a14a..0b444c5a7e77d 100644 --- a/core/src/main/scala/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/spark/api/java/JavaPairRDD.scala @@ -263,8 +263,11 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif * the merging locally on each mapper before sending results to a reducer, similarly to a * "combiner" in MapReduce. */ - def join[W](other: JavaPairRDD[K, W], partitioner: Partitioner): JavaPairRDD[K, (V, W)] = + def join[W](other: JavaPairRDD[K, W], partitioner: Partitioner): JavaPairRDD[K, (V, W)] = { + implicit val wm: ClassManifest[W] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] fromRDD(rdd.join(other, partitioner)) + } /** * Perform a left outer join of `this` and `other`. For each element (k, v) in `this`, the @@ -274,6 +277,8 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif */ def leftOuterJoin[W](other: JavaPairRDD[K, W], partitioner: Partitioner) : JavaPairRDD[K, (V, Optional[W])] = { + implicit val wm: ClassManifest[W] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] val joinResult = rdd.leftOuterJoin(other, partitioner) fromRDD(joinResult.mapValues{case (v, w) => (v, JavaUtils.optionToOptional(w))}) } @@ -286,6 +291,8 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif */ def rightOuterJoin[W](other: JavaPairRDD[K, W], partitioner: Partitioner) : JavaPairRDD[K, (Optional[V], W)] = { + implicit val wm: ClassManifest[W] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] val joinResult = rdd.rightOuterJoin(other, partitioner) fromRDD(joinResult.mapValues{case (v, w) => (JavaUtils.optionToOptional(v), w)}) } @@ -324,16 +331,22 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and * (k, v2) is in `other`. Performs a hash join across the cluster. */ - def join[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (V, W)] = + def join[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (V, W)] = { + implicit val wm: ClassManifest[W] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] fromRDD(rdd.join(other)) + } /** * Return an RDD containing all pairs of elements with matching keys in `this` and `other`. Each * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and * (k, v2) is in `other`. Performs a hash join across the cluster. */ - def join[W](other: JavaPairRDD[K, W], numPartitions: Int): JavaPairRDD[K, (V, W)] = + def join[W](other: JavaPairRDD[K, W], numPartitions: Int): JavaPairRDD[K, (V, W)] = { + implicit val wm: ClassManifest[W] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] fromRDD(rdd.join(other, numPartitions)) + } /** * Perform a left outer join of `this` and `other`. For each element (k, v) in `this`, the @@ -342,6 +355,8 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif * using the existing partitioner/parallelism level. */ def leftOuterJoin[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (V, Optional[W])] = { + implicit val wm: ClassManifest[W] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] val joinResult = rdd.leftOuterJoin(other) fromRDD(joinResult.mapValues{case (v, w) => (v, JavaUtils.optionToOptional(w))}) } @@ -353,6 +368,8 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif * into `numPartitions` partitions. */ def leftOuterJoin[W](other: JavaPairRDD[K, W], numPartitions: Int): JavaPairRDD[K, (V, Optional[W])] = { + implicit val wm: ClassManifest[W] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] val joinResult = rdd.leftOuterJoin(other, numPartitions) fromRDD(joinResult.mapValues{case (v, w) => (v, JavaUtils.optionToOptional(w))}) } @@ -364,6 +381,8 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif * RDD using the existing partitioner/parallelism level. */ def rightOuterJoin[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (Optional[V], W)] = { + implicit val wm: ClassManifest[W] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] val joinResult = rdd.rightOuterJoin(other) fromRDD(joinResult.mapValues{case (v, w) => (JavaUtils.optionToOptional(v), w)}) } @@ -375,6 +394,8 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif * RDD into the given number of partitions. */ def rightOuterJoin[W](other: JavaPairRDD[K, W], numPartitions: Int): JavaPairRDD[K, (Optional[V], W)] = { + implicit val wm: ClassManifest[W] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] val joinResult = rdd.rightOuterJoin(other, numPartitions) fromRDD(joinResult.mapValues{case (v, w) => (JavaUtils.optionToOptional(v), w)}) } @@ -411,55 +432,86 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif * list of values for that key in `this` as well as `other`. */ def cogroup[W](other: JavaPairRDD[K, W], partitioner: Partitioner) - : JavaPairRDD[K, (JList[V], JList[W])] = + : JavaPairRDD[K, (JList[V], JList[W])] = { + implicit val wm: ClassManifest[W] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] fromRDD(cogroupResultToJava(rdd.cogroup(other, partitioner))) + } /** * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a * tuple with the list of values for that key in `this`, `other1` and `other2`. */ def cogroup[W1, W2](other1: JavaPairRDD[K, W1], other2: JavaPairRDD[K, W2], partitioner: Partitioner) - : JavaPairRDD[K, (JList[V], JList[W1], JList[W2])] = + : JavaPairRDD[K, (JList[V], JList[W1], JList[W2])] = { + implicit val w1m: ClassManifest[W1] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W1]] + implicit val w2m: ClassManifest[W2] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W2]] fromRDD(cogroupResult2ToJava(rdd.cogroup(other1, other2, partitioner))) + } /** * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the * list of values for that key in `this` as well as `other`. */ - def cogroup[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (JList[V], JList[W])] = + def cogroup[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (JList[V], JList[W])] = { + implicit val wm: ClassManifest[W] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] fromRDD(cogroupResultToJava(rdd.cogroup(other))) + } /** * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a * tuple with the list of values for that key in `this`, `other1` and `other2`. */ def cogroup[W1, W2](other1: JavaPairRDD[K, W1], other2: JavaPairRDD[K, W2]) - : JavaPairRDD[K, (JList[V], JList[W1], JList[W2])] = + : JavaPairRDD[K, (JList[V], JList[W1], JList[W2])] = { + implicit val w1m: ClassManifest[W1] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W1]] + implicit val w2m: ClassManifest[W2] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W2]] fromRDD(cogroupResult2ToJava(rdd.cogroup(other1, other2))) + } /** * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the * list of values for that key in `this` as well as `other`. */ - def cogroup[W](other: JavaPairRDD[K, W], numPartitions: Int): JavaPairRDD[K, (JList[V], JList[W])] - = fromRDD(cogroupResultToJava(rdd.cogroup(other, numPartitions))) - + def cogroup[W](other: JavaPairRDD[K, W], numPartitions: Int): JavaPairRDD[K, (JList[V], JList[W])] = { + implicit val wm: ClassManifest[W] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] + fromRDD(cogroupResultToJava(rdd.cogroup(other, numPartitions))) + } /** * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a * tuple with the list of values for that key in `this`, `other1` and `other2`. */ def cogroup[W1, W2](other1: JavaPairRDD[K, W1], other2: JavaPairRDD[K, W2], numPartitions: Int) - : JavaPairRDD[K, (JList[V], JList[W1], JList[W2])] = + : JavaPairRDD[K, (JList[V], JList[W1], JList[W2])] = { + implicit val w1m: ClassManifest[W1] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W1]] + implicit val w2m: ClassManifest[W2] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W2]] fromRDD(cogroupResult2ToJava(rdd.cogroup(other1, other2, numPartitions))) + } /** Alias for cogroup. */ - def groupWith[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (JList[V], JList[W])] = + def groupWith[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (JList[V], JList[W])] = { + implicit val wm: ClassManifest[W] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] fromRDD(cogroupResultToJava(rdd.groupWith(other))) + } /** Alias for cogroup. */ def groupWith[W1, W2](other1: JavaPairRDD[K, W1], other2: JavaPairRDD[K, W2]) - : JavaPairRDD[K, (JList[V], JList[W1], JList[W2])] = + : JavaPairRDD[K, (JList[V], JList[W1], JList[W2])] = { + implicit val w1m: ClassManifest[W1] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W1]] + implicit val w2m: ClassManifest[W2] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W2]] fromRDD(cogroupResult2ToJava(rdd.groupWith(other1, other2))) + } /** * Return the list of values in the RDD for key `key`. This operation is done efficiently if the diff --git a/core/src/main/scala/spark/rdd/IndexedRDD.scala b/core/src/main/scala/spark/rdd/IndexedRDD.scala index ab1e460aeb651..ac80a06c8a65d 100644 --- a/core/src/main/scala/spark/rdd/IndexedRDD.scala +++ b/core/src/main/scala/spark/rdd/IndexedRDD.scala @@ -25,7 +25,6 @@ import java.util.{HashMap => JHashMap, BitSet => JBitSet, HashSet => JHashSet} import scala.collection.JavaConversions._ import scala.collection.mutable.ArrayBuffer - import spark._ import spark.rdd._ import spark.SparkContext._ @@ -77,7 +76,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( /** * An internal representation which joins the block indices with the values */ - protected val tuples = new ZippedRDD(index.rdd.context, index.rdd, valuesRDD) + protected[spark] val tuples = new ZippedRDD(index.rdd.context, index.rdd, valuesRDD) /** @@ -108,341 +107,14 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( } - /** - * Construct a new IndexedRDD that is indexed by only the keys in the RDD - */ - def reindex(): IndexedRDD[K,V] = IndexedRDD(this) - - - /** - * Pass each value in the key-value pair RDD through a map function without changing the keys; - * this also retains the original RDD's partitioning. - */ - def mapValues[U: ClassManifest](f: V => U): IndexedRDD[K, U] = { - val cleanF = index.rdd.context.clean(f) - val newValues = valuesRDD.mapPartitions(_.map(values => values.map{ - case null => null - case row => row.map(x => f(x)) - }), true) - new IndexedRDD[K,U](index, newValues) - } - - - /** - * Pass each value in the key-value pair RDD through a flatMap function without changing the - * keys; this also retains the original RDD's partitioning. - */ - def flatMapValues[U: ClassManifest](f: V => TraversableOnce[U]): IndexedRDD[K,U] = { - val cleanF = index.rdd.context.clean(f) - val newValues = valuesRDD.mapPartitions(_.map(values => values.map{ - case null => null - case row => row.flatMap(x => f(x)) - }), true) - new IndexedRDD[K,U](index, newValues) - } - - - /** - * Generic function to combine the elements for each key using a custom set of aggregation - * functions. Turns an RDD[(K, V)] into a result of type RDD[(K, C)], for a "combined type" C - * Note that V and C can be different -- for example, one might group an RDD of type - * (Int, Int) into an RDD of type (Int, Seq[Int]). Users provide three functions: - * - * - `createCombiner`, which turns a V into a C (e.g., creates a one-element list) - * - `mergeValue`, to merge a V into a C (e.g., adds it to the end of a list) - * - `mergeCombiners`, to combine two C's into a single one. - */ - def combineByKey[C: ClassManifest](createCombiner: V => C, - mergeValue: (C, V) => C, - mergeCombiners: (C, C) => C, - serializerClass: String = null): IndexedRDD[K, C] = { - val newValues = valuesRDD.mapPartitions( - _.map{ groups: Seq[Seq[V]] => - groups.map{ group: Seq[V] => - if (group != null && !group.isEmpty) { - val c: C = createCombiner(group.head) - val sum: C = group.tail.foldLeft(c)(mergeValue) - Seq(sum) - } else { - null - } - } - }, true) - new IndexedRDD[K,C](index, newValues) - } - - /** - * Merge the values for each key using an associative function and a neutral "zero value" which may - * be added to the result an arbitrary number of times, and must not change the result (e.g., Nil for - * list concatenation, 0 for addition, or 1 for multiplication.). - */ - def foldByKey(zeroValue: V)(func: (V, V) => V): IndexedRDD[K, V] = { - // Serialize the zero value to a byte array so that we can get a new clone of it on each key - val zeroBuffer = SparkEnv.get.closureSerializer.newInstance().serialize(zeroValue) - val zeroArray = new Array[Byte](zeroBuffer.limit) - zeroBuffer.get(zeroArray) - - // When deserializing, use a lazy val to create just one instance of the serializer per task - lazy val cachedSerializer = SparkEnv.get.closureSerializer.newInstance() - def createZero() = cachedSerializer.deserialize[V](ByteBuffer.wrap(zeroArray)) - combineByKey[V]((v: V) => func(createZero(), v), func, func) - } - - /** - * Merge the values for each key using an associative reduce function. This will also perform - * the merging locally on each mapper before sending results to a reducer, similarly to a - * "combiner" in MapReduce. Output will be hash-partitioned with the existing partitioner/ - * parallelism level. - */ - def reduceByKey(func: (V, V) => V): IndexedRDD[K, V] = { - combineByKey[V]((v: V) => v, func, func) - } - - - /** - * Group the values for each key in the RDD into a single sequence. Hash-partitions the - * resulting RDD with the existing partitioner/parallelism level. - */ - def groupByKey(): IndexedRDD[K, Seq[V]] = { - val newValues = valuesRDD.mapPartitions(_.map{ar => ar.map{s => Seq(s)} }, true) - new IndexedRDD[K, Seq[V]](index, newValues) - } - - - /** - * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the - * list of values for that key in `this` as well as `other`. - */ - def cogroup[W: ClassManifest](other: RDD[(K, W)]): IndexedRDD[K, (Seq[V], Seq[W])] = { - //RDD[(K, (Seq[V], Seq[W]))] = { - other match { - case other: IndexedRDD[_, _] if other.index == index => { - // if both RDDs share exactly the same index and therefore the same super set of keys - // then we simply merge the value RDDs. - // However it is possible that both RDDs are missing a value for a given key in - // which case the returned RDD should have a null value - val newValues = - valuesRDD.zipPartitions(other.valuesRDD)( - (thisIter, otherIter) => { - val thisValues: Seq[Seq[V]] = thisIter.next() - assert(!thisIter.hasNext()) - val otherValues: Seq[Seq[W]] = otherIter.next() - assert(!otherIter.hasNext()) - // Zip the values and if both arrays are null then the key is not present and - // so the resulting value must be null (not a tuple of empty sequences) - val tmp: Seq[Seq[(Seq[V], Seq[W])]] = thisValues.view.zip(otherValues).map{ - case (null, null) => null // The key is not present in either RDD - case (a, null) => Seq((a, Seq.empty[W])) - case (null, b) => Seq((Seq.empty[V], b)) - case (a, b) => Seq((a,b)) - }.toSeq - List(tmp).iterator - }) - new IndexedRDD[K, (Seq[V], Seq[W])](index, newValues) - } - case other: IndexedRDD[_, _] if other.index.rdd.partitioner == index.rdd.partitioner => { - // If both RDDs are indexed using different indices but with the same partitioners - // then we we need to first merge the indicies and then use the merged index to - // merge the values. - val newIndex = - index.rdd.zipPartitions(other.index.rdd)( - (thisIter, otherIter) => { - val thisIndex = thisIter.next() - assert(!thisIter.hasNext()) - val otherIndex = otherIter.next() - assert(!otherIter.hasNext()) - val newIndex = new BlockIndex[K]() - // @todo Merge only the keys that correspond to non-null values - // Merge the keys - newIndex.putAll(thisIndex) - newIndex.putAll(otherIndex) - // We need to rekey the index - var ctr = 0 - for (e <- newIndex.entrySet) { - e.setValue(ctr) - ctr += 1 - } - List(newIndex).iterator - }).cache() - // Use the new index along with the this and the other indices to merge the values - val newValues = - newIndex.zipPartitions(tuples, other.tuples)( - (newIndexIter, thisTuplesIter, otherTuplesIter) => { - // Get the new index for this partition - val newIndex = newIndexIter.next() - assert(!newIndexIter.hasNext()) - // Get the corresponding indicies and values for this and the other IndexedRDD - val (thisIndex, thisValues) = thisTuplesIter.next() - assert(!thisTuplesIter.hasNext()) - val (otherIndex, otherValues) = otherTuplesIter.next() - assert(!otherTuplesIter.hasNext()) - // Preallocate the new Values array - val newValues = new Array[Seq[(Seq[V],Seq[W])]](newIndex.size) - // Lookup the sequences in both submaps - for ((k,ind) <- newIndex) { - val thisSeq = if (thisIndex.contains(k)) thisValues(thisIndex.get(k)) else null - val otherSeq = if (otherIndex.contains(k)) otherValues(otherIndex.get(k)) else null - // if either of the sequences is not null then the key was in one of the two tables - // and so the value should appear in the returned table - newValues(ind) = (thisSeq, otherSeq) match { - case (null, null) => null - case (a, null) => Seq( (a, Seq.empty[W]) ) - case (null, b) => Seq( (Seq.empty[V], b) ) - case (a, b) => Seq( (a,b) ) - } - } - List(newValues.toSeq).iterator - }) - new IndexedRDD(new RDDIndex(newIndex), newValues) - } - case _ => { - // Get the partitioner from the index - val partitioner = index.rdd.partitioner match { - case Some(p) => p - case None => throw new SparkException("An index must have a partitioner.") - } - // Shuffle the other RDD using the partitioner for this index - val otherShuffled = - if (other.partitioner == Some(partitioner)) { - other - } else { - new ShuffledRDD[K,W](other, partitioner) - } - // Join the other RDD with this RDD building a new valueset and new index on the fly - val groups = - tuples.zipPartitions(otherShuffled)( - (thisTuplesIter, otherTuplesIter) => { - // Get the corresponding indicies and values for this IndexedRDD - val (thisIndex, thisValues) = thisTuplesIter.next() - assert(!thisTuplesIter.hasNext()) - // Construct a new index - val newIndex = thisIndex.clone().asInstanceOf[BlockIndex[K]] - // Construct a new array Buffer to store the values - val newValues = ArrayBuffer.fill[(Seq[V], Seq[W])](thisValues.size)(null) - // populate the newValues with the values in this IndexedRDD - for ((k,i) <- thisIndex) { - if (thisValues(i) != null) { - newValues(i) = (thisValues(i), ArrayBuffer.empty[W]) - } - } - // Now iterate through the other tuples updating the map - for ((k,w) <- otherTuplesIter){ - if (!newIndex.contains(k)) { - // update the index - val ind = newIndex.size - newIndex.put(k, ind) - // Update the values - newValues.append( (Seq.empty[V], ArrayBuffer(w) ) ) - } else { - val ind = newIndex.get(k) - if(newValues(ind) == null) { - // If the other key was in the index but not in the values - // of this indexed RDD then create a new values entry for it - newValues(ind) = (Seq.empty[V], ArrayBuffer(w)) - } else { - newValues(ind)._2.asInstanceOf[ArrayBuffer[W]].append(w) - } - } - } - // Finalize the new values array - val newValuesArray: Seq[Seq[(Seq[V],Seq[W])]] = - newValues.view.map{ - case null => null - case (s, ab) => Seq((s, ab.toSeq)) - }.toSeq - List( (newIndex, newValuesArray) ).iterator - }).cache() - - // Extract the index and values from the above RDD - val newIndex = groups.mapPartitions(_.map{ case (kMap,vAr) => kMap }, true) - val newValues = groups.mapPartitions(_.map{ case (kMap,vAr) => vAr }, true) - - new IndexedRDD[K, (Seq[V], Seq[W])](new RDDIndex(newIndex), newValues) - } - } - } - - - - // /** - // * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a - // * tuple with the list of values for that key in `this`, `other1` and `other2`. - // */ - // def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)]) - // : IndexedRDD[K, (Seq[V], Seq[W1], Seq[W2])] = { - // cogroup(other1, other2, defaultPartitioner(this, other1, other2)) - // } - - // /** - // * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a - // * tuple with the list of values for that key in `this`, `other1` and `other2`. - // */ - // def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)], numPartitions: Int) - // : RDD[(K, (Seq[V], Seq[W1], Seq[W2]))] = { - // cogroup(other1, other2, new HashPartitioner(numPartitions)) - // } - - /** Alias for cogroup. */ - def groupWith[W: ClassManifest](other: RDD[(K, W)]): IndexedRDD[K, (Seq[V], Seq[W])] = { - cogroup(other) + override def pairRDDFunctions[K1, V1]( + implicit t: (K, V) <:< (K1,V1), k: ClassManifest[K1], v: ClassManifest[V1]): + PairRDDFunctions[K1, V1] = { + new IndexedRDDFunctions[K1,V1](this.asInstanceOf[IndexedRDD[K1,V1]]) } - // /** Alias for cogroup. */ - // def groupWith[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)]) - // : RDD[(K, (Seq[V], Seq[W1], Seq[W2]))] = { - // cogroup(other1, other2, defaultPartitioner(self, other1, other2)) - // } - - /** - * Return an RDD containing all pairs of elements with matching keys in `this` and `other`. Each - * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and - * (k, v2) is in `other`. Performs a hash join across the cluster. - */ - def join[W: ClassManifest](other: RDD[(K, W)]): IndexedRDD[K, (V, W)] = { - cogroup(other).flatMapValues { - case (vs, ws) => - for (v <- vs.iterator; w <- ws.iterator) yield (v, w) - } - } - - - /** - * Perform a left outer join of `this` and `other`. For each element (k, v) in `this`, the - * resulting RDD will either contain all pairs (k, (v, Some(w))) for w in `other`, or the - * pair (k, (v, None)) if no elements in `other` have key k. Hash-partitions the output - * using the existing partitioner/parallelism level. - */ - def leftOuterJoin[W: ClassManifest](other: RDD[(K, W)]): IndexedRDD[K, (V, Option[W])] = { - cogroup(other).flatMapValues { - case (vs, ws) => - if (ws.isEmpty) { - vs.iterator.map(v => (v, None)) - } else { - for (v <- vs.iterator; w <- ws.iterator) yield (v, Some(w)) - } - } - - } - - - /** - * Perform a right outer join of `this` and `other`. For each element (k, w) in `other`, the - * resulting RDD will either contain all pairs (k, (Some(v), w)) for v in `this`, or the - * pair (k, (None, w)) if no elements in `this` have key k. Hash-partitions the resulting - * RDD using the existing partitioner/parallelism level. - */ - def rightOuterJoin[W: ClassManifest](other: RDD[(K, W)]): IndexedRDD[K, (Option[V], W)] = { - cogroup(other).flatMapValues { - case (vs, ws) => - if (vs.isEmpty) { - ws.iterator.map(w => (None, w)) - } else { - for (v <- vs.iterator; w <- ws.iterator) yield (Some(v), w) - } - } - - } + /** From 766b6fd380341da06cb1c97de71c74e0eb6c36d0 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Tue, 27 Aug 2013 18:54:26 -0700 Subject: [PATCH 066/531] Fixing IndexedRDD unit tests. --- core/src/main/scala/spark/rdd/IndexedRDD.scala | 4 ++-- core/src/test/scala/spark/IndexedRDDSuite.scala | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/rdd/IndexedRDD.scala b/core/src/main/scala/spark/rdd/IndexedRDD.scala index 5ddfe5db2c079..8536bb4a4bb38 100644 --- a/core/src/main/scala/spark/rdd/IndexedRDD.scala +++ b/core/src/main/scala/spark/rdd/IndexedRDD.scala @@ -68,8 +68,8 @@ class RDDIndex[@specialized K: ClassManifest](private[spark] val rdd: RDD[BlockI * */ class IndexedRDD[K: ClassManifest, V: ClassManifest]( - val index: RDDIndex[K], - val valuesRDD: RDD[ Seq[Seq[V]] ]) + @transient val index: RDDIndex[K], + @transient val valuesRDD: RDD[ Seq[Seq[V]] ]) extends RDD[(K, V)](index.rdd.context, List(new OneToOneDependency(index.rdd), new OneToOneDependency(valuesRDD)) ) { diff --git a/core/src/test/scala/spark/IndexedRDDSuite.scala b/core/src/test/scala/spark/IndexedRDDSuite.scala index c1433c50950ca..1a17f7c0d3691 100644 --- a/core/src/test/scala/spark/IndexedRDDSuite.scala +++ b/core/src/test/scala/spark/IndexedRDDSuite.scala @@ -123,7 +123,7 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { // count the dependencies to make sure there is only 1 ShuffledRDD val deps = lineage(sums) - assert(deps.filter(_.isInstanceOf[ShuffledRDD[_,_]]).size === 1) // ShuffledRDD, ParallelCollection + assert(deps.filter(_.isInstanceOf[ShuffledRDD[_,_,_]]).size === 1) // ShuffledRDD, ParallelCollection } From 33250835527c1894d9cecdc35b25017bddc68670 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Wed, 28 Aug 2013 14:57:12 -0700 Subject: [PATCH 067/531] Removing commented out test code --- core/src/main/scala/spark/RDD.scala | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index 6a2d3eb78b3b6..b308d2f62df16 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -777,11 +777,9 @@ abstract class RDD[T: ClassManifest]( - // def pairRDDFunctions[K: ClassManifest, V,](implicit t: T <:< (K, V), k: ClassManifest[K], v: ClassManifest[V]): - // PairRDDFunctions[K, V] = { - // new PairRDDFunctions(this.asInstanceOf[RDD[(K,V)]]) - // } - + /** + * For RDD[(K,V)] this function returns a pair-functions object for this RDD + */ def pairRDDFunctions[K, V]( implicit t: T <:< (K, V), k: ClassManifest[K], v: ClassManifest[V]): PairRDDFunctions[K, V] = { From 205dba352f0905f77ce285aa1ad7e92f67681e4f Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 30 Aug 2013 14:51:15 -0700 Subject: [PATCH 068/531] Update README. --- README.md | 74 +------------------------------------------------------ 1 file changed, 1 insertion(+), 73 deletions(-) diff --git a/README.md b/README.md index ba24ab43b1a10..4a4cc0425eabb 100644 --- a/README.md +++ b/README.md @@ -1,73 +1 @@ -# Spark - -Lightning-Fast Cluster Computing - - - -## Online Documentation - -You can find the latest Spark documentation, including a programming -guide, on the project webpage at . -This README file only contains basic setup instructions. - - -## Building - -Spark requires Scala 2.9.2 (Scala 2.10 is not yet supported). The project is -built using Simple Build Tool (SBT), which is packaged with it. To build -Spark and its example programs, run: - - sbt/sbt package - -Spark also supports building using Maven. If you would like to build using Maven, -see the [instructions for building Spark with Maven](http://spark-project.org/docs/latest/building-with-maven.html) -in the spark documentation.. - -To run Spark, you will need to have Scala's bin directory in your `PATH`, or -you will need to set the `SCALA_HOME` environment variable to point to where -you've installed Scala. Scala must be accessible through one of these -methods on your cluster's worker nodes as well as its master. - -To run one of the examples, use `./run `. For example: - - ./run spark.examples.SparkLR local[2] - -will run the Logistic Regression example locally on 2 CPUs. - -Each of the example programs prints usage help if no params are given. - -All of the Spark samples take a `` parameter that is the cluster URL -to connect to. This can be a mesos:// or spark:// URL, or "local" to run -locally with one thread, or "local[N]" to run locally with N threads. - - -## A Note About Hadoop Versions - -Spark uses the Hadoop core library to talk to HDFS and other Hadoop-supported -storage systems. Because the HDFS API has changed in different versions of -Hadoop, you must build Spark against the same version that your cluster runs. -You can change the version by setting the `HADOOP_VERSION` variable at the top -of `project/SparkBuild.scala`, then rebuilding Spark. - - -## Configuration - -Please refer to the "Configuration" guide in the online documentation for a -full overview on how to configure Spark. At the minimum, you will need to -create a `conf/spark-env.sh` script (copy `conf/spark-env.sh.template`) and -set the following two variables: - -- `SCALA_HOME`: Location where Scala is installed. - -- `MESOS_NATIVE_LIBRARY`: Your Mesos library (only needed if you want to run - on Mesos). For example, this might be `/usr/local/lib/libmesos.so` on Linux. - - -## Contributing to Spark - -Contributions via GitHub pull requests are gladly accepted from their original -author. Along with any pull requests, please state that the contribution is -your original work and that you license the work to the project under the -project's open source license. Whether or not you state this explicitly, by -submitting any copyrighted material via pull request, email, or other means -you agree to license the material under the project's open source license and -warrant that you have the legal authority to do so. +This is a preview of GraphX that we are actively working.... From 55696e258456798d73325655428899c5b4931730 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Tue, 17 Sep 2013 22:42:12 -0700 Subject: [PATCH 069/531] GraphX now builds with all merged changes. --- .../apache}/spark/graph/Analytics.scala | 4 +- .../{ => org/apache}/spark/graph/Edge.scala | 2 +- .../apache}/spark/graph/EdgeDirection.scala | 2 +- .../apache}/spark/graph/EdgeTriplet.scala | 2 +- .../{ => org/apache}/spark/graph/Graph.scala | 9 ++- .../spark/graph/GraphKryoRegistrator.scala | 4 +- .../apache}/spark/graph/GraphLab.scala | 4 +- .../apache}/spark/graph/GraphLoader.scala | 10 +-- .../apache}/spark/graph/GraphOps.scala | 4 +- .../{ => org/apache}/spark/graph/Pregel.scala | 4 +- .../{ => org/apache}/spark/graph/Vertex.scala | 2 +- .../spark/graph/impl/EdgePartition.scala | 4 +- .../spark/graph/impl/EdgeTripletRDD.scala | 18 +++-- .../apache}/spark/graph/impl/GraphImpl.scala | 14 ++-- .../apache}/spark/graph/package.scala | 2 +- .../apache/spark/graph/perf/BagelTest.scala | 76 +++++++++++++++++++ .../apache/spark/graph/perf/SparkTest.scala | 75 ++++++++++++++++++ .../spark/graph/util/BytecodeUtils.scala | 5 +- .../apache}/spark/graph/util/HashUtils.scala | 2 +- .../scala/spark/graph/perf/BagelTest.scala | 72 ------------------ .../scala/spark/graph/perf/SparkTest.scala | 72 ------------------ project/SparkBuild.scala | 16 ++-- 22 files changed, 211 insertions(+), 192 deletions(-) rename graph/src/main/scala/{ => org/apache}/spark/graph/Analytics.scala (99%) rename graph/src/main/scala/{ => org/apache}/spark/graph/Edge.scala (91%) rename graph/src/main/scala/{ => org/apache}/spark/graph/EdgeDirection.scala (95%) rename graph/src/main/scala/{ => org/apache}/spark/graph/EdgeTriplet.scala (98%) rename graph/src/main/scala/{ => org/apache}/spark/graph/Graph.scala (99%) rename graph/src/main/scala/{ => org/apache}/spark/graph/GraphKryoRegistrator.scala (87%) rename graph/src/main/scala/{ => org/apache}/spark/graph/GraphLab.scala (98%) rename graph/src/main/scala/{ => org/apache}/spark/graph/GraphLoader.scala (87%) rename graph/src/main/scala/{ => org/apache}/spark/graph/GraphOps.scala (92%) rename graph/src/main/scala/{ => org/apache}/spark/graph/Pregel.scala (92%) rename graph/src/main/scala/{ => org/apache}/spark/graph/Vertex.scala (91%) rename graph/src/main/scala/{ => org/apache}/spark/graph/impl/EdgePartition.scala (94%) rename graph/src/main/scala/{ => org/apache}/spark/graph/impl/EdgeTripletRDD.scala (86%) rename graph/src/main/scala/{ => org/apache}/spark/graph/impl/GraphImpl.scala (98%) rename graph/src/main/scala/{ => org/apache}/spark/graph/package.scala (96%) create mode 100644 graph/src/main/scala/org/apache/spark/graph/perf/BagelTest.scala create mode 100644 graph/src/main/scala/org/apache/spark/graph/perf/SparkTest.scala rename graph/src/main/scala/{ => org/apache}/spark/graph/util/BytecodeUtils.scala (97%) rename graph/src/main/scala/{ => org/apache}/spark/graph/util/HashUtils.scala (91%) delete mode 100644 graph/src/main/scala/spark/graph/perf/BagelTest.scala delete mode 100644 graph/src/main/scala/spark/graph/perf/SparkTest.scala diff --git a/graph/src/main/scala/spark/graph/Analytics.scala b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala similarity index 99% rename from graph/src/main/scala/spark/graph/Analytics.scala rename to graph/src/main/scala/org/apache/spark/graph/Analytics.scala index 601a0785e1b2e..09cf81eeeb78b 100644 --- a/graph/src/main/scala/spark/graph/Analytics.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala @@ -1,6 +1,6 @@ -package spark.graph +package org.apache.spark.graph -import spark._ +import org.apache.spark._ diff --git a/graph/src/main/scala/spark/graph/Edge.scala b/graph/src/main/scala/org/apache/spark/graph/Edge.scala similarity index 91% rename from graph/src/main/scala/spark/graph/Edge.scala rename to graph/src/main/scala/org/apache/spark/graph/Edge.scala index cb057a467a475..20539b8af05b8 100644 --- a/graph/src/main/scala/spark/graph/Edge.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Edge.scala @@ -1,4 +1,4 @@ -package spark.graph +package org.apache.spark.graph /** diff --git a/graph/src/main/scala/spark/graph/EdgeDirection.scala b/graph/src/main/scala/org/apache/spark/graph/EdgeDirection.scala similarity index 95% rename from graph/src/main/scala/spark/graph/EdgeDirection.scala rename to graph/src/main/scala/org/apache/spark/graph/EdgeDirection.scala index 38caac44d6f82..99af2d54580bf 100644 --- a/graph/src/main/scala/spark/graph/EdgeDirection.scala +++ b/graph/src/main/scala/org/apache/spark/graph/EdgeDirection.scala @@ -1,4 +1,4 @@ -package spark.graph +package org.apache.spark.graph /** diff --git a/graph/src/main/scala/spark/graph/EdgeTriplet.scala b/graph/src/main/scala/org/apache/spark/graph/EdgeTriplet.scala similarity index 98% rename from graph/src/main/scala/spark/graph/EdgeTriplet.scala rename to graph/src/main/scala/org/apache/spark/graph/EdgeTriplet.scala index 3ed8052794af8..4ade1d7333d81 100644 --- a/graph/src/main/scala/spark/graph/EdgeTriplet.scala +++ b/graph/src/main/scala/org/apache/spark/graph/EdgeTriplet.scala @@ -1,4 +1,4 @@ -package spark.graph +package org.apache.spark.graph /** * An edge triplet represents two vertices and edge along with their attributes. diff --git a/graph/src/main/scala/spark/graph/Graph.scala b/graph/src/main/scala/org/apache/spark/graph/Graph.scala similarity index 99% rename from graph/src/main/scala/spark/graph/Graph.scala rename to graph/src/main/scala/org/apache/spark/graph/Graph.scala index 594b3b54955c6..1fb22c56ff7ff 100644 --- a/graph/src/main/scala/spark/graph/Graph.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Graph.scala @@ -1,6 +1,7 @@ -package spark.graph +package org.apache.spark.graph -import spark.RDD + +import org.apache.spark.rdd.RDD @@ -366,8 +367,8 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { object Graph { - import spark.graph.impl._ - import spark.SparkContext._ + import org.apache.spark.graph.impl._ + import org.apache.spark.SparkContext._ def apply(rawEdges: RDD[(Vid, Vid)], uniqueEdges: Boolean = true): Graph[Int, Int] = { // Reduce to unique edges. diff --git a/graph/src/main/scala/spark/graph/GraphKryoRegistrator.scala b/graph/src/main/scala/org/apache/spark/graph/GraphKryoRegistrator.scala similarity index 87% rename from graph/src/main/scala/spark/graph/GraphKryoRegistrator.scala rename to graph/src/main/scala/org/apache/spark/graph/GraphKryoRegistrator.scala index e1cb77f1141ac..13a22f9051e0d 100644 --- a/graph/src/main/scala/spark/graph/GraphKryoRegistrator.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphKryoRegistrator.scala @@ -1,8 +1,8 @@ -package spark.graph +package org.apache.spark.graph import com.esotericsoftware.kryo.Kryo -import spark.KryoRegistrator +import org.apache.spark.serializer.KryoRegistrator class GraphKryoRegistrator extends KryoRegistrator { diff --git a/graph/src/main/scala/spark/graph/GraphLab.scala b/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala similarity index 98% rename from graph/src/main/scala/spark/graph/GraphLab.scala rename to graph/src/main/scala/org/apache/spark/graph/GraphLab.scala index f89c2a39d79e0..1dba813e91cd8 100644 --- a/graph/src/main/scala/spark/graph/GraphLab.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala @@ -1,7 +1,7 @@ -package spark.graph +package org.apache.spark.graph import scala.collection.JavaConversions._ -import spark.RDD +import org.apache.spark.rdd.RDD /** * This object implement the graphlab gather-apply-scatter api. diff --git a/graph/src/main/scala/spark/graph/GraphLoader.scala b/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala similarity index 87% rename from graph/src/main/scala/spark/graph/GraphLoader.scala rename to graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala index 7e1a05441364e..4d7ca1268d36e 100644 --- a/graph/src/main/scala/spark/graph/GraphLoader.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala @@ -1,9 +1,9 @@ -package spark.graph +package org.apache.spark.graph -import spark.RDD -import spark.SparkContext -import spark.SparkContext._ -import spark.graph.impl.GraphImpl +import org.apache.spark.rdd.RDD +import org.apache.spark.SparkContext +import org.apache.spark.SparkContext._ +import org.apache.spark.graph.impl.GraphImpl object GraphLoader { diff --git a/graph/src/main/scala/spark/graph/GraphOps.scala b/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala similarity index 92% rename from graph/src/main/scala/spark/graph/GraphOps.scala rename to graph/src/main/scala/org/apache/spark/graph/GraphOps.scala index d98cd8d44cbb2..8de96680b8086 100644 --- a/graph/src/main/scala/spark/graph/GraphOps.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala @@ -1,6 +1,6 @@ -package spark.graph +package org.apache.spark.graph -import spark.RDD +import org.apache.spark.rdd.RDD class GraphOps[VD: ClassManifest, ED: ClassManifest](g: Graph[VD, ED]) { diff --git a/graph/src/main/scala/spark/graph/Pregel.scala b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala similarity index 92% rename from graph/src/main/scala/spark/graph/Pregel.scala rename to graph/src/main/scala/org/apache/spark/graph/Pregel.scala index 0a564b8041f4d..27b75a7988013 100644 --- a/graph/src/main/scala/spark/graph/Pregel.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala @@ -1,6 +1,6 @@ -package spark.graph +package org.apache.spark.graph -import spark.RDD +import org.apache.spark.rdd.RDD object Pregel { diff --git a/graph/src/main/scala/spark/graph/Vertex.scala b/graph/src/main/scala/org/apache/spark/graph/Vertex.scala similarity index 91% rename from graph/src/main/scala/spark/graph/Vertex.scala rename to graph/src/main/scala/org/apache/spark/graph/Vertex.scala index 32653571f79a4..c8671b7f13024 100644 --- a/graph/src/main/scala/spark/graph/Vertex.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Vertex.scala @@ -1,4 +1,4 @@ -package spark.graph +package org.apache.spark.graph /** * A graph vertex consists of a vertex id and attribute. diff --git a/graph/src/main/scala/spark/graph/impl/EdgePartition.scala b/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala similarity index 94% rename from graph/src/main/scala/spark/graph/impl/EdgePartition.scala rename to graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala index 4e0d5f41b9060..3d218f27b1850 100644 --- a/graph/src/main/scala/spark/graph/impl/EdgePartition.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala @@ -1,10 +1,10 @@ -package spark.graph.impl +package org.apache.spark.graph.impl import scala.collection.mutable.ArrayBuilder import it.unimi.dsi.fastutil.ints.IntArrayList -import spark.graph._ +import org.apache.spark.graph._ /** diff --git a/graph/src/main/scala/spark/graph/impl/EdgeTripletRDD.scala b/graph/src/main/scala/org/apache/spark/graph/impl/EdgeTripletRDD.scala similarity index 86% rename from graph/src/main/scala/spark/graph/impl/EdgeTripletRDD.scala rename to graph/src/main/scala/org/apache/spark/graph/impl/EdgeTripletRDD.scala index f6de8e59af0fd..18d5d2b5aae70 100644 --- a/graph/src/main/scala/spark/graph/impl/EdgeTripletRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/EdgeTripletRDD.scala @@ -1,9 +1,15 @@ -package spark.graph.impl - -import spark.{Aggregator, HashPartitioner, Partition, RDD, SparkEnv, TaskContext} -import spark.{Dependency, OneToOneDependency, ShuffleDependency} -import spark.SparkContext._ -import spark.graph._ +package org.apache.spark.graph.impl + +import org.apache.spark.Aggregator +import org.apache.spark.Partition +import org.apache.spark.SparkEnv +import org.apache.spark.TaskContext +import org.apache.spark.rdd.RDD +import org.apache.spark.Dependency +import org.apache.spark.OneToOneDependency +import org.apache.spark.ShuffleDependency +import org.apache.spark.SparkContext._ +import org.apache.spark.graph._ private[graph] diff --git a/graph/src/main/scala/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala similarity index 98% rename from graph/src/main/scala/spark/graph/impl/GraphImpl.scala rename to graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index 08fc016a4300b..68ac9f724c3bb 100644 --- a/graph/src/main/scala/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -1,12 +1,16 @@ -package spark.graph.impl +package org.apache.spark.graph.impl import scala.collection.JavaConversions._ -import spark.{ClosureCleaner, Partitioner, HashPartitioner, RDD} -import spark.SparkContext._ +import org.apache.spark.SparkContext._ +import org.apache.spark.Partitioner +import org.apache.spark.HashPartitioner +import org.apache.spark.util.ClosureCleaner -import spark.graph._ -import spark.graph.impl.GraphImpl._ +import org.apache.spark.rdd.RDD + +import org.apache.spark.graph._ +import org.apache.spark.graph.impl.GraphImpl._ diff --git a/graph/src/main/scala/spark/graph/package.scala b/graph/src/main/scala/org/apache/spark/graph/package.scala similarity index 96% rename from graph/src/main/scala/spark/graph/package.scala rename to graph/src/main/scala/org/apache/spark/graph/package.scala index d95dcdce0848c..474ace520f440 100644 --- a/graph/src/main/scala/spark/graph/package.scala +++ b/graph/src/main/scala/org/apache/spark/graph/package.scala @@ -1,4 +1,4 @@ -package spark +package org.apache.spark package object graph { diff --git a/graph/src/main/scala/org/apache/spark/graph/perf/BagelTest.scala b/graph/src/main/scala/org/apache/spark/graph/perf/BagelTest.scala new file mode 100644 index 0000000000000..eaff27a33eda5 --- /dev/null +++ b/graph/src/main/scala/org/apache/spark/graph/perf/BagelTest.scala @@ -0,0 +1,76 @@ +///// This file creates circular dependencies between examples bagle and graph + +// package org.apache.spark.graph.perf + +// import org.apache.spark._ +// import org.apache.spark.SparkContext._ +// import org.apache.spark.bagel.Bagel + +// import org.apache.spark.examples.bagel +// //import org.apache.spark.bagel.examples._ +// import org.apache.spark.graph._ + + +// object BagelTest { + +// def main(args: Array[String]) { +// val host = args(0) +// val taskType = args(1) +// val fname = args(2) +// val options = args.drop(3).map { arg => +// arg.dropWhile(_ == '-').split('=') match { +// case Array(opt, v) => (opt -> v) +// case _ => throw new IllegalArgumentException("Invalid argument: " + arg) +// } +// } + +// System.setProperty("spark.serializer", "org.apache.spark.serializer.KryoSerializer") +// //System.setProperty("spark.shuffle.compress", "false") +// System.setProperty("spark.kryo.registrator", "org.apache.spark.bagel.examples.PRKryoRegistrator") + +// var numIter = Int.MaxValue +// var isDynamic = false +// var tol:Float = 0.001F +// var outFname = "" +// var numVPart = 4 +// var numEPart = 4 + +// options.foreach{ +// case ("numIter", v) => numIter = v.toInt +// case ("dynamic", v) => isDynamic = v.toBoolean +// case ("tol", v) => tol = v.toFloat +// case ("output", v) => outFname = v +// case ("numVPart", v) => numVPart = v.toInt +// case ("numEPart", v) => numEPart = v.toInt +// case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) +// } + +// val sc = new SparkContext(host, "PageRank(" + fname + ")") +// val g = GraphLoader.textFile(sc, fname, a => 1.0F).withPartitioner(numVPart, numEPart).cache() +// val startTime = System.currentTimeMillis + +// val numVertices = g.vertices.count() + +// val vertices = g.collectNeighborIds(EdgeDirection.Out).map { case (vid, neighbors) => +// (vid.toString, new PRVertex(1.0, neighbors.map(_.toString))) +// } + +// // Do the computation +// val epsilon = 0.01 / numVertices +// val messages = sc.parallelize(Array[(String, PRMessage)]()) +// val utils = new PageRankUtils +// val result = +// Bagel.run( +// sc, vertices, messages, combiner = new PRCombiner(), +// numPartitions = numVPart)( +// utils.computeWithCombiner(numVertices, epsilon, numIter)) + +// println("Total rank: " + result.map{ case (id, r) => r.value }.reduce(_+_) ) +// if (!outFname.isEmpty) { +// println("Saving pageranks of pages to " + outFname) +// result.map{ case (id, r) => id + "\t" + r.value }.saveAsTextFile(outFname) +// } +// println("Runtime: " + ((System.currentTimeMillis - startTime)/1000.0) + " seconds") +// sc.stop() +// } +// } diff --git a/graph/src/main/scala/org/apache/spark/graph/perf/SparkTest.scala b/graph/src/main/scala/org/apache/spark/graph/perf/SparkTest.scala new file mode 100644 index 0000000000000..01bd96855082c --- /dev/null +++ b/graph/src/main/scala/org/apache/spark/graph/perf/SparkTest.scala @@ -0,0 +1,75 @@ +///// This file creates circular dependencies between examples bagle and graph + + +// package org.apache.spark.graph.perf + +// import org.apache.spark._ +// import org.apache.spark.SparkContext._ +// import org.apache.spark.bagel.Bagel +// import org.apache.spark.bagel.examples._ +// import org.apache.spark.graph._ + + +// object SparkTest { + +// def main(args: Array[String]) { +// val host = args(0) +// val taskType = args(1) +// val fname = args(2) +// val options = args.drop(3).map { arg => +// arg.dropWhile(_ == '-').split('=') match { +// case Array(opt, v) => (opt -> v) +// case _ => throw new IllegalArgumentException("Invalid argument: " + arg) +// } +// } + +// System.setProperty("spark.serializer", "org.apache.spark.KryoSerializer") +// //System.setProperty("spark.shuffle.compress", "false") +// System.setProperty("spark.kryo.registrator", "spark.bagel.examples.PRKryoRegistrator") + +// var numIter = Int.MaxValue +// var isDynamic = false +// var tol:Float = 0.001F +// var outFname = "" +// var numVPart = 4 +// var numEPart = 4 + +// options.foreach{ +// case ("numIter", v) => numIter = v.toInt +// case ("dynamic", v) => isDynamic = v.toBoolean +// case ("tol", v) => tol = v.toFloat +// case ("output", v) => outFname = v +// case ("numVPart", v) => numVPart = v.toInt +// case ("numEPart", v) => numEPart = v.toInt +// case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) +// } + +// val sc = new SparkContext(host, "PageRank(" + fname + ")") +// val g = GraphLoader.textFile(sc, fname, a => 1.0F).withPartitioner(numVPart, numEPart).cache() +// val startTime = System.currentTimeMillis + +// val numVertices = g.vertices.count() + +// val vertices = g.collectNeighborIds(EdgeDirection.Out).map { case (vid, neighbors) => +// (vid.toString, new PRVertex(1.0, neighbors.map(_.toString))) +// } + +// // Do the computation +// val epsilon = 0.01 / numVertices +// val messages = sc.parallelize(Array[(String, PRMessage)]()) +// val utils = new PageRankUtils +// val result = +// Bagel.run( +// sc, vertices, messages, combiner = new PRCombiner(), +// numPartitions = numVPart)( +// utils.computeWithCombiner(numVertices, epsilon, numIter)) + +// println("Total rank: " + result.map{ case (id, r) => r.value }.reduce(_+_) ) +// if (!outFname.isEmpty) { +// println("Saving pageranks of pages to " + outFname) +// result.map{ case (id, r) => id + "\t" + r.value }.saveAsTextFile(outFname) +// } +// println("Runtime: " + ((System.currentTimeMillis - startTime)/1000.0) + " seconds") +// sc.stop() +// } +// } diff --git a/graph/src/main/scala/spark/graph/util/BytecodeUtils.scala b/graph/src/main/scala/org/apache/spark/graph/util/BytecodeUtils.scala similarity index 97% rename from graph/src/main/scala/spark/graph/util/BytecodeUtils.scala rename to graph/src/main/scala/org/apache/spark/graph/util/BytecodeUtils.scala index ac3a1fb9573fa..5db13fe3bc587 100644 --- a/graph/src/main/scala/spark/graph/util/BytecodeUtils.scala +++ b/graph/src/main/scala/org/apache/spark/graph/util/BytecodeUtils.scala @@ -1,4 +1,4 @@ -package spark.graph.util +package org.apache.spark.util import java.io.{ByteArrayInputStream, ByteArrayOutputStream} @@ -7,10 +7,9 @@ import scala.collection.mutable.HashSet import org.objectweb.asm.{ClassReader, ClassVisitor, MethodVisitor} import org.objectweb.asm.Opcodes._ -import spark.Utils -private[graph] object BytecodeUtils { +private[spark] object BytecodeUtils { /** * Test whether the given closure invokes the specified method in the specified class. diff --git a/graph/src/main/scala/spark/graph/util/HashUtils.scala b/graph/src/main/scala/org/apache/spark/graph/util/HashUtils.scala similarity index 91% rename from graph/src/main/scala/spark/graph/util/HashUtils.scala rename to graph/src/main/scala/org/apache/spark/graph/util/HashUtils.scala index 0dfaef4c48dda..cb18ef3d26bae 100644 --- a/graph/src/main/scala/spark/graph/util/HashUtils.scala +++ b/graph/src/main/scala/org/apache/spark/graph/util/HashUtils.scala @@ -1,4 +1,4 @@ -package spark.graph.util +package org.apache.spark.graph.util object HashUtils { diff --git a/graph/src/main/scala/spark/graph/perf/BagelTest.scala b/graph/src/main/scala/spark/graph/perf/BagelTest.scala deleted file mode 100644 index 7547292500d6c..0000000000000 --- a/graph/src/main/scala/spark/graph/perf/BagelTest.scala +++ /dev/null @@ -1,72 +0,0 @@ -package spark.graph.perf - -import spark._ -import spark.SparkContext._ -import spark.bagel.Bagel -import spark.bagel.examples._ -import spark.graph._ - - -object BagelTest { - - def main(args: Array[String]) { - val host = args(0) - val taskType = args(1) - val fname = args(2) - val options = args.drop(3).map { arg => - arg.dropWhile(_ == '-').split('=') match { - case Array(opt, v) => (opt -> v) - case _ => throw new IllegalArgumentException("Invalid argument: " + arg) - } - } - - System.setProperty("spark.serializer", "spark.KryoSerializer") - //System.setProperty("spark.shuffle.compress", "false") - System.setProperty("spark.kryo.registrator", "spark.bagel.examples.PRKryoRegistrator") - - var numIter = Int.MaxValue - var isDynamic = false - var tol:Float = 0.001F - var outFname = "" - var numVPart = 4 - var numEPart = 4 - - options.foreach{ - case ("numIter", v) => numIter = v.toInt - case ("dynamic", v) => isDynamic = v.toBoolean - case ("tol", v) => tol = v.toFloat - case ("output", v) => outFname = v - case ("numVPart", v) => numVPart = v.toInt - case ("numEPart", v) => numEPart = v.toInt - case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) - } - - val sc = new SparkContext(host, "PageRank(" + fname + ")") - val g = GraphLoader.textFile(sc, fname, a => 1.0F).withPartitioner(numVPart, numEPart).cache() - val startTime = System.currentTimeMillis - - val numVertices = g.vertices.count() - - val vertices = g.collectNeighborIds(EdgeDirection.Out).map { case (vid, neighbors) => - (vid.toString, new PRVertex(1.0, neighbors.map(_.toString))) - } - - // Do the computation - val epsilon = 0.01 / numVertices - val messages = sc.parallelize(Array[(String, PRMessage)]()) - val utils = new PageRankUtils - val result = - Bagel.run( - sc, vertices, messages, combiner = new PRCombiner(), - numPartitions = numVPart)( - utils.computeWithCombiner(numVertices, epsilon, numIter)) - - println("Total rank: " + result.map{ case (id, r) => r.value }.reduce(_+_) ) - if (!outFname.isEmpty) { - println("Saving pageranks of pages to " + outFname) - result.map{ case (id, r) => id + "\t" + r.value }.saveAsTextFile(outFname) - } - println("Runtime: " + ((System.currentTimeMillis - startTime)/1000.0) + " seconds") - sc.stop() - } -} diff --git a/graph/src/main/scala/spark/graph/perf/SparkTest.scala b/graph/src/main/scala/spark/graph/perf/SparkTest.scala deleted file mode 100644 index 85ebd14bcb63c..0000000000000 --- a/graph/src/main/scala/spark/graph/perf/SparkTest.scala +++ /dev/null @@ -1,72 +0,0 @@ -package spark.graph.perf - -import spark._ -import spark.SparkContext._ -import spark.bagel.Bagel -import spark.bagel.examples._ -import spark.graph._ - - -object SparkTest { - - def main(args: Array[String]) { - val host = args(0) - val taskType = args(1) - val fname = args(2) - val options = args.drop(3).map { arg => - arg.dropWhile(_ == '-').split('=') match { - case Array(opt, v) => (opt -> v) - case _ => throw new IllegalArgumentException("Invalid argument: " + arg) - } - } - - System.setProperty("spark.serializer", "spark.KryoSerializer") - //System.setProperty("spark.shuffle.compress", "false") - System.setProperty("spark.kryo.registrator", "spark.bagel.examples.PRKryoRegistrator") - - var numIter = Int.MaxValue - var isDynamic = false - var tol:Float = 0.001F - var outFname = "" - var numVPart = 4 - var numEPart = 4 - - options.foreach{ - case ("numIter", v) => numIter = v.toInt - case ("dynamic", v) => isDynamic = v.toBoolean - case ("tol", v) => tol = v.toFloat - case ("output", v) => outFname = v - case ("numVPart", v) => numVPart = v.toInt - case ("numEPart", v) => numEPart = v.toInt - case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) - } - - val sc = new SparkContext(host, "PageRank(" + fname + ")") - val g = GraphLoader.textFile(sc, fname, a => 1.0F).withPartitioner(numVPart, numEPart).cache() - val startTime = System.currentTimeMillis - - val numVertices = g.vertices.count() - - val vertices = g.collectNeighborIds(EdgeDirection.Out).map { case (vid, neighbors) => - (vid.toString, new PRVertex(1.0, neighbors.map(_.toString))) - } - - // Do the computation - val epsilon = 0.01 / numVertices - val messages = sc.parallelize(Array[(String, PRMessage)]()) - val utils = new PageRankUtils - val result = - Bagel.run( - sc, vertices, messages, combiner = new PRCombiner(), - numPartitions = numVPart)( - utils.computeWithCombiner(numVertices, epsilon, numIter)) - - println("Total rank: " + result.map{ case (id, r) => r.value }.reduce(_+_) ) - if (!outFname.isEmpty) { - println("Saving pageranks of pages to " + outFname) - result.map{ case (id, r) => id + "\t" + r.value }.saveAsTextFile(outFname) - } - println("Runtime: " + ((System.currentTimeMillis - startTime)/1000.0) + " seconds") - sc.stop() - } -} diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 7598060cb9857..7dc6c58401477 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -42,16 +42,16 @@ object SparkBuild extends Build { lazy val core = Project("core", file("core"), settings = coreSettings) lazy val repl = Project("repl", file("repl"), settings = replSettings) - .dependsOn(core, bagel, mllib) + .dependsOn(core, graph, bagel, mllib) lazy val examples = Project("examples", file("examples"), settings = examplesSettings) - .dependsOn(core, mllib, bagel, streaming) + .dependsOn(core, mllib, graph, bagel, streaming) lazy val tools = Project("tools", file("tools"), settings = toolsSettings) dependsOn(core) dependsOn(streaming) lazy val bagel = Project("bagel", file("bagel"), settings = bagelSettings) dependsOn(core) - lazy val graph = Project("graph", file("graph"), settings = graphSettings) dependsOn(core) + lazy val graph = Project("graph", file("graph"), settings = graphSettings) dependsOn(core) lazy val streaming = Project("streaming", file("streaming"), settings = streamingSettings) dependsOn(core) @@ -60,7 +60,7 @@ object SparkBuild extends Build { lazy val yarn = Project("yarn", file("yarn"), settings = yarnSettings) dependsOn(core) lazy val assemblyProj = Project("assembly", file("assembly"), settings = assemblyProjSettings) - .dependsOn(core, bagel, mllib, repl, streaming) dependsOn(maybeYarn: _*) + .dependsOn(core, graph, bagel, mllib, repl, streaming) dependsOn(maybeYarn: _*) // A configuration to set an alternative publishLocalConfiguration lazy val MavenCompile = config("m2r") extend(Compile) @@ -77,7 +77,7 @@ object SparkBuild extends Build { lazy val maybeYarn = if(isYarnEnabled) Seq[ClasspathDependency](yarn) else Seq[ClasspathDependency]() lazy val maybeYarnRef = if(isYarnEnabled) Seq[ProjectReference](yarn) else Seq[ProjectReference]() lazy val allProjects = Seq[ProjectReference]( - core, repl, examples, bagel, streaming, mllib, tools, assemblyProj) ++ maybeYarnRef + core, repl, examples, graph, bagel, streaming, mllib, tools, assemblyProj) ++ maybeYarnRef def sharedSettings = Defaults.defaultSettings ++ Seq( organization := "org.apache.spark", @@ -254,6 +254,10 @@ object SparkBuild extends Build { name := "spark-tools" ) + def graphSettings = sharedSettings ++ Seq( + name := "spark-graphx" + ) + def bagelSettings = sharedSettings ++ Seq( name := "spark-bagel" ) @@ -265,8 +269,6 @@ object SparkBuild extends Build { ) ) - def graphSettings = sharedSettings ++ Seq(name := "spark-graph") - def streamingSettings = sharedSettings ++ Seq( name := "spark-streaming", resolvers ++= Seq( From 9ff783599b908bb1244848d2bf6fb0d402d69008 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Mon, 2 Sep 2013 20:52:03 -0700 Subject: [PATCH 070/531] Return Graph from aggregateNeighbors; update callers This commit only affects the Graph API, not GraphImpl. --- .../scala/org/apache/spark/graph/Graph.scala | 10 +++--- .../org/apache/spark/graph/GraphLab.scala | 32 ++++++++++++------- .../org/apache/spark/graph/GraphOps.scala | 27 ++++++++++------ .../scala/org/apache/spark/graph/Pregel.scala | 21 ++++++++---- 4 files changed, 56 insertions(+), 34 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/Graph.scala b/graph/src/main/scala/org/apache/spark/graph/Graph.scala index 1fb22c56ff7ff..b5c4fcc99b164 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Graph.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Graph.scala @@ -227,12 +227,12 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * }}} * */ - def aggregateNeighbors[VD2: ClassManifest]( - mapFunc: (Vid, EdgeTriplet[VD, ED]) => Option[VD2], - mergeFunc: (VD2, VD2) => VD2, + def aggregateNeighbors[A: ClassManifest]( + mapFunc: (Vid, EdgeTriplet[VD, ED]) => Option[A], + mergeFunc: (A, A) => A, direction: EdgeDirection) - : RDD[(Vid, VD2)] - + : Graph[(VD, Option[A]), ED] + // TODO: consider a version that doesn't preserve the original VD /** * This function is used to compute a statistic for the neighborhood of each diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala b/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala index 1dba813e91cd8..01f24a13024c7 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala @@ -44,13 +44,13 @@ object GraphLab { // Add an active attribute to all vertices to track convergence. - var activeGraph = graph.mapVertices { + var activeGraph: Graph[(Boolean, VD), ED] = graph.mapVertices { case Vertex(id, data) => (true, data) }.cache() // The gather function wrapper strips the active attribute and // only invokes the gather function on active vertices - def gather(vid: Vid, e: EdgeTriplet[(Boolean, VD), ED]) = { + def gather(vid: Vid, e: EdgeTriplet[(Boolean, VD), ED]): Option[A] = { if (e.vertex(vid).data._1) { val edge = new EdgeTriplet[VD,ED] edge.src = Vertex(e.src.id, e.src.data._2) @@ -64,14 +64,15 @@ object GraphLab { // The apply function wrapper strips the vertex of the active attribute // and only invokes the apply function on active vertices - def apply(v: Vertex[(Boolean, VD)], accum: Option[A]) = { - if (v.data._1) (true, applyFunc(Vertex(v.id, v.data._2), accum)) - else (false, v.data._2) + def apply(v: Vertex[((Boolean, VD), Option[A])]): (Boolean, VD) = { + val ((active, vData), accum) = v.data + if (active) (true, applyFunc(Vertex(v.id, vData), accum)) + else (false, vData) } // The scatter function wrapper strips the vertex of the active attribute // and only invokes the scatter function on active vertices - def scatter(rawVid: Vid, e: EdgeTriplet[(Boolean, VD), ED]) = { + def scatter(rawVid: Vid, e: EdgeTriplet[(Boolean, VD), ED]): Option[Boolean] = { val vid = e.otherVertex(rawVid).id if (e.vertex(vid).data._1) { val edge = new EdgeTriplet[VD,ED] @@ -88,24 +89,31 @@ object GraphLab { } // Used to set the active status of vertices for the next round - def applyActive(v: Vertex[(Boolean, VD)], accum: Option[Boolean]) = - (accum.getOrElse(false), v.data._2) + def applyActive(v: Vertex[((Boolean, VD), Option[Boolean])]): (Boolean, VD) = { + val ((prevActive, vData), newActive) = v.data + (newActive.getOrElse(false), vData) + } // Main Loop --------------------------------------------------------------------- var i = 0 var numActive = activeGraph.numVertices while (i < numIter && numActive > 0) { - val accUpdates: RDD[(Vid, A)] = + val gathered: Graph[((Boolean, VD), Option[A]), ED] = activeGraph.aggregateNeighbors(gather, mergeFunc, gatherDirection) - activeGraph = activeGraph.leftJoinVertices(accUpdates, apply).cache() + val applied: Graph[(Boolean, VD), ED] = gathered.mapVertices(apply).cache() + + activeGraph = applied.cache() // Scatter is basically a gather in the opposite direction so we reverse the edge direction - val activeVertices: RDD[(Vid, Boolean)] = + // activeGraph: Graph[(Boolean, VD), ED] + val scattered: Graph[((Boolean, VD), Option[Boolean]), ED] = activeGraph.aggregateNeighbors(scatter, _ || _, scatterDirection.reverse) + val newActiveGraph: Graph[(Boolean, VD), ED] = + scattered.mapVertices(applyActive) - activeGraph = activeGraph.leftJoinVertices(activeVertices, applyActive).cache() + activeGraph = newActiveGraph.cache() numActive = activeGraph.vertices.map(v => if (v.data._1) 1 else 0).reduce(_ + _) println("Number active vertices: " + numActive) diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala b/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala index 8de96680b8086..9e8cc0a6d52a7 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala @@ -9,22 +9,29 @@ class GraphOps[VD: ClassManifest, ED: ClassManifest](g: Graph[VD, ED]) { lazy val numVertices: Long = g.vertices.count() - lazy val inDegrees: RDD[(Vid, Int)] = { - g.aggregateNeighbors((vid, edge) => Some(1), _+_, EdgeDirection.In) - } + lazy val inDegrees: RDD[(Vid, Int)] = degreesRDD(EdgeDirection.In) - lazy val outDegrees: RDD[(Vid, Int)] = { - g.aggregateNeighbors((vid, edge) => Some(1), _+_, EdgeDirection.Out) - } + lazy val outDegrees: RDD[(Vid, Int)] = degreesRDD(EdgeDirection.Out) - lazy val degrees: RDD[(Vid, Int)] = { - g.aggregateNeighbors((vid, edge) => Some(1), _+_, EdgeDirection.Both) - } + lazy val degrees: RDD[(Vid, Int)] = degreesRDD(EdgeDirection.Both) def collectNeighborIds(edgeDirection: EdgeDirection) : RDD[(Vid, Array[Vid])] = { - g.aggregateNeighbors( + val graph: Graph[(VD, Option[Array[Vid]]), ED] = g.aggregateNeighbors( (vid, edge) => Some(Array(edge.otherVertex(vid).id)), (a, b) => a ++ b, edgeDirection) + graph.vertices.map(v => { + val (_, neighborIds) = v.data + (v.id, neighborIds.getOrElse(Array())) + }) + } + + private def degreesRDD(edgeDirection: EdgeDirection): RDD[(Vid, Int)] = { + val degreeGraph: Graph[(VD, Option[Int]), ED] = + g.aggregateNeighbors((vid, edge) => Some(1), _+_, edgeDirection) + degreeGraph.vertices.map(v => { + val (_, degree) = v.data + (v.id, degree.getOrElse(0)) + }) } } diff --git a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala index 27b75a7988013..09bcc67c8ced9 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala @@ -19,18 +19,25 @@ object Pregel { def mapF(vid: Vid, edge: EdgeTriplet[VD,ED]) = sendMsg(edge.otherVertex(vid).id, edge) - def runProg(v: Vertex[VD], msg: Option[A]): VD = { - if (msg.isEmpty) v.data else vprog(v, msg.get) + def runProg(vertexWithMsgs: Vertex[(VD, Option[A])]): VD = { + val (vData, msg) = vertexWithMsgs.data + val v = Vertex(vertexWithMsgs.id, vData) + msg match { + case Some(m) => vprog(v, m) + case None => v.data + } } - var msgs: RDD[(Vid, A)] = g.vertices.map{ v => (v.id, initialMsg) } + var graphWithMsgs: Graph[(VD, Option[A]), ED] = + g.mapVertices(v => (v.data, Some(initialMsg))) while (i < numIter) { - g = g.leftJoinVertices(msgs, runProg).cache() - msgs = g.aggregateNeighbors(mapF, mergeMsg, EdgeDirection.In) + val newGraph: Graph[VD, ED] = graphWithMsgs.mapVertices(runProg).cache() + graphWithMsgs = newGraph.aggregateNeighbors(mapF, mergeMsg, EdgeDirection.In) i += 1 } - g + graphWithMsgs.mapVertices(vertexWithMsgs => vertexWithMsgs.data match { + case (vData, _) => vData + }) } - } From b04f1a40197dde5ba34845e5ac8f0862d95cb3e2 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Wed, 18 Sep 2013 16:03:27 -0700 Subject: [PATCH 071/531] Implement aggregateNeighbors returning Graph --- .../apache/spark/graph/impl/GraphImpl.scala | 81 ++++++++++--------- 1 file changed, 42 insertions(+), 39 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index 68ac9f724c3bb..8bd15e2eb95f2 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -179,55 +179,58 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( * Same as aggregateNeighbors but map function can return none and there is no default value. * As a consequence, the resulting table may be much smaller than the set of vertices. */ - override def aggregateNeighbors[VD2: ClassManifest]( - mapFunc: (Vid, EdgeTriplet[VD, ED]) => Option[VD2], - reduceFunc: (VD2, VD2) => VD2, - gatherDirection: EdgeDirection): RDD[(Vid, VD2)] = { + override def aggregateNeighbors[A: ClassManifest]( + mapFunc: (Vid, EdgeTriplet[VD, ED]) => Option[A], + reduceFunc: (A, A) => A, + gatherDirection: EdgeDirection): Graph[(VD, Option[A]), ED] = { ClosureCleaner.clean(mapFunc) ClosureCleaner.clean(reduceFunc) val newVTable = vTableReplicated.mapPartitions({ part => - part.map { v => (v._1, MutableTuple2(v._2, Option.empty[VD2])) } + part.map { v => (v._1, MutableTuple2(v._2, Option.empty[A])) } }, preservesPartitioning = true) - new EdgeTripletRDD[MutableTuple2[VD, Option[VD2]], ED](newVTable, eTable) - .mapPartitions { part => - val (vmap, edges) = part.next() - val edgeSansAcc = new EdgeTriplet[VD, ED]() - edgeSansAcc.src = new Vertex[VD] - edgeSansAcc.dst = new Vertex[VD] - edges.foreach { e: EdgeTriplet[MutableTuple2[VD, Option[VD2]], ED] => - edgeSansAcc.data = e.data - edgeSansAcc.src.data = e.src.data._1 - edgeSansAcc.dst.data = e.dst.data._1 - edgeSansAcc.src.id = e.src.id - edgeSansAcc.dst.id = e.dst.id - if (gatherDirection == EdgeDirection.In || gatherDirection == EdgeDirection.Both) { - e.dst.data._2 = - if (e.dst.data._2.isEmpty) { - mapFunc(edgeSansAcc.dst.id, edgeSansAcc) - } else { - val tmp = mapFunc(edgeSansAcc.dst.id, edgeSansAcc) - if (!tmp.isEmpty) Some(reduceFunc(e.dst.data._2.get, tmp.get)) else e.dst.data._2 - } + val newVertices: RDD[(Vid, A)] = + new EdgeTripletRDD[MutableTuple2[VD, Option[A]], ED](newVTable, eTable) + .mapPartitions { part => + val (vmap, edges) = part.next() + val edgeSansAcc = new EdgeTriplet[VD, ED]() + edgeSansAcc.src = new Vertex[VD] + edgeSansAcc.dst = new Vertex[VD] + edges.foreach { e: EdgeTriplet[MutableTuple2[VD, Option[A]], ED] => + edgeSansAcc.data = e.data + edgeSansAcc.src.data = e.src.data._1 + edgeSansAcc.dst.data = e.dst.data._1 + edgeSansAcc.src.id = e.src.id + edgeSansAcc.dst.id = e.dst.id + if (gatherDirection == EdgeDirection.In || gatherDirection == EdgeDirection.Both) { + e.dst.data._2 = + if (e.dst.data._2.isEmpty) { + mapFunc(edgeSansAcc.dst.id, edgeSansAcc) + } else { + val tmp = mapFunc(edgeSansAcc.dst.id, edgeSansAcc) + if (!tmp.isEmpty) Some(reduceFunc(e.dst.data._2.get, tmp.get)) else e.dst.data._2 + } + } + if (gatherDirection == EdgeDirection.Out || gatherDirection == EdgeDirection.Both) { + e.src.data._2 = + if (e.src.data._2.isEmpty) { + mapFunc(edgeSansAcc.src.id, edgeSansAcc) + } else { + val tmp = mapFunc(edgeSansAcc.src.id, edgeSansAcc) + if (!tmp.isEmpty) Some(reduceFunc(e.src.data._2.get, tmp.get)) else e.src.data._2 + } + } } - if (gatherDirection == EdgeDirection.Out || gatherDirection == EdgeDirection.Both) { - e.src.data._2 = - if (e.src.data._2.isEmpty) { - mapFunc(edgeSansAcc.src.id, edgeSansAcc) - } else { - val tmp = mapFunc(edgeSansAcc.src.id, edgeSansAcc) - if (!tmp.isEmpty) Some(reduceFunc(e.src.data._2.get, tmp.get)) else e.src.data._2 - } + vmap.long2ObjectEntrySet().fastIterator().filter(!_.getValue()._2.isEmpty).map{ entry => + (entry.getLongKey(), entry.getValue()._2) } } - vmap.long2ObjectEntrySet().fastIterator().filter(!_.getValue()._2.isEmpty).map{ entry => - (entry.getLongKey(), entry.getValue()._2) - } - } - .map{ case (vid, aOpt) => (vid, aOpt.get) } - .combineByKey((v: VD2) => v, reduceFunc, null, vertexPartitioner, false) + .map{ case (vid, aOpt) => (vid, aOpt.get) } + .combineByKey((v: A) => v, reduceFunc, null, vertexPartitioner, false) + + this.leftJoinVertices(newVertices, (v: Vertex[VD], a: Option[A]) => (v.data, a)) } override def leftJoinVertices[U: ClassManifest, VD2: ClassManifest]( From 4e967af6aff5605890fa77f6408c8781b7a8d9a4 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Wed, 18 Sep 2013 16:11:38 -0700 Subject: [PATCH 072/531] Return Graph from default aggregateNeighbors also --- .../scala/org/apache/spark/graph/Graph.scala | 11 ++- .../apache/spark/graph/impl/GraphImpl.scala | 83 ++++++++++--------- 2 files changed, 48 insertions(+), 46 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/Graph.scala b/graph/src/main/scala/org/apache/spark/graph/Graph.scala index b5c4fcc99b164..6f8b5143b2be1 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Graph.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Graph.scala @@ -232,7 +232,6 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { mergeFunc: (A, A) => A, direction: EdgeDirection) : Graph[(VD, Option[A]), ED] - // TODO: consider a version that doesn't preserve the original VD /** * This function is used to compute a statistic for the neighborhood of each @@ -276,12 +275,12 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * @todo Should this return a graph with the new vertex values? * */ - def aggregateNeighbors[VD2: ClassManifest]( - mapFunc: (Vid, EdgeTriplet[VD, ED]) => Option[VD2], - reduceFunc: (VD2, VD2) => VD2, - default: VD2, // Should this be a function or a value? + def aggregateNeighbors[A: ClassManifest]( + mapFunc: (Vid, EdgeTriplet[VD, ED]) => Option[A], + reduceFunc: (A, A) => A, + default: A, // Should this be a function or a value? direction: EdgeDirection) - : RDD[(Vid, VD2)] + : Graph[(VD, Option[A]), ED] /** diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index 8bd15e2eb95f2..e397293a3d63d 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -122,57 +122,60 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( // Lower level transformation methods ////////////////////////////////////////////////////////////////////////////////////////////////// - override def aggregateNeighbors[VD2: ClassManifest]( - mapFunc: (Vid, EdgeTriplet[VD, ED]) => Option[VD2], - reduceFunc: (VD2, VD2) => VD2, - default: VD2, + override def aggregateNeighbors[A: ClassManifest]( + mapFunc: (Vid, EdgeTriplet[VD, ED]) => Option[A], + reduceFunc: (A, A) => A, + default: A, gatherDirection: EdgeDirection) - : RDD[(Vid, VD2)] = { + : Graph[(VD, Option[A]), ED] = { ClosureCleaner.clean(mapFunc) ClosureCleaner.clean(reduceFunc) val newVTable = vTableReplicated.mapPartitions({ part => - part.map { v => (v._1, MutableTuple2(v._2, Option.empty[VD2])) } + part.map { v => (v._1, MutableTuple2(v._2, Option.empty[A])) } }, preservesPartitioning = true) - new EdgeTripletRDD[MutableTuple2[VD, Option[VD2]], ED](newVTable, eTable) - .mapPartitions { part => - val (vmap, edges) = part.next() - val edgeSansAcc = new EdgeTriplet[VD, ED]() - edgeSansAcc.src = new Vertex[VD] - edgeSansAcc.dst = new Vertex[VD] - edges.foreach { e: EdgeTriplet[MutableTuple2[VD, Option[VD2]], ED] => - edgeSansAcc.data = e.data - edgeSansAcc.src.data = e.src.data._1 - edgeSansAcc.dst.data = e.dst.data._1 - edgeSansAcc.src.id = e.src.id - edgeSansAcc.dst.id = e.dst.id - if (gatherDirection == EdgeDirection.In || gatherDirection == EdgeDirection.Both) { - e.dst.data._2 = - if (e.dst.data._2.isEmpty) { - mapFunc(edgeSansAcc.dst.id, edgeSansAcc) - } else { - val tmp = mapFunc(edgeSansAcc.dst.id, edgeSansAcc) - if (!tmp.isEmpty) Some(reduceFunc(e.dst.data._2.get, tmp.get)) else e.dst.data._2 - } + val newVertices: RDD[(Vid, A)] = + new EdgeTripletRDD[MutableTuple2[VD, Option[A]], ED](newVTable, eTable) + .mapPartitions { part => + val (vmap, edges) = part.next() + val edgeSansAcc = new EdgeTriplet[VD, ED]() + edgeSansAcc.src = new Vertex[VD] + edgeSansAcc.dst = new Vertex[VD] + edges.foreach { e: EdgeTriplet[MutableTuple2[VD, Option[A]], ED] => + edgeSansAcc.data = e.data + edgeSansAcc.src.data = e.src.data._1 + edgeSansAcc.dst.data = e.dst.data._1 + edgeSansAcc.src.id = e.src.id + edgeSansAcc.dst.id = e.dst.id + if (gatherDirection == EdgeDirection.In || gatherDirection == EdgeDirection.Both) { + e.dst.data._2 = + if (e.dst.data._2.isEmpty) { + mapFunc(edgeSansAcc.dst.id, edgeSansAcc) + } else { + val tmp = mapFunc(edgeSansAcc.dst.id, edgeSansAcc) + if (!tmp.isEmpty) Some(reduceFunc(e.dst.data._2.get, tmp.get)) else e.dst.data._2 + } + } + if (gatherDirection == EdgeDirection.Out || gatherDirection == EdgeDirection.Both) { + e.dst.data._2 = + if (e.dst.data._2.isEmpty) { + mapFunc(edgeSansAcc.src.id, edgeSansAcc) + } else { + val tmp = mapFunc(edgeSansAcc.src.id, edgeSansAcc) + if (!tmp.isEmpty) Some(reduceFunc(e.src.data._2.get, tmp.get)) else e.src.data._2 + } + } } - if (gatherDirection == EdgeDirection.Out || gatherDirection == EdgeDirection.Both) { - e.dst.data._2 = - if (e.dst.data._2.isEmpty) { - mapFunc(edgeSansAcc.src.id, edgeSansAcc) - } else { - val tmp = mapFunc(edgeSansAcc.src.id, edgeSansAcc) - if (!tmp.isEmpty) Some(reduceFunc(e.src.data._2.get, tmp.get)) else e.src.data._2 - } + vmap.long2ObjectEntrySet().fastIterator().filter(!_.getValue()._2.isEmpty).map{ entry => + (entry.getLongKey(), entry.getValue()._2) } } - vmap.long2ObjectEntrySet().fastIterator().filter(!_.getValue()._2.isEmpty).map{ entry => - (entry.getLongKey(), entry.getValue()._2) - } - } - .map{ case (vid, aOpt) => (vid, aOpt.get) } - .combineByKey((v: VD2) => v, reduceFunc, null, vertexPartitioner, false) + .map{ case (vid, aOpt) => (vid, aOpt.get) } + .combineByKey((v: A) => v, reduceFunc, null, vertexPartitioner, false) + + this.leftJoinVertices(newVertices, (v: Vertex[VD], a: Option[A]) => (v.data, a)) } /** From 4c694bd705e815d0c634f5023abd8bcc40055681 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Thu, 19 Sep 2013 19:13:07 -0700 Subject: [PATCH 073/531] Move IndexedRDD and GraphSuite to org.apache.spark --- .../scala/{ => org/apache}/spark/IndexedRDDFunctions.scala | 0 .../main/scala/{ => org/apache}/spark/rdd/IndexedRDD.scala | 0 .../scala/{ => org/apache}/spark/graph/GraphSuite.scala | 6 +++--- .../{ => org/apache}/spark/graph/LocalSparkContext.scala | 6 +++--- .../apache}/spark/graph/util/BytecodeUtilsSuite.scala | 2 +- 5 files changed, 7 insertions(+), 7 deletions(-) rename core/src/main/scala/{ => org/apache}/spark/IndexedRDDFunctions.scala (100%) rename core/src/main/scala/{ => org/apache}/spark/rdd/IndexedRDD.scala (100%) rename graph/src/test/scala/{ => org/apache}/spark/graph/GraphSuite.scala (95%) rename graph/src/test/scala/{ => org/apache}/spark/graph/LocalSparkContext.scala (92%) rename graph/src/test/scala/{ => org/apache}/spark/graph/util/BytecodeUtilsSuite.scala (99%) diff --git a/core/src/main/scala/spark/IndexedRDDFunctions.scala b/core/src/main/scala/org/apache/spark/IndexedRDDFunctions.scala similarity index 100% rename from core/src/main/scala/spark/IndexedRDDFunctions.scala rename to core/src/main/scala/org/apache/spark/IndexedRDDFunctions.scala diff --git a/core/src/main/scala/spark/rdd/IndexedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/IndexedRDD.scala similarity index 100% rename from core/src/main/scala/spark/rdd/IndexedRDD.scala rename to core/src/main/scala/org/apache/spark/rdd/IndexedRDD.scala diff --git a/graph/src/test/scala/spark/graph/GraphSuite.scala b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala similarity index 95% rename from graph/src/test/scala/spark/graph/GraphSuite.scala rename to graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala index 804542f1bd9cc..c9dba6144d609 100644 --- a/graph/src/test/scala/spark/graph/GraphSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala @@ -1,9 +1,9 @@ -package spark.graph +package org.apache.spark.graph import org.scalatest.FunSuite -import spark.SparkContext -import spark.graph._ +import org.apache.spark.SparkContext +import org.apache.spark.graph._ diff --git a/graph/src/test/scala/spark/graph/LocalSparkContext.scala b/graph/src/test/scala/org/apache/spark/graph/LocalSparkContext.scala similarity index 92% rename from graph/src/test/scala/spark/graph/LocalSparkContext.scala rename to graph/src/test/scala/org/apache/spark/graph/LocalSparkContext.scala index cd160cbf0c377..4a0155b6bd281 100644 --- a/graph/src/test/scala/spark/graph/LocalSparkContext.scala +++ b/graph/src/test/scala/org/apache/spark/graph/LocalSparkContext.scala @@ -1,9 +1,9 @@ -package spark.graph +package org.apache.spark.graph import org.scalatest.Suite import org.scalatest.BeforeAndAfterEach -import spark.SparkContext +import org.apache.spark.SparkContext /** Manages a local `sc` {@link SparkContext} variable, correctly stopping it after each test. */ @@ -41,4 +41,4 @@ object LocalSparkContext { } } -} \ No newline at end of file +} diff --git a/graph/src/test/scala/spark/graph/util/BytecodeUtilsSuite.scala b/graph/src/test/scala/org/apache/spark/graph/util/BytecodeUtilsSuite.scala similarity index 99% rename from graph/src/test/scala/spark/graph/util/BytecodeUtilsSuite.scala rename to graph/src/test/scala/org/apache/spark/graph/util/BytecodeUtilsSuite.scala index 8d18cf39e8919..d85e877ddfaf1 100644 --- a/graph/src/test/scala/spark/graph/util/BytecodeUtilsSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/util/BytecodeUtilsSuite.scala @@ -1,4 +1,4 @@ -package spark.graph.util +package org.apache.spark.graph.util import org.scalatest.FunSuite From 9632ad3b2127a7154626a9a4c705121e1d3c0a71 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Thu, 19 Sep 2013 19:25:52 -0700 Subject: [PATCH 074/531] Move IndexedRDDSuite to org.apache.spark --- core/src/test/scala/{ => org/apache}/spark/IndexedRDDSuite.scala | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename core/src/test/scala/{ => org/apache}/spark/IndexedRDDSuite.scala (100%) diff --git a/core/src/test/scala/spark/IndexedRDDSuite.scala b/core/src/test/scala/org/apache/spark/IndexedRDDSuite.scala similarity index 100% rename from core/src/test/scala/spark/IndexedRDDSuite.scala rename to core/src/test/scala/org/apache/spark/IndexedRDDSuite.scala From c278907bf0dbed4ff0559e92b30d2081df7d7f60 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Thu, 19 Sep 2013 19:28:22 -0700 Subject: [PATCH 075/531] Move BytecodeUtils to the right package --- .../main/scala/org/apache/spark/graph/util/BytecodeUtils.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/util/BytecodeUtils.scala b/graph/src/main/scala/org/apache/spark/graph/util/BytecodeUtils.scala index 5db13fe3bc587..ec6eb0f911eb8 100644 --- a/graph/src/main/scala/org/apache/spark/graph/util/BytecodeUtils.scala +++ b/graph/src/main/scala/org/apache/spark/graph/util/BytecodeUtils.scala @@ -1,4 +1,4 @@ -package org.apache.spark.util +package org.apache.spark.graph.util import java.io.{ByteArrayInputStream, ByteArrayOutputStream} From d3cbde008568d16eb56dd3dd1f9c97cb3ec0b625 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Thu, 19 Sep 2013 19:29:58 -0700 Subject: [PATCH 076/531] Import appropriate Spark core classes --- core/src/test/scala/org/apache/spark/IndexedRDDSuite.scala | 1 + .../main/scala/org/apache/spark/graph/util/BytecodeUtils.scala | 2 ++ 2 files changed, 3 insertions(+) diff --git a/core/src/test/scala/org/apache/spark/IndexedRDDSuite.scala b/core/src/test/scala/org/apache/spark/IndexedRDDSuite.scala index 4f784f94d33d5..dadb183bdca09 100644 --- a/core/src/test/scala/org/apache/spark/IndexedRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/IndexedRDDSuite.scala @@ -29,6 +29,7 @@ import com.google.common.io.Files import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashSet +import org.apache.spark.rdd.RDD import org.apache.spark.rdd.ShuffledRDD import org.apache.spark.rdd.IndexedRDD diff --git a/graph/src/main/scala/org/apache/spark/graph/util/BytecodeUtils.scala b/graph/src/main/scala/org/apache/spark/graph/util/BytecodeUtils.scala index ec6eb0f911eb8..bc00ce215153d 100644 --- a/graph/src/main/scala/org/apache/spark/graph/util/BytecodeUtils.scala +++ b/graph/src/main/scala/org/apache/spark/graph/util/BytecodeUtils.scala @@ -4,6 +4,8 @@ import java.io.{ByteArrayInputStream, ByteArrayOutputStream} import scala.collection.mutable.HashSet +import org.apache.spark.util.Utils + import org.objectweb.asm.{ClassReader, ClassVisitor, MethodVisitor} import org.objectweb.asm.Opcodes._ From 3ebbcaf21cb9bd0f44930f2f0ba5596a3d04ebbb Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Thu, 19 Sep 2013 20:53:02 -0700 Subject: [PATCH 077/531] After unit tests, clear port properties unconditionally --- .../scala/org/apache/spark/LocalSparkContext.scala | 10 +++++----- .../scala/org/apache/spark/SharedSparkContext.scala | 6 ++---- 2 files changed, 7 insertions(+), 9 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/LocalSparkContext.scala b/core/src/test/scala/org/apache/spark/LocalSparkContext.scala index 6ec124da9c7b1..459e257d79a36 100644 --- a/core/src/test/scala/org/apache/spark/LocalSparkContext.scala +++ b/core/src/test/scala/org/apache/spark/LocalSparkContext.scala @@ -40,17 +40,17 @@ trait LocalSparkContext extends BeforeAndAfterEach with BeforeAndAfterAll { self } def resetSparkContext() = { - if (sc != null) { - LocalSparkContext.stop(sc) - sc = null - } + LocalSparkContext.stop(sc) + sc = null } } object LocalSparkContext { def stop(sc: SparkContext) { - sc.stop() + if (sc != null) { + sc.stop() + } // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown System.clearProperty("spark.driver.port") System.clearProperty("spark.hostPort") diff --git a/core/src/test/scala/org/apache/spark/SharedSparkContext.scala b/core/src/test/scala/org/apache/spark/SharedSparkContext.scala index 97cbca09bfa26..288aa14eeb03b 100644 --- a/core/src/test/scala/org/apache/spark/SharedSparkContext.scala +++ b/core/src/test/scala/org/apache/spark/SharedSparkContext.scala @@ -33,10 +33,8 @@ trait SharedSparkContext extends BeforeAndAfterAll { self: Suite => } override def afterAll() { - if (_sc != null) { - LocalSparkContext.stop(_sc) - _sc = null - } + LocalSparkContext.stop(_sc) + _sc = null super.afterAll() } } From f02d5c8c5304b47d819b516eddc6cb19a2fb5a8f Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Thu, 19 Sep 2013 23:03:38 -0700 Subject: [PATCH 078/531] Fix typo in aggregateNeighbors docs --- graph/src/main/scala/org/apache/spark/graph/Graph.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/Graph.scala b/graph/src/main/scala/org/apache/spark/graph/Graph.scala index 1fb22c56ff7ff..7d2a586037530 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Graph.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Graph.scala @@ -219,7 +219,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * {{{ * val graph: Graph[Int,Int] = loadGraph() * val averageFollowerAge: RDD[(Int, Int)] = - * graph.aggregateNeigbhros[(Int,Double)]( + * graph.aggregateNeighbors[(Int,Double)]( * (vid, edge) => (edge.otherVertex(vid).data, 1), * (a, b) => (a._1 + b._1, a._2 + b._2), * EdgeDirection.In) @@ -265,7 +265,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * {{{ * val graph: Graph[Int,Int] = loadGraph() * val averageFollowerAge: RDD[(Int, Int)] = - * graph.aggregateNeigbhros[(Int,Double)]( + * graph.aggregateNeighbors[(Int,Double)]( * (vid, edge) => (edge.otherVertex(vid).data, 1), * (a, b) => (a._1 + b._1, a._2 + b._2), * -1, From f08e520f4cd3ddc017361c03eb79bdd6ef8873bf Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Thu, 19 Sep 2013 21:03:57 -0700 Subject: [PATCH 079/531] Initialize sc in GraphSuite to avoid NullPointerException --- .../scala/org/apache/spark/graph/GraphSuite.scala | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala index c9dba6144d609..6f070aac5917b 100644 --- a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala @@ -3,8 +3,7 @@ package org.apache.spark.graph import org.scalatest.FunSuite import org.apache.spark.SparkContext -import org.apache.spark.graph._ - +import org.apache.spark.graph.LocalSparkContext._ class GraphSuite extends FunSuite with LocalSparkContext { @@ -12,10 +11,12 @@ class GraphSuite extends FunSuite with LocalSparkContext { // val sc = new SparkContext("local[4]", "test") test("Graph Creation") { - val rawEdges = (0L to 100L).zip((1L to 99L) :+ 0L) - val edges = sc.parallelize(rawEdges) - val graph = Graph(edges) - assert( graph.edges.count() === rawEdges.size ) + withSpark(new SparkContext("local", "test")) { sc => + val rawEdges = (0L to 100L).zip((1L to 99L) :+ 0L) + val edges = sc.parallelize(rawEdges) + val graph = Graph(edges) + assert( graph.edges.count() === rawEdges.size ) + } } test("aggregateNeighbors") { From bf05dc7e789aad0d00a76744902014c73c598134 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Thu, 19 Sep 2013 23:45:15 -0700 Subject: [PATCH 080/531] Add a unit test for aggregateNeighbors --- .../org/apache/spark/graph/GraphSuite.scala | 20 +++++++++++++++++++ 1 file changed, 20 insertions(+) diff --git a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala index 6f070aac5917b..aa885de957939 100644 --- a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala @@ -20,7 +20,27 @@ class GraphSuite extends FunSuite with LocalSparkContext { } test("aggregateNeighbors") { + withSpark(new SparkContext("local", "test")) { sc => + val star = Graph(sc.parallelize(List((0, 1), (0, 2), (0, 3)))) + + val indegrees = star.aggregateNeighbors( + (vid, edge) => Some(1), + (a: Int, b: Int) => a + b, + EdgeDirection.In).vertices.map(v => (v.id, v.data._2.getOrElse(0))) + assert(indegrees.collect().toSet === Set((0, 0), (1, 1), (2, 1), (3, 1))) + val outdegrees = star.aggregateNeighbors( + (vid, edge) => Some(1), + (a: Int, b: Int) => a + b, + EdgeDirection.Out).vertices.map(v => (v.id, v.data._2.getOrElse(0))) + assert(outdegrees.collect().toSet === Set((0, 3), (1, 0), (2, 0), (3, 0))) + + val noVertexValues = star.aggregateNeighbors[Int]( + (vid: Vid, edge: EdgeTriplet[Int, Int]) => None, + (a: Int, b: Int) => throw new Exception("reduceFunc called unexpectedly"), + EdgeDirection.In).vertices.map(v => (v.id, v.data._2)) + assert(noVertexValues.collect().toSet === Set((0, None), (1, None), (2, None), (3, None))) + } } /* test("joinVertices") { From 3c3cc1508be7898d51b1944918bae18e0a653965 Mon Sep 17 00:00:00 2001 From: Dan Crankshaw Date: Sat, 28 Sep 2013 16:00:44 -0700 Subject: [PATCH 081/531] Added initial implementation of lognormal graph generator. Haven't tested it yet. --- .../spark/graph/util/GraphGenerators.scala | 89 +++++++++++++++++++ 1 file changed, 89 insertions(+) create mode 100644 graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala diff --git a/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala b/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala new file mode 100644 index 0000000000000..43d8b5f0728ca --- /dev/null +++ b/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala @@ -0,0 +1,89 @@ +package org.apache.spark.graph.util + +import util.Random.nextGaussian +import math._ + +import org.apache.spark.rdd.RDD +import org.apache.spark.SparkContext +import org.apache.spark.SparkContext._ +import org.apache.spark.graph.Graph +import org.apache.spark.graph.Vertex +import org.apache.spark.graph.Edge + +object GraphGenerator { + + /* + TODO(crankshaw) delete + Just notes for me: + for every vertex: + generate the number of outdegrees + create the vertex Vertex(vid, outdegrees) + create the edges: generateRandomEdges + add vertex to vertex list + add edges to edgelist + + */ + def main(args: Array[String]) { + println("hello world") + + } + + // For now just writes graph to a file. Eventually + // it will return a spark.graph.Graph + + + // Right now it just generates a bunch of edges where + // the edge data is the weight (default 1) + def lognormalGraph(numVertices: Long, fname: String) = { + // based on Pregel settings + val mu = 4 + val sigma = 1.3 + val vertsAndEdges = Range(0, numVertices).flatmap { src => { + val outdegree = sampleLogNormal(mu, sigma, numVertices) + val vertex = Vertex(src, outdegree) + val edges = generateRandomEdges(src, outdegree, numVertices) + (vertex, edges) } + } + val vertices, edges = vertsAndEdges.unzip + val graph = new GraphImpl[Int, Int](vertices, edges.flatten) + } + + def generateRandomEdges(src: Long, numEdges: Long, maxVid): Array[Edge[Int]] = { + var dsts = new Set() + while (dsts.size() < numEdges) { + val nextDst = nextInt(maxVid) + if (nextDst != src) { + dsts += nextDst + } + } + val edges = dsts.map(dst => Array(Edge(src, dst, 1))).toList + } + + + /** + * 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 the + * mean, standard deviation of the lognormal distribution + * and Z~N(0, 1). In this function, + * m = e^(mu+sigma^2/2) and + * s = sqrt[(e^(sigma^2) - 1)(e^(2*mu+sigma^2))]. + * + * @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 + */ + def sampleLogNormal(mu: Float, sigma: Float, maxVal: Long): Long = { + val m = math.exp(mu+(sigma*sigma)/2.0) + val s = math.sqrt((math.exp(sigma*sigma) - 1) * math.exp(2*mu + sigma*sigma)) + // Z ~ N(0, 1) + var X = maxVal + while (X >= maxVal) { + val Z = nextGaussian() + X = math.exp(m + s*Z) + } + math.round(X) + } + +} From 8edd499effc049528d769956486f497137458298 Mon Sep 17 00:00:00 2001 From: Dan Crankshaw Date: Thu, 3 Oct 2013 10:21:34 -0700 Subject: [PATCH 082/531] Added rmat graph generator --- .../spark/graph/util/GraphGenerators.scala | 205 +++++++++++++++--- 1 file changed, 175 insertions(+), 30 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala b/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala index 43d8b5f0728ca..4a146b0509bdf 100644 --- a/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala +++ b/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala @@ -1,62 +1,88 @@ package org.apache.spark.graph.util -import util.Random.nextGaussian +import util._ import math._ +//import scala.collection.mutable + import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext import org.apache.spark.SparkContext._ +import org.apache.spark.graph._ import org.apache.spark.graph.Graph import org.apache.spark.graph.Vertex import org.apache.spark.graph.Edge +import org.apache.spark.graph.impl.GraphImpl + +// TODO(crankshaw) I might want to pull at least RMAT out into a separate class. +// Might simplify the code to have classwide variables and such. object GraphGenerator { - /* - TODO(crankshaw) delete - Just notes for me: - for every vertex: - generate the number of outdegrees - create the vertex Vertex(vid, outdegrees) - create the edges: generateRandomEdges - add vertex to vertex list - add edges to edgelist + val RMATa = 0.45 + val RMATb = 0.15 + val RMATc = 0.15 + val RMATd = 0.25 + /* + * TODO(crankshaw) delete + * How do I create a spark context and RDD and stuff? + * Like how do I actually make this program run? */ def main(args: Array[String]) { + + + System.setProperty("spark.serializer", "spark.KryoSerializer") + //System.setProperty("spark.shuffle.compress", "false") + System.setProperty("spark.kryo.registrator", "spark.graph.GraphKryoRegistrator") + val host = "local[4]" + val sc = new SparkContext(host, "Lognormal graph generator") println("hello world") } - + // For now just writes graph to a file. Eventually // it will return a spark.graph.Graph // Right now it just generates a bunch of edges where // the edge data is the weight (default 1) - def lognormalGraph(numVertices: Long, fname: String) = { + def lognormalGraph(sc: SparkContext, numVertices: Int): GraphImpl[Int, Int] = { // based on Pregel settings val mu = 4 val sigma = 1.3 - val vertsAndEdges = Range(0, numVertices).flatmap { src => { - val outdegree = sampleLogNormal(mu, sigma, numVertices) - val vertex = Vertex(src, outdegree) - val edges = generateRandomEdges(src, outdegree, numVertices) - (vertex, edges) } - } - val vertices, edges = vertsAndEdges.unzip - val graph = new GraphImpl[Int, Int](vertices, edges.flatten) + //val vertsAndEdges = (0 until numVertices).flatMap { src => { + val vertices = (0 until numVertices).flatMap { src => + Array(Vertex(src, sampleLogNormal(mu, sigma, numVertices))) } + val edges = vertices.flatMap( { v => + generateRandomEdges(v.id.toInt, v.data, numVertices) }) + + + + new GraphImpl[Int, Int](sc.parallelize(vertices), sc.parallelize(edges)) + //println("Vertices:") + //for (v <- vertices) { + // println(v.id) + //} + + //println("Edges") + //for (e <- edges) { + // println(e.src, e.dst, e.data) + //} + } - def generateRandomEdges(src: Long, numEdges: Long, maxVid): Array[Edge[Int]] = { - var dsts = new Set() - while (dsts.size() < numEdges) { - val nextDst = nextInt(maxVid) + + def generateRandomEdges(src: Int, numEdges: Int, maxVid: Int): Array[Edge[Int]] = { + val rand = new Random() + var dsts: Set[Int] = Set() + while (dsts.size < numEdges) { + val nextDst = rand.nextInt(maxVid) if (nextDst != src) { dsts += nextDst } } - val edges = dsts.map(dst => Array(Edge(src, dst, 1))).toList + dsts.map {dst => Edge[Int](src, dst, 1) }.toArray } @@ -74,16 +100,135 @@ object GraphGenerator { * @param sigma the standard deviation of the normal distribution * @param macVal exclusive upper bound on the value of the sample */ - def sampleLogNormal(mu: Float, sigma: Float, maxVal: Long): Long = { + 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)) // Z ~ N(0, 1) - var X = maxVal + var X: Double = maxVal while (X >= maxVal) { - val Z = nextGaussian() - X = math.exp(m + s*Z) + val Z = rand.nextGaussian() + X = math.exp((m + s*Z)) } - math.round(X) + math.round(X.toFloat) } + + + def rmatGraph(sc: SparkContext, requestedNumVertices: Int, numEdges: Int): GraphImpl[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 + var edges: Set[Edge[Int]] = Set() + while (edges.size < numEdges) { + edges += addEdge(numVertices) + + } + val graph = outDegreeFromEdges(sc.parallelize(edges.toList)) + graph + + } + + def outDegreeFromEdges[ED: ClassManifest](edges: RDD[Edge[ED]]): GraphImpl[Int, ED] = { + + val vertices = edges.flatMap { edge => List((edge.src, 1)) } + .reduceByKey(_ + _) + .map{ case (vid, degree) => Vertex(vid, degree) } + new GraphImpl[Int, ED](vertices, edges) + } + + /** + * @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] = { + //val (src, dst) = chooseCell(numVertices/2.0, numVertices/2.0, numVertices/2.0) + val v = math.round(numVertices.toFloat/2.0).toInt + + val (src, dst) = chooseCell(v, v, v) + 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 + * those of the R-MAT paper. There are a power of 2 number of nodes in the graph. + * The adjacency matrix looks like: + * + * dst -> + * (x,y) *************** _ + * | | | | + * | a | b | | + * src | | | | + * | *************** | T + * \|/ | | | | + * | c | d | | + * | | | | + * *************** - + * + * where this represents the subquadrant of the adj matrix currently being + * subdivided. (x,y) represent the upper left hand corner of the subquadrant, + * and T represents the side length (guaranteed to be a power of 2). + * + * After choosing the next level subquadrant, we get the resulting sets + * of parameters: + * quad = a, x'=x, y'=y, T'=T/2 + * quad = b, x'=x+T/2, y'=y, T'=T/2 + * quad = c, x'=x, y'=y+T/2, T'=T/2 + * quad = d, x'=x+T/2, y'=y+T/2, T'=T/2 + * + * @param src is the + */ + @tailrec 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) + case 1 => chooseCell(x+newT, y, newT) + case 2 => chooseCell(x, y+newT, newT) + case 3 => chooseCell(x+newT, y+newT, newT) + } + } + + + + + + + + + } + + // 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") + } + 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 _ => 3 // 3 corresponds to d + } + } + + + } + + + + + + + + + + + From 27b442dc060d3f948f4f9adeb31adf76c26e4adf Mon Sep 17 00:00:00 2001 From: Dan Crankshaw Date: Thu, 3 Oct 2013 10:29:00 -0700 Subject: [PATCH 083/531] Fixed annotation import --- .../scala/org/apache/spark/graph/util/GraphGenerators.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala b/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala index 4a146b0509bdf..d01b59c04d008 100644 --- a/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala +++ b/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala @@ -2,6 +2,7 @@ package org.apache.spark.graph.util import util._ import math._ +import scala.annotation.tailrec //import scala.collection.mutable @@ -181,7 +182,8 @@ object GraphGenerator { * * @param src is the */ - @tailrec def chooseCell(x: Int, y: Int, t: Int): (Int, Int) = { + @tailrec + def chooseCell(x: Int, y: Int, t: Int): (Int, Int) = { if (t <= 1) (x,y) else { From 1ee60d3b3417559ae4e6638731c4051dca0c2018 Mon Sep 17 00:00:00 2001 From: Dan Crankshaw Date: Thu, 3 Oct 2013 17:46:37 -0700 Subject: [PATCH 084/531] Fixed bug in sampleLogNormal --- .../spark/graph/util/GraphGenerators.scala | 47 ++++++++++++++----- 1 file changed, 35 insertions(+), 12 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala b/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala index d01b59c04d008..5a9d8fbd6b5db 100644 --- a/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala +++ b/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala @@ -6,6 +6,8 @@ import scala.annotation.tailrec //import scala.collection.mutable +import org.apache.spark._ +import org.apache.spark.serializer._ import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext import org.apache.spark.SparkContext._ @@ -18,7 +20,7 @@ import org.apache.spark.graph.impl.GraphImpl // TODO(crankshaw) I might want to pull at least RMAT out into a separate class. // Might simplify the code to have classwide variables and such. -object GraphGenerator { +object GraphGenerators { val RMATa = 0.45 val RMATb = 0.15 @@ -33,12 +35,36 @@ object GraphGenerator { def main(args: Array[String]) { - System.setProperty("spark.serializer", "spark.KryoSerializer") + val serializer = "org.apache.spark.serializer.KryoSerializer" + System.setProperty("spark.serializer", serializer) //System.setProperty("spark.shuffle.compress", "false") System.setProperty("spark.kryo.registrator", "spark.graph.GraphKryoRegistrator") val host = "local[4]" val sc = new SparkContext(host, "Lognormal graph generator") - println("hello world") + + val lnGraph = lognormalGraph(sc, 10000) + + val rmat = rmatGraph(sc, 1000, 3000) + + //for (v <- lnGraph.vertices) { + // println(v.id + ":\t" + v.data) + //} + + val times = 100000 + //val nums = (1 to times).flatMap { n => List(sampleLogNormal(4.0, 1.3, times)) }.toList + //val avg = nums.sum / nums.length + //val sumSquares = nums.foldLeft(0.0) {(total, next) => + // (total + math.pow((next - avg), 2)) } + //val stdev = math.sqrt(sumSquares/(nums.length - 1)) + + //println("avg: " + avg + "+-" + stdev) + + + //for (i <- 1 to 1000) { + // println(sampleLogNormal(4.0, 1.3, 1000)) + //} + + sc.stop() } @@ -107,9 +133,11 @@ object GraphGenerator { val s = math.sqrt((math.exp(sigma*sigma) - 1) * math.exp(2*mu + sigma*sigma)) // Z ~ N(0, 1) var X: Double = maxVal + while (X >= maxVal) { val Z = rand.nextGaussian() - X = math.exp((m + s*Z)) + //X = math.exp((m + s*Z)) + X = math.exp((mu + sigma*Z)) } math.round(X.toFloat) } @@ -123,6 +151,9 @@ object GraphGenerator { 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) } @@ -195,14 +226,6 @@ object GraphGenerator { case 3 => chooseCell(x+newT, y+newT, newT) } } - - - - - - - - } // TODO(crankshaw) turn result into an enum (or case class for pattern matching} From da3e123afb89235e2c30df335103c423b17bd8c1 Mon Sep 17 00:00:00 2001 From: Dan Crankshaw Date: Thu, 3 Oct 2013 18:11:35 -0700 Subject: [PATCH 085/531] Removed some comments --- graph/src/main/scala/org/apache/spark/graph/Graph.scala | 2 ++ .../scala/org/apache/spark/graph/util/GraphGenerators.scala | 5 ----- 2 files changed, 2 insertions(+), 5 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/Graph.scala b/graph/src/main/scala/org/apache/spark/graph/Graph.scala index 09a1af63a6713..40673bbc90339 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Graph.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Graph.scala @@ -189,6 +189,8 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { // def combineEdges(reduce: (ED, ED) => ED): Graph[VD, ED] + + /** * This function is used to compute a statistic for the neighborhood of each * vertex. diff --git a/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala b/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala index 5a9d8fbd6b5db..d0583c48a8597 100644 --- a/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala +++ b/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala @@ -27,11 +27,6 @@ object GraphGenerators { val RMATc = 0.15 val RMATd = 0.25 - /* - * TODO(crankshaw) delete - * How do I create a spark context and RDD and stuff? - * Like how do I actually make this program run? - */ def main(args: Array[String]) { From e096cbe90ea9f6c5d7ea2afc8650c52e09d28c03 Mon Sep 17 00:00:00 2001 From: Dan Crankshaw Date: Sat, 5 Oct 2013 15:20:15 -0700 Subject: [PATCH 086/531] Added 2D canonical edge partitioner --- .../org/apache/spark/graph/impl/GraphImpl.scala | 16 +++++++++++++++- 1 file changed, 15 insertions(+), 1 deletion(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index e397293a3d63d..baf7291474ced 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -361,7 +361,7 @@ object GraphImpl { * 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 effectively suffle the vertex locations. + * to effectively 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 @@ -381,6 +381,20 @@ object GraphImpl { } + protected def canonicalEdgePartitionFunction2D(srcOrig: Vid, dstOrig: Vid, + numParts: Pid, ceilSqrtNumParts: Pid): Pid = { + val mixingPrime: Vid = 1125899906842597L + + // Partitions by canonical edge direction + // @todo(crankshaw) evaluate the cases + val src = math.min(srcOrig, dstOrig) + val dst = math.max(srcOrig, dstOrig) + val col: Pid = ((math.abs(src) * mixingPrime) % ceilSqrtNumParts).toInt + val row: Pid = ((math.abs(dst) * mixingPrime) % ceilSqrtNumParts).toInt + (col * ceilSqrtNumParts + row) % numParts + } + + /** * Create the edge table RDD, which is much more efficient for Java heap storage than the * normal edges data structure (RDD[(Vid, Vid, ED)]). From bfedbee13a3c2355c307840d3f7548c2737d37bb Mon Sep 17 00:00:00 2001 From: Dan Crankshaw Date: Sat, 5 Oct 2013 16:04:57 -0700 Subject: [PATCH 087/531] Edge partitioner now partitions by canonical edge so all edges between two vertices (in either direction) will be sent to same machine. --- .../scala/org/apache/spark/graph/impl/GraphImpl.scala | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index baf7291474ced..0ba39d8d80874 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -381,12 +381,13 @@ object GraphImpl { } + /** + * @todo(crankshaw) how does this effect load balancing? + */ protected def canonicalEdgePartitionFunction2D(srcOrig: Vid, dstOrig: Vid, numParts: Pid, ceilSqrtNumParts: Pid): Pid = { val mixingPrime: Vid = 1125899906842597L - // Partitions by canonical edge direction - // @todo(crankshaw) evaluate the cases val src = math.min(srcOrig, dstOrig) val dst = math.max(srcOrig, dstOrig) val col: Pid = ((math.abs(src) * mixingPrime) % ceilSqrtNumParts).toInt @@ -411,7 +412,8 @@ object GraphImpl { .map { e => // Random partitioning based on the source vertex id. // val part: Pid = edgePartitionFunction1D(e.src, e.dst, numPartitions) - val part: Pid = edgePartitionFunction2D(e.src, e.dst, numPartitions, ceilSqrt) + //val part: Pid = edgePartitionFunction2D(e.src, e.dst, numPartitions, ceilSqrt) + val part: Pid = canonicalEdgePartitionFunction2D(e.src, e.dst, numPartitions, ceilSqrt) // Should we be using 3-tuple or an optimized class (part, (e.src, e.dst, e.data)) From 730a3156d394b6ba7b7caa0087bd8c09d62dcfad Mon Sep 17 00:00:00 2001 From: Dan Crankshaw Date: Sat, 5 Oct 2013 19:44:28 -0700 Subject: [PATCH 088/531] Added initial groupEdges code. Still a prototype, I haven't figured out quite how it should all work yet. --- .../scala/org/apache/spark/graph/Graph.scala | 2 + .../apache/spark/graph/impl/GraphImpl.scala | 53 +++++++++++++++++++ 2 files changed, 55 insertions(+) diff --git a/graph/src/main/scala/org/apache/spark/graph/Graph.scala b/graph/src/main/scala/org/apache/spark/graph/Graph.scala index 40673bbc90339..6fe71b4756c0d 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Graph.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Graph.scala @@ -188,6 +188,8 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { // */ // def combineEdges(reduce: (ED, ED) => ED): Graph[VD, ED] + def groupEdges[ED2: ClassManifest](f: Iter[EdgeTriplet[ED,VD]] => ED2 ): Graph[VD,ED2] + diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index 0ba39d8d80874..d0e03e0ce22b0 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -118,6 +118,59 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( new GraphImpl(newVertices, newEdges) } + + // Because of the edgepartitioner, we know that all edges with the same src and dst + // will be in the same partition + + // We will want to keep the same partitioning scheme. Use newGraph() rather than + // new GraphImpl() + override def groupEdges[ED2: ClassManifest](f: Iterator[EdgeTriplet[ED,VD]] => ED2 ): + Graph[VD,ED2] = { + + // I think that + // myRDD.mapPartitions { part => + // val (vmap, edges) = part.next() + // gives me access to the vertex map and the set of + // edges within that partition + + // This is what happens during mapPartitions + // The iterator iterates over all partitions + // val result: RDD[U] = new RDD[T]().mapPartitions(f: Iterator[T] => Iterator[U]) + + // TODO(crankshaw) figure out how to actually get the new Edge RDD and what + // type that should have + val newEdges: RDD[Edge[ED2]] = triplets.mapPartitions { partIterator => + // toList lets us operate on all EdgeTriplets in a single partition at once + partIterator.toList + // groups all ETs in this partition that have the same src and dst + // Because all ETs with the same src and dst will live on the same + // partition due to the EdgePartitioner, this guarantees that these + // ET groups will be complete. + .groupBy { t => (t.src.id, t.dst.id) } + // Apply the user supplied supplied edge group function to + // each group of edges + // The result of this line is Map[(Long, Long, ED2] + .mapValues { ts => f(ts.toIterator) } + // convert the resulting map back to a list of tuples + .toList + // map over those tuples that contain src and dst info plus the + // new edge data to make my new edges + .map { case ((src, dst), data) => Edge(src, dst, data) } + + // How do I convert from a scala map to a list? + // I want to be able to apply a function like: + // f: (key, value): (K, V) => result: [R] + // so that I can transfrom a Map[K, V] to List[R] + + // Maybe look at collections.breakOut + // see http://stackoverflow.com/questions/1715681/scala-2-8-breakout + // and http://stackoverflow.com/questions/6998676/converting-a-scala-map-to-a-list + + } + newGraph(vertices, newEdges) + + } + ////////////////////////////////////////////////////////////////////////////////////////////////// // Lower level transformation methods ////////////////////////////////////////////////////////////////////////////////////////////////// From 6cb21ce8894c4d62c1473a09b98882bf69a8e79d Mon Sep 17 00:00:00 2001 From: Dan Crankshaw Date: Sun, 6 Oct 2013 15:33:35 -0700 Subject: [PATCH 089/531] groupEdges() now compiles. Still need some unit tests --- graph/src/main/scala/org/apache/spark/graph/Graph.scala | 2 +- .../main/scala/org/apache/spark/graph/impl/GraphImpl.scala | 4 +++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/Graph.scala b/graph/src/main/scala/org/apache/spark/graph/Graph.scala index 6fe71b4756c0d..8d6430da2c637 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Graph.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Graph.scala @@ -188,7 +188,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { // */ // def combineEdges(reduce: (ED, ED) => ED): Graph[VD, ED] - def groupEdges[ED2: ClassManifest](f: Iter[EdgeTriplet[ED,VD]] => ED2 ): Graph[VD,ED2] + def groupEdges[ED2: ClassManifest](f: Iterator[EdgeTriplet[VD,ED]] => ED2 ): Graph[VD,ED2] diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index d0e03e0ce22b0..4ce9239abba68 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -124,7 +124,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( // We will want to keep the same partitioning scheme. Use newGraph() rather than // new GraphImpl() - override def groupEdges[ED2: ClassManifest](f: Iterator[EdgeTriplet[ED,VD]] => ED2 ): + override def groupEdges[ED2: ClassManifest](f: Iterator[EdgeTriplet[VD,ED]] => ED2 ): Graph[VD,ED2] = { // I think that @@ -153,6 +153,8 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( .mapValues { ts => f(ts.toIterator) } // convert the resulting map back to a list of tuples .toList + // TODO(crankshaw) needs an iterator over the tuples? Why can't I map over the list? + .toIterator // map over those tuples that contain src and dst info plus the // new edge data to make my new edges .map { case ((src, dst), data) => Edge(src, dst, data) } From 0d3ea36fd88786b22fd7f7eae609da0336b4b72e Mon Sep 17 00:00:00 2001 From: Dan Crankshaw Date: Sun, 6 Oct 2013 18:34:23 -0700 Subject: [PATCH 090/531] Added a groupEdges and a groupEdgeTriplets method. For some reason the groupEdgeTriplets method isn't properly iterating through the set of edges and thus is returning the wrong result. groupEdges seems to be working. --- .../scala/org/apache/spark/graph/Graph.scala | 5 +- .../apache/spark/graph/impl/GraphImpl.scala | 59 ++++++++++++++++--- .../org/apache/spark/graph/GraphSuite.scala | 35 +++++++++++ 3 files changed, 91 insertions(+), 8 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/Graph.scala b/graph/src/main/scala/org/apache/spark/graph/Graph.scala index 8d6430da2c637..de0a090504547 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Graph.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Graph.scala @@ -188,7 +188,10 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { // */ // def combineEdges(reduce: (ED, ED) => ED): Graph[VD, ED] - def groupEdges[ED2: ClassManifest](f: Iterator[EdgeTriplet[VD,ED]] => ED2 ): Graph[VD,ED2] + def groupEdgeTriplets[ED2: ClassManifest](f: Iterator[EdgeTriplet[VD,ED]] => ED2 ): Graph[VD,ED2] + + def groupEdges[ED2: ClassManifest](f: Iterator[Edge[ED]] => ED2 ): Graph[VD,ED2] + diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index 4ce9239abba68..6d2ce70eadd8b 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -124,7 +124,11 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( // We will want to keep the same partitioning scheme. Use newGraph() rather than // new GraphImpl() - override def groupEdges[ED2: ClassManifest](f: Iterator[EdgeTriplet[VD,ED]] => ED2 ): + // TODO(crankshaw) is there a better way to do this using RDD.groupBy() + // functions? + + override def groupEdgeTriplets[ED2: ClassManifest](f: Iterator[EdgeTriplet[VD,ED]] => ED2 ): + //override def groupEdges[ED2: ClassManifest](f: Iterator[Edge[ED]] => ED2 ): Graph[VD,ED2] = { // I think that @@ -139,18 +143,21 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( // TODO(crankshaw) figure out how to actually get the new Edge RDD and what // type that should have - val newEdges: RDD[Edge[ED2]] = triplets.mapPartitions { partIterator => + val newEdges: RDD[Edge[ED2]] = triplets.mapPartitions { partIter => // toList lets us operate on all EdgeTriplets in a single partition at once - partIterator.toList + partIter.toList // groups all ETs in this partition that have the same src and dst // Because all ETs with the same src and dst will live on the same // partition due to the EdgePartitioner, this guarantees that these // ET groups will be complete. - .groupBy { t => (t.src.id, t.dst.id) } + .groupBy { t: EdgeTriplet[VD, ED] => + println(t.src.id + " " + t.dst.id) + (t.src.id, t.dst.id) } + //.groupBy { e => (e.src, e.dst) } // Apply the user supplied supplied edge group function to // each group of edges // The result of this line is Map[(Long, Long, ED2] - .mapValues { ts => f(ts.toIterator) } + .mapValues { ts: List[EdgeTriplet[VD, ED]] => f(ts.toIterator) } // convert the resulting map back to a list of tuples .toList // TODO(crankshaw) needs an iterator over the tuples? Why can't I map over the list? @@ -173,6 +180,44 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( } + + override def groupEdges[ED2: ClassManifest](f: Iterator[Edge[ED]] => ED2 ): + Graph[VD,ED2] = { + + val newEdges: RDD[Edge[ED2]] = edges.mapPartitions { partIter => + partIter.toList + .groupBy { e: Edge[ED] => + println(e.src + " " + e.dst) + (e.src, e.dst) } + .mapValues { ts => f(ts.toIterator) } + .toList + .toIterator + .map { case ((src, dst), data) => Edge(src, dst, data) } + + + } + newGraph(vertices, newEdges) + + } + + + + + //override def groupEdges[ED2: ClassManifest](f: Iterator[EdgeTriplet[VD,ED]] => ED2 ): + // Graph[VD,ED] = { + // val groups = triplets.collect.toList.groupBy { t => (t.src.id, t.dst.id) } + // for (k <- groups.keys) { + // println("^^^^^^^^^^^^^^^^^ " + k + " ^^^^^^^^^^^^^^^^^^^^^") + + // } + // val transformMap: Map[(Vid, Vid), ED2] = groups.mapValues { ts => f(ts.toIterator) } + // val newList: List[((Vid, Vid), ED2)] = transformMap.toList + // val newEdges: List[Edge[ED2]] = newList.map { case ((src, dst), data) => Edge(src, dst, data) } + + // newGraph(vertices, edges) + + //} + ////////////////////////////////////////////////////////////////////////////////////////////////// // Lower level transformation methods ////////////////////////////////////////////////////////////////////////////////////////////////// @@ -467,8 +512,8 @@ object GraphImpl { .map { e => // Random partitioning based on the source vertex id. // val part: Pid = edgePartitionFunction1D(e.src, e.dst, numPartitions) - //val part: Pid = edgePartitionFunction2D(e.src, e.dst, numPartitions, ceilSqrt) - val part: Pid = canonicalEdgePartitionFunction2D(e.src, e.dst, numPartitions, ceilSqrt) + val part: Pid = edgePartitionFunction2D(e.src, e.dst, numPartitions, ceilSqrt) + //val part: Pid = canonicalEdgePartitionFunction2D(e.src, e.dst, numPartitions, ceilSqrt) // Should we be using 3-tuple or an optimized class (part, (e.src, e.dst, e.data)) diff --git a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala index aa885de957939..ce9d2104a2108 100644 --- a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala @@ -43,6 +43,41 @@ class GraphSuite extends FunSuite with LocalSparkContext { } } + test("groupEdges") { + withSpark(new SparkContext("local", "test")) { sc => + val vertices = sc.parallelize(List(Vertex(6, 1),Vertex(7, 1), Vertex(8,1))) + val edges = sc.parallelize(List( + Edge(6, 7, 0.4), + Edge(6, 7, 0.9), + Edge(6, 7, 0.7), + Edge(7, 6, 25.0), + Edge(7, 6, 300.0), + Edge(7, 6, 600.0), + Edge(8, 7, 11.0), + Edge(7, 8, 89.0))) + + val original = Graph(vertices, edges) + for (e <- original.edges) { + println("(" + e.src + ", " + e.dst + ", " + e.data + ")") + } + //assert(original.edges.count() === 6) + val grouped = original.groupEdgeTriplets { iter => + println("----------------------------------------") + iter.map(_.data).sum } + + for (e <- grouped.edges) { + println("******************************(" + e.src + ", " + e.dst + ", " + e.data + ")") + } + + //val groups: Map[(Vid, Vid), List[Edge[Double]]] = original.edges.collect.toList.groupBy { e => (e.src, e.dst) } + //for (k <- groups.keys) { + // println("################# " + k + " #################") + //} + //assert(grouped.edges.count() === 2) + //assert(grouped.edges.collect().toSet === Set(Edge(0, 1, 2.0), Edge(1, 0, 6.0))) + } + } + /* test("joinVertices") { sc = new SparkContext("local", "test") val vertices = sc.parallelize(Seq(Vertex(1, "one"), Vertex(2, "two"), Vertex(3, "three")), 2) From 2a8f3db94d779c6155d6bef8df952a90ef2b640c Mon Sep 17 00:00:00 2001 From: Dan Crankshaw Date: Sun, 6 Oct 2013 19:52:40 -0700 Subject: [PATCH 091/531] Fixed groupEdgeTriplets - it now passes a basic unit test. The problem was with the way the EdgeTripletRDD iterator worked. Calling toList on it returned the last value repeatedly. Fixed by overriding toList in the iterator. --- .../spark/graph/impl/EdgeTripletRDD.scala | 29 ++++++++++++++- .../apache/spark/graph/impl/GraphImpl.scala | 21 ++--------- .../org/apache/spark/graph/GraphSuite.scala | 37 +++++++------------ 3 files changed, 44 insertions(+), 43 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/EdgeTripletRDD.scala b/graph/src/main/scala/org/apache/spark/graph/impl/EdgeTripletRDD.scala index 18d5d2b5aae70..1cd48120a1aa4 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/EdgeTripletRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/EdgeTripletRDD.scala @@ -1,5 +1,7 @@ package org.apache.spark.graph.impl +import scala.collection.mutable + import org.apache.spark.Aggregator import org.apache.spark.Partition import org.apache.spark.SparkEnv @@ -29,8 +31,8 @@ class EdgeTripletRDD[VD: ClassManifest, ED: ClassManifest]( eTable: RDD[(Pid, EdgePartition[ED])]) extends RDD[(VertexHashMap[VD], Iterator[EdgeTriplet[VD, ED]])](eTable.context, Nil) { - println(vTableReplicated.partitioner.get.numPartitions) - println(eTable.partitioner.get.numPartitions) + //println("ddshfkdfhds" + vTableReplicated.partitioner.get.numPartitions) + //println("9757984589347598734549" + eTable.partitioner.get.numPartitions) assert(vTableReplicated.partitioner == eTable.partitioner) @@ -77,10 +79,33 @@ class EdgeTripletRDD[VD: ClassManifest, ED: ClassManifest]( // assert(vmap.containsKey(e.dst.id)) e.dst.data = vmap.get(e.dst.id) + //println("Iter called: " + pos) e.data = edgePartition.data(pos) pos += 1 e } + + override def toList: List[EdgeTriplet[VD, ED]] = { + val lb = new mutable.ListBuffer[EdgeTriplet[VD,ED]] + for (i <- (0 until edgePartition.size)) { + val currentEdge = new EdgeTriplet[VD, ED] + currentEdge.src = new Vertex[VD] + currentEdge.dst = new Vertex[VD] + currentEdge.src.id = edgePartition.srcIds.getLong(i) + // assert(vmap.containsKey(e.src.id)) + currentEdge.src.data = vmap.get(currentEdge.src.id) + + currentEdge.dst.id = edgePartition.dstIds.getLong(i) + // assert(vmap.containsKey(e.dst.id)) + currentEdge.dst.data = vmap.get(currentEdge.dst.id) + + currentEdge.data = edgePartition.data(i) + //println("Iter: " + pos + " " + e.src.id + " " + e.dst.id + " " + e.data) + //println("List: " + i + " " + currentEdge.src.id + " " + currentEdge.dst.id + " " + currentEdge.data) + lb += currentEdge + } + lb.toList + } } Iterator((vmap, iter)) } diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index 6d2ce70eadd8b..a6953d764c2e7 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -145,13 +145,14 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( // type that should have val newEdges: RDD[Edge[ED2]] = triplets.mapPartitions { partIter => // toList lets us operate on all EdgeTriplets in a single partition at once - partIter.toList + partIter + .toList // groups all ETs in this partition that have the same src and dst // Because all ETs with the same src and dst will live on the same // partition due to the EdgePartitioner, this guarantees that these // ET groups will be complete. .groupBy { t: EdgeTriplet[VD, ED] => - println(t.src.id + " " + t.dst.id) + //println("(" + t.src.id + ", " + t.dst.id + ", " + t.data + ")") (t.src.id, t.dst.id) } //.groupBy { e => (e.src, e.dst) } // Apply the user supplied supplied edge group function to @@ -202,22 +203,6 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( - - //override def groupEdges[ED2: ClassManifest](f: Iterator[EdgeTriplet[VD,ED]] => ED2 ): - // Graph[VD,ED] = { - // val groups = triplets.collect.toList.groupBy { t => (t.src.id, t.dst.id) } - // for (k <- groups.keys) { - // println("^^^^^^^^^^^^^^^^^ " + k + " ^^^^^^^^^^^^^^^^^^^^^") - - // } - // val transformMap: Map[(Vid, Vid), ED2] = groups.mapValues { ts => f(ts.toIterator) } - // val newList: List[((Vid, Vid), ED2)] = transformMap.toList - // val newEdges: List[Edge[ED2]] = newList.map { case ((src, dst), data) => Edge(src, dst, data) } - - // newGraph(vertices, edges) - - //} - ////////////////////////////////////////////////////////////////////////////////////////////////// // Lower level transformation methods ////////////////////////////////////////////////////////////////////////////////////////////////// diff --git a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala index ce9d2104a2108..8c85260c1b520 100644 --- a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala @@ -47,34 +47,25 @@ class GraphSuite extends FunSuite with LocalSparkContext { withSpark(new SparkContext("local", "test")) { sc => val vertices = sc.parallelize(List(Vertex(6, 1),Vertex(7, 1), Vertex(8,1))) val edges = sc.parallelize(List( - Edge(6, 7, 0.4), - Edge(6, 7, 0.9), - Edge(6, 7, 0.7), - Edge(7, 6, 25.0), - Edge(7, 6, 300.0), - Edge(7, 6, 600.0), - Edge(8, 7, 11.0), - Edge(7, 8, 89.0))) + Edge(6, 7, 4), + Edge(6, 7, 9), + Edge(6, 7, 7), + Edge(7, 6, 25), + Edge(7, 6, 300), + Edge(7, 6, 600), + Edge(8, 7, 11), + Edge(7, 8, 89))) val original = Graph(vertices, edges) - for (e <- original.edges) { - println("(" + e.src + ", " + e.dst + ", " + e.data + ")") - } - //assert(original.edges.count() === 6) val grouped = original.groupEdgeTriplets { iter => - println("----------------------------------------") iter.map(_.data).sum } - for (e <- grouped.edges) { - println("******************************(" + e.src + ", " + e.dst + ", " + e.data + ")") - } - - //val groups: Map[(Vid, Vid), List[Edge[Double]]] = original.edges.collect.toList.groupBy { e => (e.src, e.dst) } - //for (k <- groups.keys) { - // println("################# " + k + " #################") - //} - //assert(grouped.edges.count() === 2) - //assert(grouped.edges.collect().toSet === Set(Edge(0, 1, 2.0), Edge(1, 0, 6.0))) + assert(grouped.edges.count() === 4) + assert(grouped.edges.collect().toSet === Set( + Edge(6, 7, 20), + Edge(7, 6, 925), + Edge(8, 7, 11), + Edge(7, 8, 89))) } } From 4f916f5302ef56cf0f53a8f214602623ccdae841 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 7 Oct 2013 11:31:00 -0700 Subject: [PATCH 092/531] Created a MessageToPartition class to send messages without saving the partition id. --- .../apache/spark/graph/impl/GraphImpl.scala | 22 +++++---- .../spark/graph/impl/MessageToPartition.scala | 49 +++++++++++++++++++ 2 files changed, 61 insertions(+), 10 deletions(-) create mode 100644 graph/src/main/scala/org/apache/spark/graph/impl/MessageToPartition.scala diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index e397293a3d63d..fb7698f915357 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -11,9 +11,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.graph._ import org.apache.spark.graph.impl.GraphImpl._ - - - +import org.apache.spark.graph.impl.MessageToPartitionRDDFunctions._ /** @@ -309,11 +307,13 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( // Join vid2pid and vTable, generate a shuffle dependency on the joined result, and get // the shuffle id so we can use it on the slave. vTable - .flatMap { case (vid, (vdata, pids)) => pids.iterator.map { pid => (pid, (vid, vdata)) } } + .flatMap { case (vid, (vdata, pids)) => + pids.iterator.map { pid => MessageToPartition(pid, (vid, vdata)) } + } .partitionBy(edgePartitioner) - .mapPartitions( - { part => part.map { case(pid, (vid, vdata)) => (vid, vdata) } }, - preservesPartitioning = true) + .mapPartitions({ part => + part.map { message => (message.data._1, message.data._2) } + }, preservesPartitioning = true) } } @@ -400,14 +400,16 @@ object GraphImpl { val part: Pid = edgePartitionFunction2D(e.src, e.dst, numPartitions, ceilSqrt) // Should we be using 3-tuple or an optimized class - (part, (e.src, e.dst, e.data)) + MessageToPartition(part, (e.src, e.dst, e.data)) // (math.abs(e.src) % numPartitions, (e.src, e.dst, e.data)) - } .partitionBy(new HashPartitioner(numPartitions)) .mapPartitionsWithIndex({ (pid, iter) => val edgePartition = new EdgePartition[ED] - iter.foreach { case (_, (src, dst, data)) => edgePartition.add(src, dst, data) } + iter.foreach { message => + val data = message.data + edgePartition.add(data._1, data._2, data._3) + } edgePartition.trim() Iterator((pid, edgePartition)) }, preservesPartitioning = true) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/MessageToPartition.scala b/graph/src/main/scala/org/apache/spark/graph/impl/MessageToPartition.scala new file mode 100644 index 0000000000000..b7bbf257a4a56 --- /dev/null +++ b/graph/src/main/scala/org/apache/spark/graph/impl/MessageToPartition.scala @@ -0,0 +1,49 @@ +package org.apache.spark.graph.impl + +import org.apache.spark.Partitioner +import org.apache.spark.graph.Pid +import org.apache.spark.rdd.{ShuffledRDD, RDD} + + +/** + * A message used to send a specific value to a partition. + * @param partition index of the target partition. + * @param data value to send + */ +class MessageToPartition[@specialized(Int, Long, Double, Char, Boolean/*, AnyRef*/) T]( + @transient var partition: Pid, + var data: T) + extends Product2[Pid, T] { + + override def _1 = partition + + override def _2 = data + + override def canEqual(that: Any): Boolean = that.isInstanceOf[MessageToPartition[_]] +} + +/** + * Companion object for MessageToPartition. + */ +object MessageToPartition { + def apply[T](partition: Pid, value: T) = new MessageToPartition(partition, value) +} + + +class MessageToPartitionRDDFunctions[T: ClassManifest](self: RDD[MessageToPartition[T]]) { + + /** + * Return a copy of the RDD partitioned using the specified partitioner. + */ + def partitionBy(partitioner: Partitioner): RDD[MessageToPartition[T]] = { + new ShuffledRDD[Pid, T, MessageToPartition[T]](self, partitioner) + } + +} + + +object MessageToPartitionRDDFunctions { + implicit def rdd2PartitionRDDFunctions[T: ClassManifest](rdd: RDD[MessageToPartition[T]]) = { + new MessageToPartitionRDDFunctions(rdd) + } +} From 5218e46178030c05306ea2dcecc1736f5f796d7c Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 7 Oct 2013 11:48:50 -0700 Subject: [PATCH 093/531] Updated Kryo registration. --- .../apache/spark/graph/GraphKryoRegistrator.scala | 14 +++++--------- 1 file changed, 5 insertions(+), 9 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphKryoRegistrator.scala b/graph/src/main/scala/org/apache/spark/graph/GraphKryoRegistrator.scala index 13a22f9051e0d..297506e1e351e 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphKryoRegistrator.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphKryoRegistrator.scala @@ -2,23 +2,19 @@ package org.apache.spark.graph import com.esotericsoftware.kryo.Kryo +import org.apache.spark.graph.impl.MessageToPartition import org.apache.spark.serializer.KryoRegistrator class GraphKryoRegistrator extends KryoRegistrator { def registerClasses(kryo: Kryo) { - //kryo.register(classOf[(Int, Float, Float)]) - registerClass[Int, Int, Int](kryo) + kryo.register(classOf[Vertex[Object]]) + kryo.register(classOf[Edge[Object]]) + kryo.register(classOf[MutableTuple2[Object, Object]]) + kryo.register(classOf[MessageToPartition[Object]]) // This avoids a large number of hash table lookups. kryo.setReferences(false) } - - private def registerClass[VD: Manifest, ED: Manifest, VD2: Manifest](kryo: Kryo) { - kryo.register(classOf[Vertex[VD]]) - kryo.register(classOf[Edge[ED]]) - kryo.register(classOf[MutableTuple2[VD, VD2]]) - kryo.register(classOf[(Vid, VD2)]) - } } From 8dfac4ea8f1e69894266e80f5ab8dff4c8acefdd Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Thu, 10 Oct 2013 14:09:01 -0700 Subject: [PATCH 094/531] added support for random vertex cuts --- .../scala/org/apache/spark/graph/impl/GraphImpl.scala | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index bcb2ae1ec8f0e..09aeade8c12be 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -466,6 +466,15 @@ object GraphImpl { } + /** + * Assign edges to an aribtrary machine corresponding to a + * random vertex cut. + */ + protected def randomVertexCut(src: Vid, dst: Vid, numParts: Pid): Pid = { + math.abs((src, dst).hashCode()) % Pid + } + + /** * @todo(crankshaw) how does this effect load balancing? */ From 5f756fb63f284de13672562d084c6e37f7b62846 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Thu, 10 Oct 2013 14:10:47 -0700 Subject: [PATCH 095/531] added support for random vertex cuts --- .../src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index 09aeade8c12be..db982a8fe4b9e 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -471,7 +471,7 @@ object GraphImpl { * random vertex cut. */ protected def randomVertexCut(src: Vid, dst: Vid, numParts: Pid): Pid = { - math.abs((src, dst).hashCode()) % Pid + math.abs((src, dst).hashCode()) % numParts } From fa2f87ca63ec1c13196ff86705de1d07cfbf0af2 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Thu, 10 Oct 2013 14:48:40 -0700 Subject: [PATCH 096/531] added replication and balance reporting --- .../main/scala/org/apache/spark/graph/Analytics.scala | 11 +++++++++++ .../src/main/scala/org/apache/spark/graph/Graph.scala | 6 ++++++ .../scala/org/apache/spark/graph/impl/GraphImpl.scala | 10 ++++++++++ 3 files changed, 27 insertions(+) diff --git a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala index 09cf81eeeb78b..b411c60cee15e 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala @@ -47,6 +47,17 @@ object Analytics extends Logging { val pagerankGraph = graph.leftJoinVertices[Int, (Int, Double)](graph.outDegrees, (vertex, deg) => (deg.getOrElse(0), 1.0) ) + + println("Vertex Replication: " + pagerankGraph.replication) + + val edgeCounts = pagerankGraph.balance + + println("Edge Balance: " + (edgeCounts.max.toDouble / edgeCounts.min ) ) + println("Min edge block: " + edgeCounts.min) + println("Max edge block: " + edgeCounts.max) + + + Pregel.iterate[(Int, Double), ED, Double](pagerankGraph)( (vertex, a: Double) => (vertex.data._1, (resetProb + (1.0 - resetProb) * a)), // apply (me_id, edge) => Some(edge.src.data._2 / edge.src.data._1), // gather diff --git a/graph/src/main/scala/org/apache/spark/graph/Graph.scala b/graph/src/main/scala/org/apache/spark/graph/Graph.scala index de0a090504547..61032bf0be309 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Graph.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Graph.scala @@ -18,6 +18,12 @@ import org.apache.spark.rdd.RDD */ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { + + def replication: Double + + def balance: Array[Int] + + /** * Get the vertices and their data. * diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index db982a8fe4b9e..e178df3841531 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -56,6 +56,16 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( this } + + override def replication(): Double = { + val rep = vTable.map{ case (_, (_, a)) => a.size }.sum + rep / vTable.count + } + + override def balance(): Array[Int] = { + eTable.map{ case (_, epart) => epart.data.size }.collect + } + override def reverse: Graph[VD, ED] = { newGraph(vertices, edges.map{ case Edge(s, t, e) => Edge(t, s, e) }) } From c4a23f95c30feda17254fee4313d07bb4503bb77 Mon Sep 17 00:00:00 2001 From: Dan Crankshaw Date: Fri, 11 Oct 2013 22:57:43 +0000 Subject: [PATCH 097/531] Updated code so benchmarks actually run. --- conf/core-site.xml | 43 ++ conf/slaves | 12 +- conf/spark-env.sh.template | 20 +- .../scala/org/apache/spark/SparkEnv.scala | 1 + .../spark/serializer/KryoSerializer.scala | 14 +- .../spark/serializer/SerializerManager.scala | 2 +- .../org/apache/spark/graph/Analytics.scala | 582 +++++++----------- .../spark/graph/GraphKryoRegistrator.scala | 4 +- .../org/apache/spark/graph/GraphLoader.scala | 2 +- .../apache/spark/graph/impl/GraphImpl.scala | 2 +- .../spark/graph/util/GraphGenerators.scala | 3 - 11 files changed, 292 insertions(+), 393 deletions(-) create mode 100644 conf/core-site.xml diff --git a/conf/core-site.xml b/conf/core-site.xml new file mode 100644 index 0000000000000..eefd875fc819b --- /dev/null +++ b/conf/core-site.xml @@ -0,0 +1,43 @@ + + + + + + + + + hadoop.tmp.dir + /mnt/ephemeral-hdfs + + + + fs.default.name + hdfs://ec2-50-17-7-68.compute-1.amazonaws.com:9000 + + + + io.file.buffer.size + 65536 + + + + dfs.client.read.shortcircuit + false + + + + dfs.client.read.shortcircuit.skip.checksum + false + + + + dfs.domain.socket.path + /var/run/hadoop-hdfs/dn._PORT + + + + dfs.client.file-block-storage-locations.timeout + 3000 + + + diff --git a/conf/slaves b/conf/slaves index da0a01343d20a..728d22ac2ebb6 100644 --- a/conf/slaves +++ b/conf/slaves @@ -1,2 +1,10 @@ -# A Spark Worker will be started on each of the machines listed below. -localhost \ No newline at end of file +ec2-23-20-12-62.compute-1.amazonaws.com +ec2-54-205-173-19.compute-1.amazonaws.com +ec2-54-225-4-124.compute-1.amazonaws.com +ec2-23-22-209-112.compute-1.amazonaws.com +ec2-50-16-69-88.compute-1.amazonaws.com +ec2-54-205-163-126.compute-1.amazonaws.com +ec2-54-242-235-95.compute-1.amazonaws.com +ec2-54-211-169-232.compute-1.amazonaws.com +ec2-54-237-31-30.compute-1.amazonaws.com +ec2-54-235-15-124.compute-1.amazonaws.com diff --git a/conf/spark-env.sh.template b/conf/spark-env.sh.template index 0a35ee7c79b12..b8936314ecce2 100755 --- a/conf/spark-env.sh.template +++ b/conf/spark-env.sh.template @@ -1,21 +1,19 @@ #!/usr/bin/env bash # This file contains environment variables required to run Spark. Copy it as -# spark-env.sh and edit that to configure Spark for your site. -# -# The following variables can be set in this file: -# - SPARK_LOCAL_IP, to set the IP address Spark binds to on this node +# spark-env.sh and edit that to configure Spark for your site. At a minimum, +# the following two variables should be set: +# - SCALA_HOME, to point to your Scala installation, or SCALA_LIBRARY_PATH to +# point to the directory for Scala library JARs (if you install Scala as a +# Debian or RPM package, these are in a separate path, often /usr/share/java) # - MESOS_NATIVE_LIBRARY, to point to your libmesos.so if you use Mesos -# - SPARK_JAVA_OPTS, to set node-specific JVM options for Spark. Note that -# we recommend setting app-wide options in the application's driver program. -# Examples of node-specific options : -Dspark.local.dir, GC options -# Examples of app-wide options : -Dspark.serializer # -# If using the standalone deploy mode, you can also set variables for it here: -# - SPARK_MASTER_IP, to bind the master to a different IP address or hostname +# If using the standalone deploy mode, you can also set variables for it: +# - SPARK_MASTER_IP, to bind the master to a different IP address # - SPARK_MASTER_PORT / SPARK_MASTER_WEBUI_PORT, to use non-default ports # - SPARK_WORKER_CORES, to set the number of cores to use on this machine # - SPARK_WORKER_MEMORY, to set how much memory to use (e.g. 1000m, 2g) # - SPARK_WORKER_PORT / SPARK_WORKER_WEBUI_PORT -# - SPARK_WORKER_INSTANCES, to set the number of worker processes per node +# - SPARK_WORKER_INSTANCES, to set the number of worker instances/processes +# to be spawned on every slave machine diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 29968c273c31d..9b8384bcbb58c 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -156,6 +156,7 @@ object SparkEnv extends Logging { val serializer = serializerManager.setDefault( System.getProperty("spark.serializer", "org.apache.spark.serializer.JavaSerializer")) + logInfo("spark.serializer is " + System.getProperty("spark.serializer")) val closureSerializer = serializerManager.get( System.getProperty("spark.closure.serializer", "org.apache.spark.serializer.JavaSerializer")) diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index 24ef204aa1990..3feafde8b6ff4 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -61,14 +61,10 @@ class KryoSerializer extends org.apache.spark.serializer.Serializer with Logging kryo.register(classOf[HttpBroadcast[_]], new KryoJavaSerializer()) // Allow the user to register their own classes by setting spark.kryo.registrator - try { - Option(System.getProperty("spark.kryo.registrator")).foreach { regCls => - logDebug("Running user registrator: " + regCls) - val reg = Class.forName(regCls, true, classLoader).newInstance().asInstanceOf[KryoRegistrator] - reg.registerClasses(kryo) - } - } catch { - case _: Exception => println("Failed to register spark.kryo.registrator") + Option(System.getProperty("spark.kryo.registrator")).foreach { regCls => + logDebug("Running user registrator: " + regCls) + val reg = Class.forName(regCls, true, classLoader).newInstance().asInstanceOf[KryoRegistrator] + reg.registerClasses(kryo) } kryo.setClassLoader(classLoader) @@ -116,7 +112,7 @@ class KryoDeserializationStream(kryo: Kryo, inStream: InputStream) extends Deser } } -private[spark] class KryoSerializerInstance(ks: KryoSerializer) extends SerializerInstance { +private[spark] class KryoSerializerInstance(ks: KryoSerializer) extends SerializerInstance with Logging { val kryo = ks.newKryo() val output = ks.newKryoOutput() val input = ks.newKryoInput() diff --git a/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala b/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala index 2955986feced5..5082730ae3fa3 100644 --- a/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala +++ b/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala @@ -25,7 +25,7 @@ import java.util.concurrent.ConcurrentHashMap * instance of the serializer object has been created, the get method returns that instead of * creating a new one. */ -private[spark] class SerializerManager { +private[spark] class SerializerManager extends org.apache.spark.Logging { private val serializers = new ConcurrentHashMap[String, Serializer] private var _default: Serializer = _ diff --git a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala index b411c60cee15e..454df22a0805b 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala @@ -6,37 +6,6 @@ import org.apache.spark._ object Analytics extends Logging { -// def main(args: Array[String]) { -// //pregelPagerank() -// } - - // /** - // * Compute the PageRank of a graph returning the pagerank of each vertex as an RDD - // */ - // // def pagerank[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], numIter: Int) = { - // // // Compute the out degree of each vertex - // // val pagerankGraph = graph.updateVertices[Int, (Int, Float)](graph.outDegrees, - // // (vertex, deg) => (deg.getOrElse(0), 1.0F) - // // ) - // // GraphLab.iterateGA[(Int, Float), ED, Float](pagerankGraph)( - // // (me_id, edge) => edge.src.data._2 / edge.src.data._1, // gather - // // (a: Float, b: Float) => a + b, // merge - // // (vertex, a: Option[Float]) => (vertex.data._1, (0.15F + 0.85F * a.getOrElse(0F))), // apply - // // numIter).mapVertices{ case Vertex(id, (outDeg, r)) => Vertex(id, r) } - // // } - // def pagerank[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], numIter: Int) = { - // // Compute the out degree of each vertex - // val pagerankGraph = graph.updateVertices[Int, (Int, Double)](graph.outDegrees, - // (vertex, deg) => (deg.getOrElse(0), 1.0) - // ) - // GraphLab.iterateGA2[(Int, Double), ED, Double](pagerankGraph)( - // (me_id, edge) => edge.src.data._2 / edge.src.data._1, // gather - // (a: Double, b: Double) => a + b, // merge - // 0.0, // default - // (vertex, a: Double) => (vertex.data._1, (0.15 + 0.85 * a)), // apply - // numIter).mapVertices{ case Vertex(id, (outDeg, r)) => Vertex(id, r) } - // } - /** * Compute the PageRank of a graph returning the pagerank of each vertex as an RDD */ @@ -106,335 +75,220 @@ object Analytics extends Logging { gatherDirection = EdgeDirection.Both, scatterDirection = EdgeDirection.Both ) } - - // /** - // * Compute the shortest path to a set of markers - // */ - // def shortestPath[VD: Manifest](graph: Graph[VD, Float], sources: List[Int], numIter: Int) = { - // val sourceSet = sources.toSet - // val spGraph = graph.mapVertices { - // case Vertex(vid, _) => Vertex(vid, (if(sourceSet.contains(vid)) 0.0F else Float.MaxValue)) - // } - // GraphLab.iterateGA[Float, Float, Float](spGraph)( - // (me_id, edge) => edge.otherVertex(me_id).data + edge.data, // gather - // (a: Float, b: Float) => math.min(a, b), // merge - // (v, a: Option[Float]) => math.min(v.data, a.getOrElse(Float.MaxValue)), // apply - // numIter, - // gatherDirection = EdgeDirection.In) - // } - - // // /** - // // * Compute the connected component membership of each vertex - // // * and return an RDD with the vertex value containing the - // // * lowest vertex id in the connected component containing - // // * that vertex. - // // */ - // // def dynamicConnectedComponents[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], - // // numIter: Int = Int.MaxValue) = { - - // // val vertices = graph.vertices.mapPartitions(iter => iter.map { case (vid, _) => (vid, vid) }) - // // val edges = graph.edges // .mapValues(v => None) - // // val ccGraph = new Graph(vertices, edges) - - // // ccGraph.iterateDynamic( - // // (me_id, edge) => edge.otherVertex(me_id).data, // gather - // // (a: Int, b: Int) => math.min(a, b), // merge - // // Integer.MAX_VALUE, - // // (v, a: Int) => math.min(v.data, a), // apply - // // (me_id, edge) => edge.otherVertex(me_id).data > edge.vertex(me_id).data, // scatter - // // numIter, - // // gatherEdges = EdgeDirection.Both, - // // scatterEdges = EdgeDirection.Both).vertices - // // // - // // // graph_ret.vertices.collect.foreach(println) - // // // graph_ret.edges.take(10).foreach(println) - // // } - - - // // /** - // // * Compute the shortest path to a set of markers - // // */ - // // def dynamicShortestPath[VD: Manifest, ED: Manifest](graph: Graph[VD, Float], - // // sources: List[Int], numIter: Int) = { - // // val sourceSet = sources.toSet - // // val vertices = graph.vertices.mapPartitions( - // // iter => iter.map { - // // case (vid, _) => (vid, (if(sourceSet.contains(vid)) 0.0F else Float.MaxValue) ) - // // }); - - // // val edges = graph.edges // .mapValues(v => None) - // // val spGraph = new Graph(vertices, edges) - - // // val niterations = Int.MaxValue - // // spGraph.iterateDynamic( - // // (me_id, edge) => edge.otherVertex(me_id).data + edge.data, // gather - // // (a: Float, b: Float) => math.min(a, b), // merge - // // Float.MaxValue, - // // (v, a: Float) => math.min(v.data, a), // apply - // // (me_id, edge) => edge.vertex(me_id).data + edge.data < edge.otherVertex(me_id).data, // scatter - // // numIter, - // // gatherEdges = EdgeDirection.In, - // // scatterEdges = EdgeDirection.Out).vertices - // // } - - - // // /** - // // * - // // */ - // // def alternatingLeastSquares[VD: ClassManifest, ED: ClassManifest](graph: Graph[VD, Double], - // // latentK: Int, lambda: Double, numIter: Int) = { - // // val vertices = graph.vertices.mapPartitions( _.map { - // // case (vid, _) => (vid, Array.fill(latentK){ scala.util.Random.nextDouble() } ) - // // }).cache - // // val maxUser = graph.edges.map(_._1).reduce(math.max(_,_)) - // // val edges = graph.edges // .mapValues(v => None) - // // val alsGraph = new Graph(vertices, edges) - // // alsGraph.numVPart = graph.numVPart - // // alsGraph.numEPart = graph.numEPart - - // // val niterations = Int.MaxValue - // // alsGraph.iterateDynamic[(Array[Double], Array[Double])]( - // // (me_id, edge) => { // gather - // // val X = edge.otherVertex(me_id).data - // // val y = edge.data - // // val Xy = X.map(_ * y) - // // val XtX = (for(i <- 0 until latentK; j <- i until latentK) yield(X(i) * X(j))).toArray - // // (Xy, XtX) - // // }, - // // (a, b) => { - // // // The difference between the while loop and the zip is a FACTOR OF TWO in overall - // // // runtime - // // var i = 0 - // // while(i < a._1.length) { a._1(i) += b._1(i); i += 1 } - // // i = 0 - // // while(i < a._2.length) { a._2(i) += b._2(i); i += 1 } - // // a - // // // (a._1.zip(b._1).map{ case (q,r) => q+r }, a._2.zip(b._2).map{ case (q,r) => q+r }) - // // }, - // // (Array.empty[Double], Array.empty[Double]), // default value is empty - // // (vertex, accum) => { // apply - // // val XyArray = accum._1 - // // val XtXArray = accum._2 - // // if(XyArray.isEmpty) vertex.data // no neighbors - // // else { - // // val XtX = DenseMatrix.tabulate(latentK,latentK){ (i,j) => - // // (if(i < j) XtXArray(i + (j+1)*j/2) else XtXArray(i + (j+1)*j/2)) + - // // (if(i == j) lambda else 1.0F) //regularization - // // } - // // val Xy = DenseMatrix.create(latentK,1,XyArray) - // // val w = XtX \ Xy - // // w.data - // // } - // // }, - // // (me_id, edge) => true, - // // numIter, - // // gatherEdges = EdgeDirection.Both, - // // scatterEdges = EdgeDirection.Both, - // // vertex => vertex.id < maxUser).vertices - // // } - - // def main(args: Array[String]) = { - // val host = args(0) - // val taskType = args(1) - // val fname = args(2) - // val options = args.drop(3).map { arg => - // arg.dropWhile(_ == '-').split('=') match { - // case Array(opt, v) => (opt -> v) - // case _ => throw new IllegalArgumentException("Invalid argument: " + arg) - // } - // } - - // System.setProperty("spark.serializer", "spark.KryoSerializer") - // //System.setProperty("spark.shuffle.compress", "false") - // System.setProperty("spark.kryo.registrator", "spark.graph.GraphKryoRegistrator") - - // taskType match { - // case "pagerank" => { - - // var numIter = Int.MaxValue - // var isDynamic = false - // var tol:Float = 0.001F - // var outFname = "" - // var numVPart = 4 - // var numEPart = 4 - - // options.foreach{ - // case ("numIter", v) => numIter = v.toInt - // case ("dynamic", v) => isDynamic = v.toBoolean - // case ("tol", v) => tol = v.toFloat - // case ("output", v) => outFname = v - // case ("numVPart", v) => numVPart = v.toInt - // case ("numEPart", v) => numEPart = v.toInt - // case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) - // } - - // if(!isDynamic && numIter == Int.MaxValue) { - // println("Set number of iterations!") - // sys.exit(1) - // } - // println("======================================") - // println("| PageRank |") - // println("--------------------------------------") - // println(" Using parameters:") - // println(" \tDynamic: " + isDynamic) - // if(isDynamic) println(" \t |-> Tolerance: " + tol) - // println(" \tNumIter: " + numIter) - // println("======================================") - - // val sc = new SparkContext(host, "PageRank(" + fname + ")") - - // val graph = Graph.textFile(sc, fname, a => 1.0F).withPartitioner(numVPart, numEPart).cache() - - // val startTime = System.currentTimeMillis - // logInfo("GRAPHX: starting tasks") - // logInfo("GRAPHX: Number of vertices " + graph.vertices.count) - // logInfo("GRAPHX: Number of edges " + graph.edges.count) - - // val pr = Analytics.pagerank(graph, numIter) - // // val pr = if(isDynamic) Analytics.dynamicPagerank(graph, tol, numIter) - // // else Analytics.pagerank(graph, numIter) - // logInfo("GRAPHX: Total rank: " + pr.vertices.map{ case Vertex(id,r) => r }.reduce(_+_) ) - // if (!outFname.isEmpty) { - // println("Saving pageranks of pages to " + outFname) - // pr.vertices.map{case Vertex(id, r) => id + "\t" + r}.saveAsTextFile(outFname) - // } - // logInfo("GRAPHX: Runtime: " + ((System.currentTimeMillis - startTime)/1000.0) + " seconds") - // sc.stop() - // } - - // case "cc" => { - - // var numIter = Int.MaxValue - // var isDynamic = false - - // options.foreach{ - // case ("numIter", v) => numIter = v.toInt - // case ("dynamic", v) => isDynamic = v.toBoolean - // case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) - // } - - // 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 + ")") - // val graph = Graph.textFile(sc, fname, a => 1.0F) - // val cc = Analytics.connectedComponents(graph, numIter) - // // val cc = if(isDynamic) Analytics.dynamicConnectedComponents(graph, numIter) - // // else Analytics.connectedComponents(graph, numIter) - // println("Components: " + cc.vertices.map(_.data).distinct()) - - // sc.stop() - // } - - // case "shortestpath" => { - - // var numIter = Int.MaxValue - // var isDynamic = true - // var sources: List[Int] = List.empty - - // options.foreach{ - // case ("numIter", v) => numIter = v.toInt - // case ("dynamic", v) => isDynamic = v.toBoolean - // case ("source", v) => sources ++= List(v.toInt) - // case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) - // } - - - // if(!isDynamic && numIter == Int.MaxValue) { - // println("Set number of iterations!") - // sys.exit(1) - // } - - // if(sources.isEmpty) { - // println("No sources provided!") - // sys.exit(1) - // } - - // println("======================================") - // println("| Shortest Path |") - // println("--------------------------------------") - // println(" Using parameters:") - // println(" \tDynamic: " + isDynamic) - // println(" \tNumIter: " + numIter) - // println(" \tSources: [" + sources.mkString(", ") + "]") - // println("======================================") - - // val sc = new SparkContext(host, "ShortestPath(" + fname + ")") - // val graph = Graph.textFile(sc, fname, a => (if(a.isEmpty) 1.0F else a(0).toFloat ) ) - // val sp = Analytics.shortestPath(graph, sources, numIter) - // // val cc = if(isDynamic) Analytics.dynamicShortestPath(graph, sources, numIter) - // // else Analytics.shortestPath(graph, sources, numIter) - // println("Longest Path: " + sp.vertices.map(_.data).reduce(math.max(_,_))) - - // sc.stop() - // } - - - // // case "als" => { - - // // var numIter = 5 - // // var lambda = 0.01 - // // var latentK = 10 - // // var usersFname = "usersFactors.tsv" - // // var moviesFname = "moviesFname.tsv" - // // var numVPart = 4 - // // var numEPart = 4 - - // // options.foreach{ - // // case ("numIter", v) => numIter = v.toInt - // // case ("lambda", v) => lambda = v.toDouble - // // case ("latentK", v) => latentK = v.toInt - // // case ("usersFname", v) => usersFname = v - // // case ("moviesFname", v) => moviesFname = v - // // case ("numVPart", v) => numVPart = v.toInt - // // case ("numEPart", v) => numEPart = v.toInt - // // case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) - // // } - - // // println("======================================") - // // println("| Alternating Least Squares |") - // // println("--------------------------------------") - // // println(" Using parameters:") - // // println(" \tNumIter: " + numIter) - // // println(" \tLambda: " + lambda) - // // println(" \tLatentK: " + latentK) - // // println(" \tusersFname: " + usersFname) - // // println(" \tmoviesFname: " + moviesFname) - // // println("======================================") - - // // val sc = new SparkContext(host, "ALS(" + fname + ")") - // // val graph = Graph.textFile(sc, fname, a => a(0).toDouble ) - // // graph.numVPart = numVPart - // // graph.numEPart = numEPart - - // // val maxUser = graph.edges.map(_._1).reduce(math.max(_,_)) - // // val minMovie = graph.edges.map(_._2).reduce(math.min(_,_)) - // // assert(maxUser < minMovie) - - // // val factors = Analytics.alternatingLeastSquares(graph, latentK, lambda, numIter).cache - // // factors.filter(_._1 <= maxUser).map(r => r._1 + "\t" + r._2.mkString("\t")) - // // .saveAsTextFile(usersFname) - // // factors.filter(_._1 >= minMovie).map(r => r._1 + "\t" + r._2.mkString("\t")) - // // .saveAsTextFile(moviesFname) - - // // sc.stop() - // // } - - - // case _ => { - // println("Invalid task type.") - // } - // } - // } + def main(args: Array[String]) = { + val host = args(0) + val taskType = args(1) + val fname = args(2) + val options = args.drop(3).map { arg => + arg.dropWhile(_ == '-').split('=') match { + case Array(opt, v) => (opt -> v) + case _ => throw new IllegalArgumentException("Invalid argument: " + arg) + } + } + + def setLogLevels(level: org.apache.log4j.Level, loggers: TraversableOnce[String]) = { + loggers.map{ + loggerName => + val logger = org.apache.log4j.Logger.getLogger(loggerName) + val prevLevel = logger.getLevel() + logger.setLevel(level) + loggerName -> prevLevel + }.toMap + } +// setLogLevels(org.apache.log4j.Level.DEBUG, Seq("org.apache.spark")) + + val serializer = "org.apache.spark.serializer.KryoSerializer" + System.setProperty("spark.serializer", serializer) + //System.setProperty("spark.shuffle.compress", "false") + System.setProperty("spark.kryo.registrator", "org.apache.spark.graph.GraphKryoRegistrator") + + taskType match { + case "pagerank" => { + + var numIter = Int.MaxValue + var isDynamic = false + var tol:Float = 0.001F + var outFname = "" + var numVPart = 4 + var numEPart = 4 + + options.foreach{ + case ("numIter", v) => numIter = v.toInt + case ("dynamic", v) => isDynamic = v.toBoolean + case ("tol", v) => tol = v.toFloat + case ("output", v) => outFname = v + case ("numVPart", v) => numVPart = v.toInt + case ("numEPart", v) => numEPart = v.toInt + case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) + } + + if(!isDynamic && numIter == Int.MaxValue) { + println("Set number of iterations!") + sys.exit(1) + } + println("======================================") + println("| PageRank |") + println("--------------------------------------") + println(" Using parameters:") + println(" \tDynamic: " + isDynamic) + if(isDynamic) println(" \t |-> Tolerance: " + tol) + println(" \tNumIter: " + numIter) + println("======================================") + + val sc = new SparkContext(host, "PageRank(" + fname + ")") + + val graph = GraphLoader.textFile(sc, fname, a => 1.0F, numEPart).withPartitioner(numVPart, numEPart).cache() + + val startTime = System.currentTimeMillis + logInfo("GRAPHX: starting tasks") + logInfo("GRAPHX: Number of vertices " + graph.vertices.count) + logInfo("GRAPHX: Number of edges " + graph.edges.count) + + val pr = Analytics.pagerank(graph, numIter) + // val pr = if(isDynamic) Analytics.dynamicPagerank(graph, tol, numIter) + // else Analytics.pagerank(graph, numIter) + logInfo("GRAPHX: Total rank: " + pr.vertices.map{ case Vertex(id,r) => r }.reduce(_+_) ) + if (!outFname.isEmpty) { + println("Saving pageranks of pages to " + outFname) + pr.vertices.map{case Vertex(id, r) => id + "\t" + r}.saveAsTextFile(outFname) + } + logInfo("GRAPHX: Runtime: " + ((System.currentTimeMillis - startTime)/1000.0) + " seconds") + sc.stop() + } + +// case "cc" => { +// +// var numIter = Int.MaxValue +// var isDynamic = false +// +// options.foreach{ +// case ("numIter", v) => numIter = v.toInt +// case ("dynamic", v) => isDynamic = v.toBoolean +// case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) +// } +// +// 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 + ")") +// val graph = GraphLoader.textFile(sc, fname, a => 1.0F) +// //val cc = Analytics.connectedComponents(graph, numIter) +// // val cc = if(isDynamic) Analytics.dynamicConnectedComponents(graph, numIter) +// // else Analytics.connectedComponents(graph, numIter) +// println("Components: " + cc.vertices.map(_.data).distinct()) +// +// sc.stop() +// } +// +// case "shortestpath" => { +// +// var numIter = Int.MaxValue +// var isDynamic = true +// var sources: List[Int] = List.empty +// +// options.foreach{ +// case ("numIter", v) => numIter = v.toInt +// case ("dynamic", v) => isDynamic = v.toBoolean +// case ("source", v) => sources ++= List(v.toInt) +// case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) +// } +// +// +// if(!isDynamic && numIter == Int.MaxValue) { +// println("Set number of iterations!") +// sys.exit(1) +// } +// +// if(sources.isEmpty) { +// println("No sources provided!") +// sys.exit(1) +// } +// +// println("======================================") +// println("| Shortest Path |") +// println("--------------------------------------") +// println(" Using parameters:") +// println(" \tDynamic: " + isDynamic) +// println(" \tNumIter: " + numIter) +// println(" \tSources: [" + sources.mkString(", ") + "]") +// println("======================================") +// +// val sc = new SparkContext(host, "ShortestPath(" + fname + ")") +// val graph = GraphLoader.textFile(sc, fname, a => (if(a.isEmpty) 1.0F else a(0).toFloat ) ) +// //val sp = Analytics.shortestPath(graph, sources, numIter) +// // val cc = if(isDynamic) Analytics.dynamicShortestPath(graph, sources, numIter) +// // else Analytics.shortestPath(graph, sources, numIter) +// println("Longest Path: " + sp.vertices.map(_.data).reduce(math.max(_,_))) +// +// sc.stop() +// } + + + // case "als" => { + + // var numIter = 5 + // var lambda = 0.01 + // var latentK = 10 + // var usersFname = "usersFactors.tsv" + // var moviesFname = "moviesFname.tsv" + // var numVPart = 4 + // var numEPart = 4 + + // options.foreach{ + // case ("numIter", v) => numIter = v.toInt + // case ("lambda", v) => lambda = v.toDouble + // case ("latentK", v) => latentK = v.toInt + // case ("usersFname", v) => usersFname = v + // case ("moviesFname", v) => moviesFname = v + // case ("numVPart", v) => numVPart = v.toInt + // case ("numEPart", v) => numEPart = v.toInt + // case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) + // } + + // println("======================================") + // println("| Alternating Least Squares |") + // println("--------------------------------------") + // println(" Using parameters:") + // println(" \tNumIter: " + numIter) + // println(" \tLambda: " + lambda) + // println(" \tLatentK: " + latentK) + // println(" \tusersFname: " + usersFname) + // println(" \tmoviesFname: " + moviesFname) + // println("======================================") + + // val sc = new SparkContext(host, "ALS(" + fname + ")") + // val graph = GraphLoader.textFile(sc, fname, a => a(0).toDouble ) + // graph.numVPart = numVPart + // graph.numEPart = numEPart + + // val maxUser = graph.edges.map(_._1).reduce(math.max(_,_)) + // val minMovie = graph.edges.map(_._2).reduce(math.min(_,_)) + // assert(maxUser < minMovie) + + // val factors = Analytics.alternatingLeastSquares(graph, latentK, lambda, numIter).cache + // factors.filter(_._1 <= maxUser).map(r => r._1 + "\t" + r._2.mkString("\t")) + // .saveAsTextFile(usersFname) + // factors.filter(_._1 >= minMovie).map(r => r._1 + "\t" + r._2.mkString("\t")) + // .saveAsTextFile(moviesFname) + + // sc.stop() + // } + + + case _ => { + println("Invalid task type.") + } + } + } // /** // * Compute the PageRank of a graph returning the pagerank of each vertex as an RDD @@ -648,7 +502,7 @@ object Analytics extends Logging { // val sc = new SparkContext(host, "PageRank(" + fname + ")") - // val graph = Graph.textFile(sc, fname, a => 1.0).withPartitioner(numVPart, numEPart).cache() + // val graph = GraphLoader.textFile(sc, fname, a => 1.0).withPartitioner(numVPart, numEPart).cache() // val startTime = System.currentTimeMillis // logInfo("GRAPHX: starting tasks") @@ -691,7 +545,7 @@ object Analytics extends Logging { // println("======================================") // val sc = new SparkContext(host, "ConnectedComponents(" + fname + ")") - // val graph = Graph.textFile(sc, fname, a => 1.0) + // val graph = GraphLoader.textFile(sc, fname, a => 1.0) // val cc = Analytics.connectedComponents(graph, numIter) // // val cc = if(isDynamic) Analytics.dynamicConnectedComponents(graph, numIter) // // else Analytics.connectedComponents(graph, numIter) @@ -734,7 +588,7 @@ object Analytics extends Logging { // println("======================================") // val sc = new SparkContext(host, "ShortestPath(" + fname + ")") - // val graph = Graph.textFile(sc, fname, a => (if(a.isEmpty) 1.0 else a(0).toDouble ) ) + // val graph = GraphLoader.textFile(sc, fname, a => (if(a.isEmpty) 1.0 else a(0).toDouble ) ) // val sp = Analytics.shortestPath(graph, sources, numIter) // // val cc = if(isDynamic) Analytics.dynamicShortestPath(graph, sources, numIter) // // else Analytics.shortestPath(graph, sources, numIter) @@ -777,7 +631,7 @@ object Analytics extends Logging { // println("======================================") // val sc = new SparkContext(host, "ALS(" + fname + ")") - // val graph = Graph.textFile(sc, fname, a => a(0).toDouble ) + // val graph = GraphLoader.textFile(sc, fname, a => a(0).toDouble ) // graph.numVPart = numVPart // graph.numEPart = numEPart diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphKryoRegistrator.scala b/graph/src/main/scala/org/apache/spark/graph/GraphKryoRegistrator.scala index 297506e1e351e..2d72789878c1a 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphKryoRegistrator.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphKryoRegistrator.scala @@ -4,7 +4,7 @@ import com.esotericsoftware.kryo.Kryo import org.apache.spark.graph.impl.MessageToPartition import org.apache.spark.serializer.KryoRegistrator - +import org.apache.spark.graph.impl._ class GraphKryoRegistrator extends KryoRegistrator { @@ -13,6 +13,8 @@ class GraphKryoRegistrator extends KryoRegistrator { kryo.register(classOf[Edge[Object]]) kryo.register(classOf[MutableTuple2[Object, Object]]) kryo.register(classOf[MessageToPartition[Object]]) + kryo.register(classOf[(Vid, Object)]) + kryo.register(classOf[EdgePartition[Object]]) // This avoids a large number of hash table lookups. kryo.setReferences(false) diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala b/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala index 4d7ca1268d36e..bcc73691df0ff 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala @@ -20,7 +20,7 @@ object GraphLoader { : GraphImpl[Int, ED] = { // Parse the edge data table - val edges = sc.textFile(path).flatMap { line => + val edges = sc.textFile(path, minEdgePartitions).flatMap { line => if (!line.isEmpty && line(0) != '#') { val lineArray = line.split("\\s+") if(lineArray.length < 2) { diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index e178df3841531..9f5002208962f 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -516,7 +516,7 @@ object GraphImpl { .map { e => // Random partitioning based on the source vertex id. // val part: Pid = edgePartitionFunction1D(e.src, e.dst, numPartitions) - val part: Pid = edgePartitionFunction2D(e.src, e.dst, numPartitions, ceilSqrt) + val part: Pid = randomVertexCut(e.src, e.dst, numPartitions) //val part: Pid = canonicalEdgePartitionFunction2D(e.src, e.dst, numPartitions, ceilSqrt) // Should we be using 3-tuple or an optimized class diff --git a/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala b/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala index d0583c48a8597..a393cb504e377 100644 --- a/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala +++ b/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala @@ -63,9 +63,6 @@ object GraphGenerators { } - // For now just writes graph to a file. Eventually - // it will return a spark.graph.Graph - // Right now it just generates a bunch of edges where // the edge data is the weight (default 1) From 543a54dffa2fce817112452d76a65ea61cefb8f9 Mon Sep 17 00:00:00 2001 From: Dan Crankshaw Date: Fri, 11 Oct 2013 16:07:49 -0700 Subject: [PATCH 098/531] Tried to fix some indenting --- .../org/apache/spark/graph/Analytics.scala | 263 +++++++++--------- 1 file changed, 132 insertions(+), 131 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala index 454df22a0805b..b8c2f5186e9e9 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala @@ -75,87 +75,88 @@ object Analytics extends Logging { gatherDirection = EdgeDirection.Both, scatterDirection = EdgeDirection.Both ) } - def main(args: Array[String]) = { - val host = args(0) - val taskType = args(1) - val fname = args(2) - val options = args.drop(3).map { arg => - arg.dropWhile(_ == '-').split('=') match { - case Array(opt, v) => (opt -> v) - case _ => throw new IllegalArgumentException("Invalid argument: " + arg) - } - } - - def setLogLevels(level: org.apache.log4j.Level, loggers: TraversableOnce[String]) = { - loggers.map{ - loggerName => - val logger = org.apache.log4j.Logger.getLogger(loggerName) - val prevLevel = logger.getLevel() - logger.setLevel(level) - loggerName -> prevLevel - }.toMap - } + + def main(args: Array[String]) = { + val host = args(0) + val taskType = args(1) + val fname = args(2) + val options = args.drop(3).map { arg => + arg.dropWhile(_ == '-').split('=') match { + case Array(opt, v) => (opt -> v) + case _ => throw new IllegalArgumentException("Invalid argument: " + arg) + } + } + + def setLogLevels(level: org.apache.log4j.Level, loggers: TraversableOnce[String]) = { + loggers.map{ + loggerName => + val logger = org.apache.log4j.Logger.getLogger(loggerName) + val prevLevel = logger.getLevel() + logger.setLevel(level) + loggerName -> prevLevel + }.toMap + } // setLogLevels(org.apache.log4j.Level.DEBUG, Seq("org.apache.spark")) - val serializer = "org.apache.spark.serializer.KryoSerializer" - System.setProperty("spark.serializer", serializer) - //System.setProperty("spark.shuffle.compress", "false") - System.setProperty("spark.kryo.registrator", "org.apache.spark.graph.GraphKryoRegistrator") - - taskType match { - case "pagerank" => { - - var numIter = Int.MaxValue - var isDynamic = false - var tol:Float = 0.001F - var outFname = "" - var numVPart = 4 - var numEPart = 4 - - options.foreach{ - case ("numIter", v) => numIter = v.toInt - case ("dynamic", v) => isDynamic = v.toBoolean - case ("tol", v) => tol = v.toFloat - case ("output", v) => outFname = v - case ("numVPart", v) => numVPart = v.toInt - case ("numEPart", v) => numEPart = v.toInt - case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) - } - - if(!isDynamic && numIter == Int.MaxValue) { - println("Set number of iterations!") - sys.exit(1) - } - println("======================================") - println("| PageRank |") - println("--------------------------------------") - println(" Using parameters:") - println(" \tDynamic: " + isDynamic) - if(isDynamic) println(" \t |-> Tolerance: " + tol) - println(" \tNumIter: " + numIter) - println("======================================") - - val sc = new SparkContext(host, "PageRank(" + fname + ")") - - val graph = GraphLoader.textFile(sc, fname, a => 1.0F, numEPart).withPartitioner(numVPart, numEPart).cache() - - val startTime = System.currentTimeMillis - logInfo("GRAPHX: starting tasks") - logInfo("GRAPHX: Number of vertices " + graph.vertices.count) - logInfo("GRAPHX: Number of edges " + graph.edges.count) - - val pr = Analytics.pagerank(graph, numIter) - // val pr = if(isDynamic) Analytics.dynamicPagerank(graph, tol, numIter) - // else Analytics.pagerank(graph, numIter) - logInfo("GRAPHX: Total rank: " + pr.vertices.map{ case Vertex(id,r) => r }.reduce(_+_) ) - if (!outFname.isEmpty) { - println("Saving pageranks of pages to " + outFname) - pr.vertices.map{case Vertex(id, r) => id + "\t" + r}.saveAsTextFile(outFname) - } - logInfo("GRAPHX: Runtime: " + ((System.currentTimeMillis - startTime)/1000.0) + " seconds") - sc.stop() + val serializer = "org.apache.spark.serializer.KryoSerializer" + System.setProperty("spark.serializer", serializer) + //System.setProperty("spark.shuffle.compress", "false") + System.setProperty("spark.kryo.registrator", "org.apache.spark.graph.GraphKryoRegistrator") + + taskType match { + case "pagerank" => { + + var numIter = Int.MaxValue + var isDynamic = false + var tol:Float = 0.001F + var outFname = "" + var numVPart = 4 + var numEPart = 4 + + options.foreach{ + case ("numIter", v) => numIter = v.toInt + case ("dynamic", v) => isDynamic = v.toBoolean + case ("tol", v) => tol = v.toFloat + case ("output", v) => outFname = v + case ("numVPart", v) => numVPart = v.toInt + case ("numEPart", v) => numEPart = v.toInt + case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) } + if(!isDynamic && numIter == Int.MaxValue) { + println("Set number of iterations!") + sys.exit(1) + } + println("======================================") + println("| PageRank |") + println("--------------------------------------") + println(" Using parameters:") + println(" \tDynamic: " + isDynamic) + if(isDynamic) println(" \t |-> Tolerance: " + tol) + println(" \tNumIter: " + numIter) + println("======================================") + + val sc = new SparkContext(host, "PageRank(" + fname + ")") + + val graph = GraphLoader.textFile(sc, fname, a => 1.0F, numEPart).withPartitioner(numVPart, numEPart).cache() + + val startTime = System.currentTimeMillis + logInfo("GRAPHX: starting tasks") + logInfo("GRAPHX: Number of vertices " + graph.vertices.count) + logInfo("GRAPHX: Number of edges " + graph.edges.count) + + val pr = Analytics.pagerank(graph, numIter) + // val pr = if(isDynamic) Analytics.dynamicPagerank(graph, tol, numIter) + // else Analytics.pagerank(graph, numIter) + logInfo("GRAPHX: Total rank: " + pr.vertices.map{ case Vertex(id,r) => r }.reduce(_+_) ) + if (!outFname.isEmpty) { + println("Saving pageranks of pages to " + outFname) + pr.vertices.map{case Vertex(id, r) => id + "\t" + r}.saveAsTextFile(outFname) + } + logInfo("GRAPHX: Runtime: " + ((System.currentTimeMillis - startTime)/1000.0) + " seconds") + sc.stop() + } + // case "cc" => { // // var numIter = Int.MaxValue @@ -233,62 +234,62 @@ object Analytics extends Logging { // } - // case "als" => { - - // var numIter = 5 - // var lambda = 0.01 - // var latentK = 10 - // var usersFname = "usersFactors.tsv" - // var moviesFname = "moviesFname.tsv" - // var numVPart = 4 - // var numEPart = 4 - - // options.foreach{ - // case ("numIter", v) => numIter = v.toInt - // case ("lambda", v) => lambda = v.toDouble - // case ("latentK", v) => latentK = v.toInt - // case ("usersFname", v) => usersFname = v - // case ("moviesFname", v) => moviesFname = v - // case ("numVPart", v) => numVPart = v.toInt - // case ("numEPart", v) => numEPart = v.toInt - // case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) - // } - - // println("======================================") - // println("| Alternating Least Squares |") - // println("--------------------------------------") - // println(" Using parameters:") - // println(" \tNumIter: " + numIter) - // println(" \tLambda: " + lambda) - // println(" \tLatentK: " + latentK) - // println(" \tusersFname: " + usersFname) - // println(" \tmoviesFname: " + moviesFname) - // println("======================================") - - // val sc = new SparkContext(host, "ALS(" + fname + ")") - // val graph = GraphLoader.textFile(sc, fname, a => a(0).toDouble ) - // graph.numVPart = numVPart - // graph.numEPart = numEPart - - // val maxUser = graph.edges.map(_._1).reduce(math.max(_,_)) - // val minMovie = graph.edges.map(_._2).reduce(math.min(_,_)) - // assert(maxUser < minMovie) - - // val factors = Analytics.alternatingLeastSquares(graph, latentK, lambda, numIter).cache - // factors.filter(_._1 <= maxUser).map(r => r._1 + "\t" + r._2.mkString("\t")) - // .saveAsTextFile(usersFname) - // factors.filter(_._1 >= minMovie).map(r => r._1 + "\t" + r._2.mkString("\t")) - // .saveAsTextFile(moviesFname) - - // sc.stop() - // } - - - case _ => { - println("Invalid task type.") - } + // case "als" => { + + // var numIter = 5 + // var lambda = 0.01 + // var latentK = 10 + // var usersFname = "usersFactors.tsv" + // var moviesFname = "moviesFname.tsv" + // var numVPart = 4 + // var numEPart = 4 + + // options.foreach{ + // case ("numIter", v) => numIter = v.toInt + // case ("lambda", v) => lambda = v.toDouble + // case ("latentK", v) => latentK = v.toInt + // case ("usersFname", v) => usersFname = v + // case ("moviesFname", v) => moviesFname = v + // case ("numVPart", v) => numVPart = v.toInt + // case ("numEPart", v) => numEPart = v.toInt + // case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) + // } + + // println("======================================") + // println("| Alternating Least Squares |") + // println("--------------------------------------") + // println(" Using parameters:") + // println(" \tNumIter: " + numIter) + // println(" \tLambda: " + lambda) + // println(" \tLatentK: " + latentK) + // println(" \tusersFname: " + usersFname) + // println(" \tmoviesFname: " + moviesFname) + // println("======================================") + + // val sc = new SparkContext(host, "ALS(" + fname + ")") + // val graph = GraphLoader.textFile(sc, fname, a => a(0).toDouble ) + // graph.numVPart = numVPart + // graph.numEPart = numEPart + + // val maxUser = graph.edges.map(_._1).reduce(math.max(_,_)) + // val minMovie = graph.edges.map(_._2).reduce(math.min(_,_)) + // assert(maxUser < minMovie) + + // val factors = Analytics.alternatingLeastSquares(graph, latentK, lambda, numIter).cache + // factors.filter(_._1 <= maxUser).map(r => r._1 + "\t" + r._2.mkString("\t")) + // .saveAsTextFile(usersFname) + // factors.filter(_._1 >= minMovie).map(r => r._1 + "\t" + r._2.mkString("\t")) + // .saveAsTextFile(moviesFname) + + // sc.stop() + // } + + + case _ => { + println("Invalid task type.") } } + } // /** // * Compute the PageRank of a graph returning the pagerank of each vertex as an RDD From 1e5535cfcf3717f76e990a3800b07a05ddafe2e2 Mon Sep 17 00:00:00 2001 From: Dan Crankshaw Date: Fri, 11 Oct 2013 16:38:52 -0700 Subject: [PATCH 099/531] Added connected components back --- .../org/apache/spark/graph/Analytics.scala | 67 ++++++++++--------- 1 file changed, 34 insertions(+), 33 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala index b8c2f5186e9e9..59af021c8d04d 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala @@ -64,7 +64,7 @@ object Analytics extends Logging { * lowest vertex id in the connected component containing * that vertex. */ - def connectedComponents[VD: Manifest, ED: Manifest](graph: Graph[VD, ED]) = { + def connectedComponents[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], numIter: Int) = { val ccGraph = graph.mapVertices { case Vertex(vid, _) => vid } GraphLab.iterate(ccGraph)( @@ -72,6 +72,7 @@ object Analytics extends Logging { (a: Vid, b: Vid) => math.min(a, b), // merge (v, a: Option[Vid]) => math.min(v.data, a.getOrElse(Long.MaxValue)), // apply (me_id, edge) => (edge.vertex(me_id).data < edge.otherVertex(me_id).data), // scatter + numIter, gatherDirection = EdgeDirection.Both, scatterDirection = EdgeDirection.Both ) } @@ -157,38 +158,38 @@ object Analytics extends Logging { sc.stop() } -// case "cc" => { -// -// var numIter = Int.MaxValue -// var isDynamic = false -// -// options.foreach{ -// case ("numIter", v) => numIter = v.toInt -// case ("dynamic", v) => isDynamic = v.toBoolean -// case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) -// } -// -// 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 + ")") -// val graph = GraphLoader.textFile(sc, fname, a => 1.0F) -// //val cc = Analytics.connectedComponents(graph, numIter) -// // val cc = if(isDynamic) Analytics.dynamicConnectedComponents(graph, numIter) -// // else Analytics.connectedComponents(graph, numIter) -// println("Components: " + cc.vertices.map(_.data).distinct()) -// -// sc.stop() -// } + case "cc" => { + + var numIter = Int.MaxValue + var isDynamic = false + + options.foreach{ + case ("numIter", v) => numIter = v.toInt + case ("dynamic", v) => isDynamic = v.toBoolean + case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) + } + + 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 + ")") + val graph = GraphLoader.textFile(sc, fname, a => 1.0F) + val cc = Analytics.connectedComponents(graph, numIter) + //val cc = if(isDynamic) Analytics.dynamicConnectedComponents(graph, numIter) + // else Analytics.connectedComponents(graph, numIter) + println("Components: " + cc.vertices.map(_.data).distinct()) + + sc.stop() + } // // case "shortestpath" => { // From 1a961dd1f27ce94a5f79faa60a746bde21470682 Mon Sep 17 00:00:00 2001 From: Dan Crankshaw Date: Sat, 12 Oct 2013 01:47:38 +0000 Subject: [PATCH 100/531] Fixed connected components CL params --- .../src/main/scala/org/apache/spark/graph/Analytics.scala | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala index 59af021c8d04d..6aa6f50317cc3 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala @@ -161,11 +161,15 @@ object Analytics extends Logging { case "cc" => { var numIter = Int.MaxValue + var numVPart = 4 + var numEPart = 4 var isDynamic = false options.foreach{ case ("numIter", v) => numIter = v.toInt case ("dynamic", v) => isDynamic = v.toBoolean + case ("numEPart", v) => numEPart = v.toInt + case ("numVPart", v) => numVPart = v.toInt case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) } @@ -182,7 +186,8 @@ object Analytics extends Logging { println("======================================") val sc = new SparkContext(host, "ConnectedComponents(" + fname + ")") - val graph = GraphLoader.textFile(sc, fname, a => 1.0F) + //val graph = GraphLoader.textFile(sc, fname, a => 1.0F) + val graph = GraphLoader.textFile(sc, fname, a => 1.0F, numEPart).withPartitioner(numVPart, numEPart).cache() val cc = Analytics.connectedComponents(graph, numIter) //val cc = if(isDynamic) Analytics.dynamicConnectedComponents(graph, numIter) // else Analytics.connectedComponents(graph, numIter) From 494472a6ccba6aa60305f0ba7e59657f0f980709 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Sun, 13 Oct 2013 19:42:32 -0700 Subject: [PATCH 101/531] Integrated IndexedRDD into graph design. --- .../apache/spark/IndexedRDDFunctions.scala | 26 + .../org/apache/spark/rdd/IndexedRDD.scala | 5 + .../apache/spark/rdd/PairRDDFunctions.scala | 9 + .../org/apache/spark/graph/Analytics.scala | 27 +- .../scala/org/apache/spark/graph/Graph.scala | 146 ++-- .../org/apache/spark/graph/GraphLab.scala | 37 +- .../org/apache/spark/graph/GraphLoader.scala | 4 +- .../org/apache/spark/graph/GraphOps.scala | 20 +- .../scala/org/apache/spark/graph/Pregel.scala | 27 +- .../spark/graph/impl/EdgePartition.scala | 55 +- .../graph/impl/EdgePartitionBuilder.scala | 31 + .../spark/graph/impl/EdgeTripletRDD.scala | 224 +++--- .../apache/spark/graph/impl/GraphImpl.scala | 686 ++++++++++-------- .../spark/graph/util/GraphGenerators.scala | 22 +- 14 files changed, 764 insertions(+), 555 deletions(-) create mode 100644 graph/src/main/scala/org/apache/spark/graph/impl/EdgePartitionBuilder.scala diff --git a/core/src/main/scala/org/apache/spark/IndexedRDDFunctions.scala b/core/src/main/scala/org/apache/spark/IndexedRDDFunctions.scala index 65c6963b71565..358ab57b0c9e5 100644 --- a/core/src/main/scala/org/apache/spark/IndexedRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/IndexedRDDFunctions.scala @@ -49,6 +49,32 @@ class IndexedRDDFunctions[K: ClassManifest, V: ClassManifest](self: IndexedRDD[K } + /** + * Pass each value in the key-value pair RDD through a map function without changing the keys; + * this also retains the original RDD's partitioning. + */ + override def mapValuesWithKeys[U: ClassManifest](f: (K, V) => U): RDD[(K, U)] = { + val cleanF = self.index.rdd.context.clean(f) + val newValues = self.index.rdd.zipPartitions(self.valuesRDD){ (keysIter, valuesIter) => + val index = keysIter.next() + assert(keysIter.hasNext() == false) + val oldValues = valuesIter.next() + assert(valuesIter.hasNext() == false) + // Allocate the array to store the results into + val newValues: Array[Seq[U]] = new Array[Seq[U]](oldValues.size) + // Populate the new Values + for( (k,i) <- index ) { + if(oldValues(i) != null) { + newValues(i) = oldValues(i).map( v => f(k,v) ) + } + } + Array(newValues.toSeq).iterator + } + new IndexedRDD[K,U](self.index, newValues) + } + + + /** * Pass each value in the key-value pair RDD through a flatMap function without changing the * keys; this also retains the original RDD's partitioning. diff --git a/core/src/main/scala/org/apache/spark/rdd/IndexedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/IndexedRDD.scala index 79a007a939fe6..8d2e9782c2936 100644 --- a/core/src/main/scala/org/apache/spark/rdd/IndexedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/IndexedRDD.scala @@ -53,6 +53,8 @@ class RDDIndex[@specialized K: ClassManifest](private[spark] val rdd: RDD[BlockI rdd.persist(newLevel) return this } + + def partitioner: Partitioner = rdd.partitioner.get } @@ -85,6 +87,9 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( override val partitioner = index.rdd.partitioner + + + /** * The actual partitions are defined by the tuples. */ diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 435ddb9e94790..569d74ae7a6a1 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -393,6 +393,15 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) new MappedValuesRDD(self, cleanF) } + + /** + * Pass each value in the key-value pair RDD through a map function without changing the keys; + * this also retains the original RDD's partitioning. + */ + def mapValuesWithKeys[U: ClassManifest](f: (K, V) => U): RDD[(K, U)] = { + self.map{ case (k,v) => (k, f(k,v)) } + } + /** * Pass each value in the key-value pair RDD through a flatMap function without changing the * keys; this also retains the original RDD's partitioning. diff --git a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala index b411c60cee15e..49498fbcd4805 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala @@ -44,9 +44,9 @@ object Analytics extends Logging { numIter: Int, resetProb: Double = 0.15) = { // Compute the out degree of each vertex - val pagerankGraph = graph.leftJoinVertices[Int, (Int, Double)](graph.outDegrees, - (vertex, deg) => (deg.getOrElse(0), 1.0) - ) + val pagerankGraph = graph.outerJoinVertices(graph.outDegrees){ + (vid, vdata, deg) => (deg.getOrElse(0), 1.0) + } println("Vertex Replication: " + pagerankGraph.replication) @@ -59,11 +59,11 @@ object Analytics extends Logging { Pregel.iterate[(Int, Double), ED, Double](pagerankGraph)( - (vertex, a: Double) => (vertex.data._1, (resetProb + (1.0 - resetProb) * a)), // apply + (vid, data, a: Double) => (data._1, (resetProb + (1.0 - resetProb) * a)), // apply (me_id, edge) => Some(edge.src.data._2 / edge.src.data._1), // gather (a: Double, b: Double) => a + b, // merge 1.0, - numIter).mapVertices{ case Vertex(id, (outDeg, r)) => r } + numIter).mapVertices{ case (id, (outDeg, r)) => r } } /** @@ -74,18 +74,19 @@ object Analytics extends Logging { maxIter: Int = Integer.MAX_VALUE, resetProb: Double = 0.15) = { // Compute the out degree of each vertex - val pagerankGraph = graph.leftJoinVertices[Int, (Int, Double, Double)](graph.outDegrees, - (vertex, degIter) => (degIter.sum, 1.0, 1.0) - ) + val pagerankGraph = graph.outerJoinVertices(graph.outDegrees){ + (id, data, degIter) => (degIter.sum, 1.0, 1.0) + } + // Run PageRank GraphLab.iterate(pagerankGraph)( (me_id, edge) => edge.src.data._2 / edge.src.data._1, // gather (a: Double, b: Double) => a + b, - (vertex, a: Option[Double]) => - (vertex.data._1, (resetProb + (1.0 - resetProb) * a.getOrElse(0.0)), vertex.data._2), // apply + (id, data, a: Option[Double]) => + (data._1, (resetProb + (1.0 - resetProb) * a.getOrElse(0.0)), data._2), // apply (me_id, edge) => math.abs(edge.src.data._3 - edge.src.data._2) > tol, // scatter - maxIter).mapVertices { case Vertex(vid, data) => data._2 } + maxIter).mapVertices { case (vid, data) => data._2 } } @@ -96,12 +97,12 @@ object Analytics extends Logging { * that vertex. */ def connectedComponents[VD: Manifest, ED: Manifest](graph: Graph[VD, ED]) = { - val ccGraph = graph.mapVertices { case Vertex(vid, _) => vid } + val ccGraph = graph.mapVertices { case (vid, _) => vid } GraphLab.iterate(ccGraph)( (me_id, edge) => edge.otherVertex(me_id).data, // gather (a: Vid, b: Vid) => math.min(a, b), // merge - (v, a: Option[Vid]) => math.min(v.data, a.getOrElse(Long.MaxValue)), // apply + (id, data, a: Option[Vid]) => math.min(data, a.getOrElse(Long.MaxValue)), // apply (me_id, edge) => (edge.vertex(me_id).data < edge.otherVertex(me_id).data), // scatter gatherDirection = EdgeDirection.Both, scatterDirection = EdgeDirection.Both ) diff --git a/graph/src/main/scala/org/apache/spark/graph/Graph.scala b/graph/src/main/scala/org/apache/spark/graph/Graph.scala index 61032bf0be309..39c699ce8bae2 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Graph.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Graph.scala @@ -2,6 +2,7 @@ package org.apache.spark.graph import org.apache.spark.rdd.RDD +import org.apache.spark.util.ClosureCleaner @@ -33,7 +34,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * * @todo should vertices return tuples instead of vertex objects? */ - def vertices: RDD[Vertex[VD]] + def vertices: RDD[(Vid,VD)] /** * Get the Edges and their data as an RDD. The entries in the RDD contain @@ -101,7 +102,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * }}} * */ - def mapVertices[VD2: ClassManifest](map: Vertex[VD] => VD2): Graph[VD2, ED] + def mapVertices[VD2: ClassManifest](map: (Vid, VD) => VD2): Graph[VD2, ED] /** * Construct a new graph where each the value of each edge is transformed by @@ -149,13 +150,13 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { map: EdgeTriplet[VD, ED] => ED2): Graph[VD, ED2] - /** - * Remove edges conntecting vertices that are not in the graph. - * - * @todo remove this function and ensure that for a graph G=(V,E): - * if (u,v) in E then u in V and v in V - */ - def correctEdges(): Graph[VD, ED] + // /** + // * Remove edges conntecting vertices that are not in the graph. + // * + // * @todo remove this function and ensure that for a graph G=(V,E): + // * if (u,v) in E then u in V and v in V + // */ + // def correctEdges(): Graph[VD, ED] /** * Construct a new graph with all the edges reversed. If this graph contains @@ -183,8 +184,8 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * @return the subgraph containing only the vertices and edges that satisfy the * predicates. */ - def subgraph(epred: EdgeTriplet[VD,ED] => Boolean = (_ => true), - vpred: Vertex[VD] => Boolean = (_ => true) ): Graph[VD, ED] + def subgraph(epred: EdgeTriplet[VD,ED] => Boolean = (x => true), + vpred: (Vid, VD) => Boolean = ((v,d) => true) ): Graph[VD, ED] // /** @@ -200,51 +201,55 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { + def mapReduceTriplets[A: ClassManifest]( + mapFunc: EdgeTriplet[VD, ED] => Array[(Vid, A)], + reduceFunc: (A, A) => A) + : RDD[(Vid, A)] - /** - * This function is used to compute a statistic for the neighborhood of each - * vertex. - * - * This is one of the core functions in the Graph API in that enables - * neighborhood level computation. For example this function can be used to - * count neighbors satisfying a predicate or implement PageRank. - * - * @note The returned RDD may contain fewer entries than their are vertices - * in the graph. This is because some vertices may not have neighbors or the - * map function may return None for all neighbors. - * - * @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 mergeFunc 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 VD2 The returned type of the aggregation operation. - * - * @return A Spark.RDD containing tuples of vertex identifiers and thee - * resulting value. Note that the returned RDD may contain fewer vertices - * than in the original graph since some vertices may not have neighbors or - * the map function could return None for all neighbors. - * - * @example We can use this function to compute the average follower age for - * each user - * {{{ - * val graph: Graph[Int,Int] = loadGraph() - * val averageFollowerAge: RDD[(Int, Int)] = - * graph.aggregateNeighbors[(Int,Double)]( - * (vid, edge) => (edge.otherVertex(vid).data, 1), - * (a, b) => (a._1 + b._1, a._2 + b._2), - * EdgeDirection.In) - * .mapValues{ case (sum,followers) => sum.toDouble / followers} - * }}} - * - */ - def aggregateNeighbors[A: ClassManifest]( - mapFunc: (Vid, EdgeTriplet[VD, ED]) => Option[A], - mergeFunc: (A, A) => A, - direction: EdgeDirection) - : Graph[(VD, Option[A]), ED] + // /** + // * This function is used to compute a statistic for the neighborhood of each + // * vertex. + // * + // * This is one of the core functions in the Graph API in that enables + // * neighborhood level computation. For example this function can be used to + // * count neighbors satisfying a predicate or implement PageRank. + // * + // * @note The returned RDD may contain fewer entries than their are vertices + // * in the graph. This is because some vertices may not have neighbors or the + // * map function may return None for all neighbors. + // * + // * @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 mergeFunc 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 VD2 The returned type of the aggregation operation. + // * + // * @return A Spark.RDD containing tuples of vertex identifiers and thee + // * resulting value. Note that the returned RDD may contain fewer vertices + // * than in the original graph since some vertices may not have neighbors or + // * the map function could return None for all neighbors. + // * + // * @example We can use this function to compute the average follower age for + // * each user + // * {{{ + // * val graph: Graph[Int,Int] = loadGraph() + // * val averageFollowerAge: RDD[(Int, Int)] = + // * graph.aggregateNeighbors[(Int,Double)]( + // * (vid, edge) => (edge.otherVertex(vid).data, 1), + // * (a, b) => (a._1 + b._1, a._2 + b._2), + // * EdgeDirection.In) + // * .mapValues{ case (sum,followers) => sum.toDouble / followers} + // * }}} + // * + // */ + // def aggregateNeighbors[A: ClassManifest]( + // mapFunc: (Vid, EdgeTriplet[VD, ED]) => Option[A], + // mergeFunc: (A, A) => A, + // direction: EdgeDirection) + // : Graph[(VD, Option[A]), ED] /** * This function is used to compute a statistic for the neighborhood of each @@ -291,9 +296,8 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { def aggregateNeighbors[A: ClassManifest]( mapFunc: (Vid, EdgeTriplet[VD, ED]) => Option[A], reduceFunc: (A, A) => A, - default: A, // Should this be a function or a value? direction: EdgeDirection) - : Graph[(VD, Option[A]), ED] + : RDD[(Vid, A)] /** @@ -328,9 +332,8 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * }}} * @todo Is leftJoinVertices the right name? */ - def leftJoinVertices[U: ClassManifest, VD2: ClassManifest]( - table: RDD[(Vid, U)], - mapFunc: (Vertex[VD], Option[U]) => VD2) + def outerJoinVertices[U: ClassManifest, VD2: ClassManifest](table: RDD[(Vid, U)]) + (mapFunc: (Vid, VD, Option[U]) => VD2) : Graph[VD2, ED] /** @@ -366,10 +369,15 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * graph.joinVertices(tbl)( (v, row) => row ) * }}} */ - def joinVertices[U: ClassManifest]( - table: RDD[(Vid, U)], - mapFunc: (Vertex[VD], U) => VD) - : Graph[VD, ED] + def joinVertices[U: ClassManifest](table: RDD[(Vid, U)])(mapFunc: (Vid, VD, U) => VD) + : Graph[VD, ED] = { + ClosureCleaner.clean(mapFunc) + def uf(id: Vid, data: VD, o: Option[U]): VD = o match { + case Some(u) => mapFunc(id, data, u) + case None => data + } + outerJoinVertices(table)(uf) + } // 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. @@ -391,16 +399,16 @@ object Graph { rawEdges.map { case (s, t) => Edge(s, t, 1) } } // Determine unique vertices - val vertices: RDD[Vertex[Int]] = edges.flatMap{ case Edge(s, t, cnt) => Array((s, 1), (t, 1)) } - .reduceByKey(_ + _) - .map{ case (id, deg) => Vertex(id, deg) } + val vertices: RDD[(Vid, Int)] = + edges.flatMap{ case Edge(s, t, cnt) => Array((s, 1), (t, 1)) }.reduceByKey(_ + _) + // Return graph - new GraphImpl(vertices, edges) + GraphImpl(vertices, edges) } def apply[VD: ClassManifest, ED: ClassManifest]( - vertices: RDD[Vertex[VD]], edges: RDD[Edge[ED]]): Graph[VD, ED] = { - new GraphImpl(vertices, edges) + vertices: RDD[(Vid,VD)], edges: RDD[Edge[ED]]): Graph[VD, ED] = { + GraphImpl(vertices, edges) } implicit def graphToGraphOps[VD: ClassManifest, ED: ClassManifest](g: Graph[VD, ED]) = g.ops diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala b/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala index 01f24a13024c7..ccb1bd8e5dab4 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala @@ -36,7 +36,7 @@ object GraphLab { def iterate[VD: ClassManifest, ED: ClassManifest, A: ClassManifest](graph: Graph[VD, ED])( gatherFunc: (Vid, EdgeTriplet[VD, ED]) => A, mergeFunc: (A, A) => A, - applyFunc: (Vertex[VD], Option[A]) => VD, + applyFunc: (Vid, VD, Option[A]) => VD, scatterFunc: (Vid, EdgeTriplet[VD, ED]) => Boolean, numIter: Int = Integer.MAX_VALUE, gatherDirection: EdgeDirection = EdgeDirection.In, @@ -45,7 +45,7 @@ object GraphLab { // Add an active attribute to all vertices to track convergence. var activeGraph: Graph[(Boolean, VD), ED] = graph.mapVertices { - case Vertex(id, data) => (true, data) + case (id, data) => (true, data) }.cache() // The gather function wrapper strips the active attribute and @@ -64,9 +64,9 @@ object GraphLab { // The apply function wrapper strips the vertex of the active attribute // and only invokes the apply function on active vertices - def apply(v: Vertex[((Boolean, VD), Option[A])]): (Boolean, VD) = { - val ((active, vData), accum) = v.data - if (active) (true, applyFunc(Vertex(v.id, vData), accum)) + def apply(vid: Vid, data: (Boolean, VD), accum: Option[A]): (Boolean, VD) = { + val (active, vData) = data + if (active) (true, applyFunc(vid, vData, accum)) else (false, vData) } @@ -89,9 +89,9 @@ object GraphLab { } // Used to set the active status of vertices for the next round - def applyActive(v: Vertex[((Boolean, VD), Option[Boolean])]): (Boolean, VD) = { - val ((prevActive, vData), newActive) = v.data - (newActive.getOrElse(false), vData) + def applyActive(vid: Vid, data: (Boolean, VD), newActive: Boolean): (Boolean, VD) = { + val (prevActive, vData) = data + (newActive, vData) } // Main Loop --------------------------------------------------------------------- @@ -99,29 +99,32 @@ object GraphLab { var numActive = activeGraph.numVertices while (i < numIter && numActive > 0) { - val gathered: Graph[((Boolean, VD), Option[A]), ED] = + // Gather + val gathered: RDD[(Vid, A)] = activeGraph.aggregateNeighbors(gather, mergeFunc, gatherDirection) - val applied: Graph[(Boolean, VD), ED] = gathered.mapVertices(apply).cache() + // Apply + activeGraph = activeGraph.outerJoinVertices(gathered)(apply).cache() - activeGraph = applied.cache() + // Scatter is basically a gather in the opposite direction so we reverse the edge direction // activeGraph: Graph[(Boolean, VD), ED] - val scattered: Graph[((Boolean, VD), Option[Boolean]), ED] = + val scattered: RDD[(Vid, Boolean)] = activeGraph.aggregateNeighbors(scatter, _ || _, scatterDirection.reverse) - val newActiveGraph: Graph[(Boolean, VD), ED] = - scattered.mapVertices(applyActive) - activeGraph = newActiveGraph.cache() + activeGraph = activeGraph.joinVertices(scattered)(applyActive).cache() - numActive = activeGraph.vertices.map(v => if (v.data._1) 1 else 0).reduce(_ + _) + // Calculate the number of active vertices + numActive = activeGraph.vertices.map{ + case (vid, data) => if (data._1) 1 else 0 + }.reduce(_ + _) println("Number active vertices: " + numActive) i += 1 } // Remove the active attribute from the vertex data before returning the graph - activeGraph.mapVertices(v => v.data._2) + activeGraph.mapVertices{case (vid, data) => data._2 } } } diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala b/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala index 4d7ca1268d36e..903e407b2d712 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala @@ -48,7 +48,7 @@ object GraphLoader { def fromEdges[ED: ClassManifest](edges: RDD[Edge[ED]]): GraphImpl[Int, ED] = { val vertices = edges.flatMap { edge => List((edge.src, 1), (edge.dst, 1)) } .reduceByKey(_ + _) - .map{ case (vid, degree) => Vertex(vid, degree) } - new GraphImpl[Int, ED](vertices, edges) + .map{ case (vid, degree) => (vid, degree) } + GraphImpl(vertices, edges) } } diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala b/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala index 9e8cc0a6d52a7..23c783ba3a80f 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala @@ -1,7 +1,7 @@ package org.apache.spark.graph import org.apache.spark.rdd.RDD - +import org.apache.spark.SparkContext._ class GraphOps[VD: ClassManifest, ED: ClassManifest](g: Graph[VD, ED]) { @@ -16,22 +16,18 @@ class GraphOps[VD: ClassManifest, ED: ClassManifest](g: Graph[VD, ED]) { lazy val degrees: RDD[(Vid, Int)] = degreesRDD(EdgeDirection.Both) def collectNeighborIds(edgeDirection: EdgeDirection) : RDD[(Vid, Array[Vid])] = { - val graph: Graph[(VD, Option[Array[Vid]]), ED] = g.aggregateNeighbors( + val nbrs = g.aggregateNeighbors[Array[Vid]]( (vid, edge) => Some(Array(edge.otherVertex(vid).id)), (a, b) => a ++ b, edgeDirection) - graph.vertices.map(v => { - val (_, neighborIds) = v.data - (v.id, neighborIds.getOrElse(Array())) - }) + + g.vertices.leftOuterJoin(nbrs).mapValues{ + case (_, Some(nbrs)) => nbrs + case (_, None) => Array.empty[Vid] + } } private def degreesRDD(edgeDirection: EdgeDirection): RDD[(Vid, Int)] = { - val degreeGraph: Graph[(VD, Option[Int]), ED] = - g.aggregateNeighbors((vid, edge) => Some(1), _+_, edgeDirection) - degreeGraph.vertices.map(v => { - val (_, degree) = v.data - (v.id, degree.getOrElse(0)) - }) + g.aggregateNeighbors((vid, edge) => Some(1), _+_, edgeDirection) } } diff --git a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala index 09bcc67c8ced9..93c9c09ee3d0d 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala @@ -6,7 +6,7 @@ import org.apache.spark.rdd.RDD object Pregel { def iterate[VD: ClassManifest, ED: ClassManifest, A: ClassManifest](graph: Graph[VD, ED])( - vprog: (Vertex[VD], A) => VD, + vprog: (Vid, VD, A) => VD, sendMsg: (Vid, EdgeTriplet[VD, ED]) => Option[A], mergeMsg: (A, A) => A, initialMsg: A, @@ -19,25 +19,26 @@ object Pregel { def mapF(vid: Vid, edge: EdgeTriplet[VD,ED]) = sendMsg(edge.otherVertex(vid).id, edge) - def runProg(vertexWithMsgs: Vertex[(VD, Option[A])]): VD = { - val (vData, msg) = vertexWithMsgs.data - val v = Vertex(vertexWithMsgs.id, vData) + def runProg(id: Vid, data: (VD, Option[A]) ): VD = { + val (vData, msg) = data msg match { - case Some(m) => vprog(v, m) - case None => v.data + case Some(m) => vprog(id, vData, m) + case None => vData } } - var graphWithMsgs: Graph[(VD, Option[A]), ED] = - g.mapVertices(v => (v.data, Some(initialMsg))) + // Receive the first set of messages + g.mapVertices( (vid, vdata) => vprog(vid, vdata, initialMsg)) while (i < numIter) { - val newGraph: Graph[VD, ED] = graphWithMsgs.mapVertices(runProg).cache() - graphWithMsgs = newGraph.aggregateNeighbors(mapF, mergeMsg, EdgeDirection.In) + // compute the messages + val messages = g.aggregateNeighbors(mapF, mergeMsg, EdgeDirection.In) + // receive the messages + g = g.joinVertices(messages)(vprog) + // count the iteration i += 1 } - graphWithMsgs.mapVertices(vertexWithMsgs => vertexWithMsgs.data match { - case (vData, _) => vData - }) + // Return the final graph + g } } diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala b/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala index 3d218f27b1850..f0d9080d97597 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala @@ -1,9 +1,6 @@ package org.apache.spark.graph.impl import scala.collection.mutable.ArrayBuilder - -import it.unimi.dsi.fastutil.ints.IntArrayList - import org.apache.spark.graph._ @@ -11,29 +8,43 @@ import org.apache.spark.graph._ * A partition of edges in 3 large columnar arrays. */ private[graph] -class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED: ClassManifest] { +class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED: ClassManifest]( + val srcIds: Array[Vid], + val dstIds: Array[Vid], + val data: Array[ED] + ){ - private var _data: Array[ED] = _ - private var _dataBuilder = ArrayBuilder.make[ED] + // private var _data: Array[ED] = _ + // private var _dataBuilder = ArrayBuilder.make[ED] - val srcIds = new VertexArrayList - val dstIds = new VertexArrayList + // var srcIds = new VertexArrayList + // var dstIds = new VertexArrayList - def data: Array[ED] = _data + def reverse: EdgePartition[ED] = new EdgePartition(dstIds, srcIds, data) - /** Add a new edge to the partition. */ - def add(src: Vid, dst: Vid, d: ED) { - srcIds.add(src) - dstIds.add(dst) - _dataBuilder += d + def map[ED2: ClassManifest](f: Edge[ED] => ED2): EdgePartition[ED2] = { + val newData = new Array[ED2](data.size) + val edge = new Edge[ED]() + for(i <- 0 until data.size){ + edge.src = srcIds(i) + edge.dst = dstIds(i) + edge.data = data(i) + newData(i) = f(edge) + } + new EdgePartition(srcIds, dstIds, newData) } - def trim() { - srcIds.trim() - dstIds.trim() - _data = _dataBuilder.result() + def foreach(f: Edge[ED] => Unit) { + val edge = new Edge[ED] + for(i <- 0 until data.size){ + edge.src = srcIds(i) + edge.dst = dstIds(i) + edge.data = data(i) + f(edge) + } } + def size: Int = srcIds.size def iterator = new Iterator[Edge[ED]] { @@ -43,11 +54,13 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) override def hasNext: Boolean = pos < EdgePartition.this.size override def next(): Edge[ED] = { - edge.src = srcIds.get(pos) - edge.dst = dstIds.get(pos) - edge.data = _data(pos) + edge.src = srcIds(pos) + edge.dst = dstIds(pos) + edge.data = data(pos) pos += 1 edge } } } + + diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartitionBuilder.scala b/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartitionBuilder.scala new file mode 100644 index 0000000000000..f2d07d55c629d --- /dev/null +++ b/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartitionBuilder.scala @@ -0,0 +1,31 @@ +package org.apache.spark.graph.impl + +import scala.collection.mutable.ArrayBuilder +import org.apache.spark.graph._ + + +private[graph] +class EdgePartitionBuilder[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) +ED: ClassManifest]{ + val srcIds = new VertexArrayList + val dstIds = new VertexArrayList + var dataBuilder = ArrayBuilder.make[ED] + + + /** Add a new edge to the partition. */ + def add(src: Vid, dst: Vid, d: ED) { + srcIds.add(src) + dstIds.add(dst) + dataBuilder += d + } + + def toEdgePartition: EdgePartition[ED] = { + new EdgePartition(srcIds.toLongArray(), dstIds.toLongArray(), dataBuilder.result()) + } + + +} + + + + diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/EdgeTripletRDD.scala b/graph/src/main/scala/org/apache/spark/graph/impl/EdgeTripletRDD.scala index 1cd48120a1aa4..6779f4aa093ff 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/EdgeTripletRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/EdgeTripletRDD.scala @@ -1,112 +1,112 @@ -package org.apache.spark.graph.impl - -import scala.collection.mutable - -import org.apache.spark.Aggregator -import org.apache.spark.Partition -import org.apache.spark.SparkEnv -import org.apache.spark.TaskContext -import org.apache.spark.rdd.RDD -import org.apache.spark.Dependency -import org.apache.spark.OneToOneDependency -import org.apache.spark.ShuffleDependency -import org.apache.spark.SparkContext._ -import org.apache.spark.graph._ - - -private[graph] -class EdgeTripletPartition(idx: Int, val vPart: Partition, val ePart: Partition) - extends Partition { - override val index: Int = idx - override def hashCode(): Int = idx -} - - -/** - * A RDD that brings together edge data with its associated vertex data. - */ -private[graph] -class EdgeTripletRDD[VD: ClassManifest, ED: ClassManifest]( - vTableReplicated: RDD[(Vid, VD)], - eTable: RDD[(Pid, EdgePartition[ED])]) - extends RDD[(VertexHashMap[VD], Iterator[EdgeTriplet[VD, ED]])](eTable.context, Nil) { - - //println("ddshfkdfhds" + vTableReplicated.partitioner.get.numPartitions) - //println("9757984589347598734549" + eTable.partitioner.get.numPartitions) - - assert(vTableReplicated.partitioner == eTable.partitioner) - - override def getDependencies: List[Dependency[_]] = { - List(new OneToOneDependency(eTable), new OneToOneDependency(vTableReplicated)) - } - - override def getPartitions = Array.tabulate[Partition](eTable.partitions.size) { - i => new EdgeTripletPartition(i, eTable.partitions(i), vTableReplicated.partitions(i)) - } - - override val partitioner = eTable.partitioner - - override def getPreferredLocations(s: Partition) = - eTable.preferredLocations(s.asInstanceOf[EdgeTripletPartition].ePart) - - override def compute(s: Partition, context: TaskContext) - : Iterator[(VertexHashMap[VD], Iterator[EdgeTriplet[VD, ED]])] = { - - val split = s.asInstanceOf[EdgeTripletPartition] - - // Fetch the vertices and put them in a hashmap. - // TODO: use primitive hashmaps for primitive VD types. - val vmap = new VertexHashMap[VD]//(1000000) - vTableReplicated.iterator(split.vPart, context).foreach { v => vmap.put(v._1, v._2) } - - val (pid, edgePartition) = eTable.iterator(split.ePart, context).next() - .asInstanceOf[(Pid, EdgePartition[ED])] - - // Return an iterator that looks up the hash map to find matching vertices for each edge. - val iter = new Iterator[EdgeTriplet[VD, ED]] { - private var pos = 0 - private val e = new EdgeTriplet[VD, ED] - e.src = new Vertex[VD] - e.dst = new Vertex[VD] - - override def hasNext: Boolean = pos < edgePartition.size - override def next() = { - e.src.id = edgePartition.srcIds.getLong(pos) - // assert(vmap.containsKey(e.src.id)) - e.src.data = vmap.get(e.src.id) - - e.dst.id = edgePartition.dstIds.getLong(pos) - // assert(vmap.containsKey(e.dst.id)) - e.dst.data = vmap.get(e.dst.id) - - //println("Iter called: " + pos) - e.data = edgePartition.data(pos) - pos += 1 - e - } - - override def toList: List[EdgeTriplet[VD, ED]] = { - val lb = new mutable.ListBuffer[EdgeTriplet[VD,ED]] - for (i <- (0 until edgePartition.size)) { - val currentEdge = new EdgeTriplet[VD, ED] - currentEdge.src = new Vertex[VD] - currentEdge.dst = new Vertex[VD] - currentEdge.src.id = edgePartition.srcIds.getLong(i) - // assert(vmap.containsKey(e.src.id)) - currentEdge.src.data = vmap.get(currentEdge.src.id) - - currentEdge.dst.id = edgePartition.dstIds.getLong(i) - // assert(vmap.containsKey(e.dst.id)) - currentEdge.dst.data = vmap.get(currentEdge.dst.id) - - currentEdge.data = edgePartition.data(i) - //println("Iter: " + pos + " " + e.src.id + " " + e.dst.id + " " + e.data) - //println("List: " + i + " " + currentEdge.src.id + " " + currentEdge.dst.id + " " + currentEdge.data) - lb += currentEdge - } - lb.toList - } - } - Iterator((vmap, iter)) - } -} +// package org.apache.spark.graph.impl + +// import scala.collection.mutable + +// import org.apache.spark.Aggregator +// import org.apache.spark.Partition +// import org.apache.spark.SparkEnv +// import org.apache.spark.TaskContext +// import org.apache.spark.rdd.RDD +// import org.apache.spark.Dependency +// import org.apache.spark.OneToOneDependency +// import org.apache.spark.ShuffleDependency +// import org.apache.spark.SparkContext._ +// import org.apache.spark.graph._ + + +// private[graph] +// class EdgeTripletPartition(idx: Int, val vPart: Partition, val ePart: Partition) +// extends Partition { +// override val index: Int = idx +// override def hashCode(): Int = idx +// } + + +// /** +// * A RDD that brings together edge data with its associated vertex data. +// */ +// private[graph] +// class EdgeTripletRDD[VD: ClassManifest, ED: ClassManifest]( +// vTableReplicated: IndexedRDD[Pid, VertexHashMap[VD]], +// eTable: IndexedRDD[Pid, EdgePartition[ED]]) +// extends RDD[(VertexHashMap[VD], Iterator[EdgeTriplet[VD, ED]])](eTable.context, Nil) { + +// //println("ddshfkdfhds" + vTableReplicated.partitioner.get.numPartitions) +// //println("9757984589347598734549" + eTable.partitioner.get.numPartitions) + +// assert(vTableReplicated.partitioner == eTable.partitioner) + +// override def getDependencies: List[Dependency[_]] = { +// List(new OneToOneDependency(eTable), new OneToOneDependency(vTableReplicated)) +// } + +// override def getPartitions = Array.tabulate[Partition](eTable.partitions.size) { +// i => new EdgeTripletPartition(i, eTable.partitions(i), vTableReplicated.partitions(i)) +// } + +// override val partitioner = eTable.partitioner + +// override def getPreferredLocations(s: Partition) = +// eTable.preferredLocations(s.asInstanceOf[EdgeTripletPartition].ePart) + +// override def compute(s: Partition, context: TaskContext) +// : Iterator[(VertexHashMap[VD], Iterator[EdgeTriplet[VD, ED]])] = { + +// val split = s.asInstanceOf[EdgeTripletPartition] + +// // Fetch the vertices and put them in a hashmap. +// // TODO: use primitive hashmaps for primitive VD types. +// val vmap = new VertexHashMap[VD]//(1000000) +// vTableReplicated.iterator(split.vPart, context).foreach { v => vmap.put(v._1, v._2) } + +// val (pid, edgePartition) = eTable.iterator(split.ePart, context).next() +// .asInstanceOf[(Pid, EdgePartition[ED])] + +// // Return an iterator that looks up the hash map to find matching vertices for each edge. +// val iter = new Iterator[EdgeTriplet[VD, ED]] { +// private var pos = 0 +// private val e = new EdgeTriplet[VD, ED] +// e.src = new Vertex[VD] +// e.dst = new Vertex[VD] + +// override def hasNext: Boolean = pos < edgePartition.size +// override def next() = { +// e.src.id = edgePartition.srcIds.getLong(pos) +// // assert(vmap.containsKey(e.src.id)) +// e.src.data = vmap.get(e.src.id) + +// e.dst.id = edgePartition.dstIds.getLong(pos) +// // assert(vmap.containsKey(e.dst.id)) +// e.dst.data = vmap.get(e.dst.id) + +// //println("Iter called: " + pos) +// e.data = edgePartition.data(pos) +// pos += 1 +// e +// } + +// override def toList: List[EdgeTriplet[VD, ED]] = { +// val lb = new mutable.ListBuffer[EdgeTriplet[VD,ED]] +// for (i <- (0 until edgePartition.size)) { +// val currentEdge = new EdgeTriplet[VD, ED] +// currentEdge.src = new Vertex[VD] +// currentEdge.dst = new Vertex[VD] +// currentEdge.src.id = edgePartition.srcIds.getLong(i) +// // assert(vmap.containsKey(e.src.id)) +// currentEdge.src.data = vmap.get(currentEdge.src.id) + +// currentEdge.dst.id = edgePartition.dstIds.getLong(i) +// // assert(vmap.containsKey(e.dst.id)) +// currentEdge.dst.data = vmap.get(currentEdge.dst.id) + +// currentEdge.data = edgePartition.data(i) +// //println("Iter: " + pos + " " + e.src.id + " " + e.dst.id + " " + e.data) +// //println("List: " + i + " " + currentEdge.src.id + " " + currentEdge.dst.id + " " + currentEdge.data) +// lb += currentEdge +// } +// lb.toList +// } +// } +// Iterator((vmap, iter)) +// } +// } diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index e178df3841531..45dc863a6be0b 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -2,12 +2,18 @@ package org.apache.spark.graph.impl import scala.collection.JavaConversions._ +import scala.collection.mutable + import org.apache.spark.SparkContext._ import org.apache.spark.Partitioner import org.apache.spark.HashPartitioner import org.apache.spark.util.ClosureCleaner +import org.apache.spark.rdd import org.apache.spark.rdd.RDD +import org.apache.spark.rdd.IndexedRDD +import org.apache.spark.rdd.RDDIndex + import org.apache.spark.graph._ import org.apache.spark.graph.impl.GraphImpl._ @@ -18,112 +24,224 @@ import org.apache.spark.graph.impl.MessageToPartitionRDDFunctions._ * A Graph RDD that supports computation on graphs. */ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( - val numVertexPartitions: Int, - val numEdgePartitions: Int, - _rawVertices: RDD[Vertex[VD]], - _rawEdges: RDD[Edge[ED]], - _rawVTable: RDD[(Vid, (VD, Array[Pid]))], - _rawETable: RDD[(Pid, EdgePartition[ED])]) + val vTable: IndexedRDD[Vid, VD], + val vid2pid: IndexedRDD[Vid, Pid], + val eTable: IndexedRDD[Pid, EdgePartition[ED]]) extends Graph[VD, ED] { - def this(vertices: RDD[Vertex[VD]], edges: RDD[Edge[ED]]) = { - this(vertices.partitions.size, edges.partitions.size, vertices, edges, null, null) - } - def withPartitioner(numVertexPartitions: Int, numEdgePartitions: Int): Graph[VD, ED] = { - if (_cached) { - new GraphImpl(numVertexPartitions, numEdgePartitions, null, null, _rawVTable, _rawETable) - .cache() - } else { - new GraphImpl(numVertexPartitions, numEdgePartitions, _rawVertices, _rawEdges, null, null) - } + /** + * The vTableReplicated is a version of the vertex data after it is + * replicated. + */ + val vTableReplicated: IndexedRDD[Pid, VertexHashMap[VD]] = { + // Join vid2pid and vTable, generate a shuffle dependency on the joined + // result, and get the shuffle id so we can use it on the slave. + vTable.cogroup(vid2pid) + .flatMap { case (vid, (vdatas, pids)) => + pids.iterator.map { + pid => MessageToPartition(pid, (vid, vdatas.head)) + } + } + .partitionBy(eTable.partitioner.get) //@todo assert edge table has partitioner + .mapPartitionsWithIndex( (pid, iter) => { + // Build the hashmap for each partition + val vmap = new VertexHashMap[VD] + for( msg <- iter ) { vmap.put(msg.data._1, msg.data._2) } + Array((pid, vmap)).iterator + }, preservesPartitioning = true) + .indexed(eTable.index) } - def withVertexPartitioner(numVertexPartitions: Int) = { - withPartitioner(numVertexPartitions, numEdgePartitions) - } - def withEdgePartitioner(numEdgePartitions: Int) = { - withPartitioner(numVertexPartitions, numEdgePartitions) - } - protected var _cached = false + + + + // def this(vertices: RDD[Vertex[VD]], edges: RDD[Edge[ED]]) = { + // this(vertices.partitions.size, edges.partitions.size, vertices, edges, null, null) + // } + + // def withPartitioner(numVertexPartitions: Int, numEdgePartitions: Int): Graph[VD, ED] = { + // if (_cached) { + // new GraphImpl(numVertexPartitions, numEdgePartitions, null, null, _rawVTable, _rawETable) + // .cache() + // } else { + // new GraphImpl(numVertexPartitions, numEdgePartitions, _rawVertices, _rawEdges, null, null) + // } + // } + + // def withVertexPartitioner(numVertexPartitions: Int) = { + // withPartitioner(numVertexPartitions, numEdgePartitions) + // } + + // def withEdgePartitioner(numEdgePartitions: Int) = { + // withPartitioner(numVertexPartitions, numEdgePartitions) + // } + + override def cache(): Graph[VD, ED] = { eTable.cache() + vid2pid.cache() vTable.cache() - _cached = true + // @todo: should we cache the replicated data? + vTableReplicated.cache() this } override def replication(): Double = { - val rep = vTable.map{ case (_, (_, a)) => a.size }.sum + val rep = vid2pid.groupByKey().map(kv => kv._2.size).sum rep / vTable.count } override def balance(): Array[Int] = { - eTable.map{ case (_, epart) => epart.data.size }.collect + eTable.map{ case (pid, epart) => epart.data.size }.collect } override def reverse: Graph[VD, ED] = { - newGraph(vertices, edges.map{ case Edge(s, t, e) => Edge(t, s, e) }) + val etable = eTable.mapValues( _.reverse ).asInstanceOf[IndexedRDD[Pid, EdgePartition[ED]]] + new GraphImpl(vTable, vid2pid, etable) } /** Return a RDD of vertices. */ - override def vertices: RDD[Vertex[VD]] = { - if (!_cached && _rawVertices != null) { - _rawVertices - } else { - vTable.map { case(vid, (data, pids)) => new Vertex(vid, data) } - } - } + override def vertices: RDD[(Vid, VD)] = vTable + /** Return a RDD of edges. */ override def edges: RDD[Edge[ED]] = { - if (!_cached && _rawEdges != null) { - _rawEdges - } else { - eTable.mapPartitions { iter => iter.next()._2.iterator } - } + eTable.mapPartitions { iter => iter.next()._2.iterator } } /** Return a RDD that brings edges with its source and destination vertices together. */ override def triplets: RDD[EdgeTriplet[VD, ED]] = { - new EdgeTripletRDD(vTableReplicated, eTable).mapPartitions { part => part.next()._2 } + vTableReplicated.join(eTable) + .mapPartitions{ iter => + val (pid, (vmap, edgePartition)) = iter.next() + assert(iter.hasNext == false) + // Return an iterator that looks up the hash map to find matching + // vertices for each edge. + new Iterator[EdgeTriplet[VD, ED]] { + private var pos = 0 + private val e = new EdgeTriplet[VD, ED] + e.src = new Vertex[VD] + e.dst = new Vertex[VD] + + override def hasNext: Boolean = pos < edgePartition.size + override def next() = { + e.src.id = edgePartition.srcIds(pos) + // assert(vmap.containsKey(e.src.id)) + e.src.data = vmap.get(e.src.id) + e.dst.id = edgePartition.dstIds(pos) + // assert(vmap.containsKey(e.dst.id)) + e.dst.data = vmap.get(e.dst.id) + //println("Iter called: " + pos) + e.data = edgePartition.data(pos) + pos += 1 + e + } + + override def toList: List[EdgeTriplet[VD, ED]] = { + val lb = new mutable.ListBuffer[EdgeTriplet[VD,ED]] + for (i <- (0 until edgePartition.size)) { + val currentEdge = new EdgeTriplet[VD, ED] + currentEdge.src = new Vertex[VD] + currentEdge.dst = new Vertex[VD] + currentEdge.src.id = edgePartition.srcIds(i) + // assert(vmap.containsKey(e.src.id)) + currentEdge.src.data = vmap.get(currentEdge.src.id) + + currentEdge.dst.id = edgePartition.dstIds(i) + // assert(vmap.containsKey(e.dst.id)) + currentEdge.dst.data = vmap.get(currentEdge.dst.id) + + currentEdge.data = edgePartition.data(i) + lb += currentEdge + } + lb.toList + } + } // end of iterator + } // end of map partition } - override def mapVertices[VD2: ClassManifest](f: Vertex[VD] => VD2): Graph[VD2, ED] = { - newGraph(vertices.map(v => Vertex(v.id, f(v))), edges) + override def mapVertices[VD2: ClassManifest](f: (Vid, VD) => VD2): Graph[VD2, ED] = { + val newVTable = vTable.mapValuesWithKeys((vid, data) => f(vid, data)) + .asInstanceOf[IndexedRDD[Vid, VD2]] + new GraphImpl(newVTable, vid2pid, eTable) } override def mapEdges[ED2: ClassManifest](f: Edge[ED] => ED2): Graph[VD, ED2] = { - newGraph(vertices, edges.map(e => Edge(e.src, e.dst, f(e)))) + val newETable = eTable.mapValues(eBlock => eBlock.map(f)) + .asInstanceOf[IndexedRDD[Pid, EdgePartition[ED2]]] + new GraphImpl(vTable, vid2pid, newETable) } + override def mapTriplets[ED2: ClassManifest](f: EdgeTriplet[VD, ED] => ED2): Graph[VD, ED2] = { - newGraph(vertices, triplets.map(e => Edge(e.src.id, e.dst.id, f(e)))) + val newETable = eTable.join(vTableReplicated).mapValues{ + case (edgePartition, vmap) => + val et = new EdgeTriplet[VD, ED] + et.src = new Vertex[VD] + et.dst = new Vertex[VD] + + edgePartition.map{e => + et.data = e.data + et.src.id = e.src + et.src.data = vmap(e.src) + et.dst.id = e.dst + et.dst.data = vmap(e.dst) + f(et) + } + }.asInstanceOf[IndexedRDD[Pid, EdgePartition[ED2]]] + new GraphImpl(vTable, vid2pid, newETable) } - override def correctEdges(): Graph[VD, ED] = { - val sc = vertices.context - val vset = sc.broadcast(vertices.map(_.id).collect().toSet) - val newEdges = edges.filter(e => vset.value.contains(e.src) && vset.value.contains(e.dst)) - Graph(vertices, newEdges) - } + // override def correctEdges(): Graph[VD, ED] = { + // val sc = vertices.context + // val vset = sc.broadcast(vertices.map(_.id).collect().toSet) + // val newEdges = edges.filter(e => vset.value.contains(e.src) && vset.value.contains(e.dst)) + // Graph(vertices, newEdges) + // } - override def subgraph(epred: EdgeTriplet[VD,ED] => Boolean = (_ => true), - vpred: Vertex[VD] => Boolean = (_ => true) ): Graph[VD, ED] = { + override def subgraph(epred: EdgeTriplet[VD,ED] => Boolean = (x => true), + vpred: (Vid, VD) => Boolean = ((a,b) => true) ): Graph[VD, ED] = { + + /// @todo: The following code behaves deterministically on each + /// vertex predicate but uses additional space. Should we swithc to + /// this version + // val predGraph = mapVertices(v => (v.data, vpred(v))) + // val newETable = predGraph.triplets.filter(t => + // if(v.src.data._2 && v.dst.data._2) { + // val src = Vertex(t.src.id, t.src.data._1) + // val dst = Vertex(t.dst.id, t.dst.data._1) + // epred(new EdgeTriplet[VD, ED](src, dst, t.data)) + // } else { false }) + + // val newVTable = predGraph.vertices.filter(v => v.data._1) + // .map(v => (v.id, v.data._1)).indexed() + + // Reuse the partitioner (but not the index) from this graph + val newVTable = vertices.filter(v => vpred(v._1, v._2)).indexed(vTable.index.partitioner) + - // Restrict the set of vertices to those that satisfy the vertex predicate - val newVertices = vertices.filter(vpred) // Restrict the set of edges to those that satisfy the vertex and the edge predicate. - val newEdges = triplets.filter(t => vpred(t.src) && vpred(t.dst) && epred(t)) - .map( t => Edge(t.src.id, t.dst.id, t.data) ) + val newETable = createETable( + triplets.filter( + t => vpred( t.src.id, t.src.data ) && vpred( t.dst.id, t.dst.data ) && epred(t) + ) + .map( t => Edge(t.src.id, t.dst.id, t.data) ), + eTable.index.partitioner.numPartitions + ) + + // Construct the Vid2Pid map. Here we assume that the filter operation + // behaves deterministically. + // @todo reindex the vertex and edge tables + val newVid2Pid = createVid2Pid(newETable, newVTable.index) - new GraphImpl(newVertices, newEdges) + new GraphImpl(newVTable, newVid2Pid, newETable) } @@ -135,10 +253,10 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( // TODO(crankshaw) is there a better way to do this using RDD.groupBy() // functions? - override def groupEdgeTriplets[ED2: ClassManifest](f: Iterator[EdgeTriplet[VD,ED]] => ED2 ): + override def groupEdgeTriplets[ED2: ClassManifest]( + f: Iterator[EdgeTriplet[VD,ED]] => ED2 ): Graph[VD,ED2] = { //override def groupEdges[ED2: ClassManifest](f: Iterator[Edge[ED]] => ED2 ): - Graph[VD,ED2] = { - + // I think that // myRDD.mapPartitions { part => // val (vmap, edges) = part.next() @@ -169,7 +287,8 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( .mapValues { ts: List[EdgeTriplet[VD, ED]] => f(ts.toIterator) } // convert the resulting map back to a list of tuples .toList - // TODO(crankshaw) needs an iterator over the tuples? Why can't I map over the list? + // TODO(crankshaw) needs an iterator over the tuples? + // Why can't I map over the list? .toIterator // map over those tuples that contain src and dst info plus the // new edge data to make my new edges @@ -185,7 +304,12 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( // and http://stackoverflow.com/questions/6998676/converting-a-scala-map-to-a-list } - newGraph(vertices, newEdges) + + // @todo eliminate the need to call createETable + val newETable = createETable(newEdges, + eTable.index.partitioner.numPartitions) + + new GraphImpl(vTable, vid2pid, newETable) } @@ -202,11 +326,12 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( .toList .toIterator .map { case ((src, dst), data) => Edge(src, dst, data) } - - } - newGraph(vertices, newEdges) + // @todo eliminate the need to call createETable + val newETable = createETable(newEdges, + eTable.index.partitioner.numPartitions) + new GraphImpl(vTable, vid2pid, newETable) } @@ -215,156 +340,90 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( // Lower level transformation methods ////////////////////////////////////////////////////////////////////////////////////////////////// - override def aggregateNeighbors[A: ClassManifest]( - mapFunc: (Vid, EdgeTriplet[VD, ED]) => Option[A], - reduceFunc: (A, A) => A, - default: A, - gatherDirection: EdgeDirection) - : Graph[(VD, Option[A]), ED] = { + override def mapReduceTriplets[A: ClassManifest]( + mapFunc: EdgeTriplet[VD, ED] => Array[(Vid, A)], + reduceFunc: (A, A) => A) + : RDD[(Vid, A)] = { ClosureCleaner.clean(mapFunc) ClosureCleaner.clean(reduceFunc) - val newVTable = vTableReplicated.mapPartitions({ part => - part.map { v => (v._1, MutableTuple2(v._2, Option.empty[A])) } - }, preservesPartitioning = true) - - val newVertices: RDD[(Vid, A)] = - new EdgeTripletRDD[MutableTuple2[VD, Option[A]], ED](newVTable, eTable) - .mapPartitions { part => - val (vmap, edges) = part.next() - val edgeSansAcc = new EdgeTriplet[VD, ED]() - edgeSansAcc.src = new Vertex[VD] - edgeSansAcc.dst = new Vertex[VD] - edges.foreach { e: EdgeTriplet[MutableTuple2[VD, Option[A]], ED] => - edgeSansAcc.data = e.data - edgeSansAcc.src.data = e.src.data._1 - edgeSansAcc.dst.data = e.dst.data._1 - edgeSansAcc.src.id = e.src.id - edgeSansAcc.dst.id = e.dst.id - if (gatherDirection == EdgeDirection.In || gatherDirection == EdgeDirection.Both) { - e.dst.data._2 = - if (e.dst.data._2.isEmpty) { - mapFunc(edgeSansAcc.dst.id, edgeSansAcc) - } else { - val tmp = mapFunc(edgeSansAcc.dst.id, edgeSansAcc) - if (!tmp.isEmpty) Some(reduceFunc(e.dst.data._2.get, tmp.get)) else e.dst.data._2 - } - } - if (gatherDirection == EdgeDirection.Out || gatherDirection == EdgeDirection.Both) { - e.dst.data._2 = - if (e.dst.data._2.isEmpty) { - mapFunc(edgeSansAcc.src.id, edgeSansAcc) - } else { - val tmp = mapFunc(edgeSansAcc.src.id, edgeSansAcc) - if (!tmp.isEmpty) Some(reduceFunc(e.src.data._2.get, tmp.get)) else e.src.data._2 - } - } - } - vmap.long2ObjectEntrySet().fastIterator().filter(!_.getValue()._2.isEmpty).map{ entry => - (entry.getLongKey(), entry.getValue()._2) + val newVTable: RDD[(Vid, A)] = + vTableReplicated.join(eTable).flatMap{ + case (pid, (vmap, edgePartition)) => + val aggMap = new VertexHashMap[A] + val et = new EdgeTriplet[VD, ED] + et.src = new Vertex[VD] + et.dst = new Vertex[VD] + edgePartition.foreach{e => + et.data = e.data + et.src.id = e.src + et.src.data = vmap(e.src) + et.dst.id = e.dst + et.dst.data = vmap(e.dst) + mapFunc(et).foreach{case (vid, a) => + if(aggMap.containsKey(vid)) { + aggMap.put(vid, reduceFunc(aggMap.get(vid), a)) + } else { aggMap.put(vid, a) } } } - .map{ case (vid, aOpt) => (vid, aOpt.get) } - .combineByKey((v: A) => v, reduceFunc, null, vertexPartitioner, false) + // Return the aggregate map + aggMap.long2ObjectEntrySet().fastIterator().map{ + entry => (entry.getLongKey(), entry.getValue()) + } + } + .indexed(vTable.index).reduceByKey(reduceFunc) - this.leftJoinVertices(newVertices, (v: Vertex[VD], a: Option[A]) => (v.data, a)) + newVTable } - /** - * Same as aggregateNeighbors but map function can return none and there is no default value. - * As a consequence, the resulting table may be much smaller than the set of vertices. - */ - override def aggregateNeighbors[A: ClassManifest]( - mapFunc: (Vid, EdgeTriplet[VD, ED]) => Option[A], - reduceFunc: (A, A) => A, - gatherDirection: EdgeDirection): Graph[(VD, Option[A]), ED] = { + def aggregateNeighbors[A: ClassManifest]( + mapFunc: (Vid, EdgeTriplet[VD, ED]) => Option[A], + reduceFunc: (A, A) => A, + dir: EdgeDirection) + : RDD[(Vid, A)] = { ClosureCleaner.clean(mapFunc) ClosureCleaner.clean(reduceFunc) - val newVTable = vTableReplicated.mapPartitions({ part => - part.map { v => (v._1, MutableTuple2(v._2, Option.empty[A])) } - }, preservesPartitioning = true) - - val newVertices: RDD[(Vid, A)] = - new EdgeTripletRDD[MutableTuple2[VD, Option[A]], ED](newVTable, eTable) - .mapPartitions { part => - val (vmap, edges) = part.next() - val edgeSansAcc = new EdgeTriplet[VD, ED]() - edgeSansAcc.src = new Vertex[VD] - edgeSansAcc.dst = new Vertex[VD] - edges.foreach { e: EdgeTriplet[MutableTuple2[VD, Option[A]], ED] => - edgeSansAcc.data = e.data - edgeSansAcc.src.data = e.src.data._1 - edgeSansAcc.dst.data = e.dst.data._1 - edgeSansAcc.src.id = e.src.id - edgeSansAcc.dst.id = e.dst.id - if (gatherDirection == EdgeDirection.In || gatherDirection == EdgeDirection.Both) { - e.dst.data._2 = - if (e.dst.data._2.isEmpty) { - mapFunc(edgeSansAcc.dst.id, edgeSansAcc) - } else { - val tmp = mapFunc(edgeSansAcc.dst.id, edgeSansAcc) - if (!tmp.isEmpty) Some(reduceFunc(e.dst.data._2.get, tmp.get)) else e.dst.data._2 - } - } - if (gatherDirection == EdgeDirection.Out || gatherDirection == EdgeDirection.Both) { - e.src.data._2 = - if (e.src.data._2.isEmpty) { - mapFunc(edgeSansAcc.src.id, edgeSansAcc) - } else { - val tmp = mapFunc(edgeSansAcc.src.id, edgeSansAcc) - if (!tmp.isEmpty) Some(reduceFunc(e.src.data._2.get, tmp.get)) else e.src.data._2 - } - } - } - vmap.long2ObjectEntrySet().fastIterator().filter(!_.getValue()._2.isEmpty).map{ entry => - (entry.getLongKey(), entry.getValue()._2) - } - } - .map{ case (vid, aOpt) => (vid, aOpt.get) } - .combineByKey((v: A) => v, reduceFunc, null, vertexPartitioner, false) + // Define a new map function over edge triplets + def mf(et: EdgeTriplet[VD,ED]): Array[(Vid, A)] = { + // Compute the message to the dst vertex + val dstA = + if (dir == EdgeDirection.In || dir == EdgeDirection.Both) { + mapFunc(et.dst.id, et) + } else { Option.empty[A] } + // Compute the message to the source vertex + val srcA = + if (dir == EdgeDirection.Out || dir == EdgeDirection.Both) { + mapFunc(et.src.id, et) + } else { Option.empty[A] } + // construct the return array + (srcA, dstA) match { + case (None, None) => Array.empty[(Vid, A)] + case (Some(src),None) => Array((et.src.id, src)) + case (None, Some(dst)) => Array((et.dst.id, dst)) + case (Some(src), Some(dst)) => + Array((et.src.id, src), (et.dst.id, dst)) + } + } - this.leftJoinVertices(newVertices, (v: Vertex[VD], a: Option[A]) => (v.data, a)) + mapReduceTriplets(mf, reduceFunc) } - override def leftJoinVertices[U: ClassManifest, VD2: ClassManifest]( - updates: RDD[(Vid, U)], - updateF: (Vertex[VD], Option[U]) => VD2) - : Graph[VD2, ED] = { - - ClosureCleaner.clean(updateF) - val newVTable = vTable.leftOuterJoin(updates).mapPartitions({ iter => - iter.map { case (vid, ((vdata, pids), update)) => - val newVdata = updateF(Vertex(vid, vdata), update) - (vid, (newVdata, pids)) - } - }, preservesPartitioning = true).cache() - new GraphImpl(newVTable.partitions.length, eTable.partitions.length, null, null, newVTable, eTable) - } - override def joinVertices[U: ClassManifest]( - updates: RDD[(Vid, U)], - updateF: (Vertex[VD], U) => VD) - : Graph[VD, ED] = { + override def outerJoinVertices[U: ClassManifest, VD2: ClassManifest] + (updates: RDD[(Vid, U)])(updateF: (Vid, VD, Option[U]) => VD2) + : Graph[VD2, ED] = { ClosureCleaner.clean(updateF) - val newVTable = vTable.leftOuterJoin(updates).mapPartitions({ iter => - iter.map { case (vid, ((vdata, pids), update)) => - if (update.isDefined) { - val newVdata = updateF(Vertex(vid, vdata), update.get) - (vid, (newVdata, pids)) - } else { - (vid, (vdata, pids)) - } - } - }, preservesPartitioning = true).cache() - - new GraphImpl(newVTable.partitions.length, eTable.partitions.length, null, null, newVTable, eTable) + val newVTable = vTable.leftOuterJoin(updates).mapValuesWithKeys{ + case (vid, (data, other)) => updateF(vid, data, other) + }.asInstanceOf[IndexedRDD[Vid,VD2]] + new GraphImpl(newVTable, vid2pid, eTable) } @@ -372,48 +431,129 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( // Internals hidden from callers ////////////////////////////////////////////////////////////////////////////////////////////////// - // TODO: Support non-hash partitioning schemes. - protected val vertexPartitioner = new HashPartitioner(numVertexPartitions) - protected val edgePartitioner = new HashPartitioner(numEdgePartitions) - /** Create a new graph but keep the current partitioning scheme. */ - protected def newGraph[VD2: ClassManifest, ED2: ClassManifest]( - vertices: RDD[Vertex[VD2]], edges: RDD[Edge[ED2]]): Graph[VD2, ED2] = { - (new GraphImpl[VD2, ED2](vertices, edges)).withPartitioner(numVertexPartitions, numEdgePartitions) - } + + + + // /** Create a new graph but keep the current partitioning scheme. */ + // protected def newGraph[VD2: ClassManifest, ED2: ClassManifest]( + // vertices: RDD[Vertex[VD2]], edges: RDD[Edge[ED2]]): Graph[VD2, ED2] = { + // (new GraphImpl[VD2, ED2](vertices, edges)).withPartitioner(numVertexPartitions, numEdgePartitions) + // } + + // protected lazy val eTable: RDD[(Pid, EdgePartition[ED])] = { + // if (_rawETable == null) { + // createETable(_rawEdges, numEdgePartitions) + // } else { + // _rawETable + // } + // } + + // protected lazy val vTable: RDD[(Vid, (VD, Array[Pid]))] = { + // if (_rawVTable == null) { + // createVTable(_rawVertices, eTable, numVertexPartitions) + // } else { + // _rawVTable + // } + // } + + // protected lazy val vTableReplicated: RDD[(Vid, VD)] = { + // // Join vid2pid and vTable, generate a shuffle dependency on the joined result, and get + // // the shuffle id so we can use it on the slave. + // vTable + // .flatMap { case (vid, (vdata, pids)) => + // pids.iterator.map { pid => MessageToPartition(pid, (vid, vdata)) } + // } + // .partitionBy(edgePartitioner) + // .mapPartitions({ part => + // part.map { message => (message.data._1, message.data._2) } + // }, preservesPartitioning = true) + // } +} - protected lazy val eTable: RDD[(Pid, EdgePartition[ED])] = { - if (_rawETable == null) { - createETable(_rawEdges, numEdgePartitions) - } else { - _rawETable - } + + + + + + + + + + + + + + + +object GraphImpl { + +def apply[VD: ClassManifest, ED: ClassManifest]( + vertices: RDD[(Vid, VD)], edges: RDD[Edge[ED]]): + GraphImpl[VD,ED] = { + + apply(vertices, edges, + vertices.context.defaultParallelism, edges.context.defaultParallelism) } - protected lazy val vTable: RDD[(Vid, (VD, Array[Pid]))] = { - if (_rawVTable == null) { - createVTable(_rawVertices, eTable, numVertexPartitions) - } else { - _rawVTable - } + + def apply[VD: ClassManifest, ED: ClassManifest]( + vertices: RDD[(Vid, VD)], edges: RDD[Edge[ED]], + numVPart: Int, numEPart: Int): GraphImpl[VD,ED] = { + + val vtable = vertices.indexed(numVPart) + val etable = createETable(edges, numEPart) + val vid2pid = createVid2Pid(etable, vtable.index) + + new GraphImpl(vtable, vid2pid, etable) } - protected lazy val vTableReplicated: RDD[(Vid, VD)] = { - // Join vid2pid and vTable, generate a shuffle dependency on the joined result, and get - // the shuffle id so we can use it on the slave. - vTable - .flatMap { case (vid, (vdata, pids)) => - pids.iterator.map { pid => MessageToPartition(pid, (vid, vdata)) } + + + /** + * Create the edge table RDD, which is much more efficient for Java heap storage than the + * normal edges data structure (RDD[(Vid, Vid, ED)]). + * + * The edge table contains multiple partitions, and each partition contains only one RDD + * key-value pair: the key is the partition id, and the value is an EdgePartition object + * containing all the edges in a partition. + */ + protected def createETable[ED: ClassManifest]( + edges: RDD[Edge[ED]], numPartitions: Int) + : IndexedRDD[Pid, EdgePartition[ED]] = { + val ceilSqrt: Pid = math.ceil(math.sqrt(numPartitions)).toInt + edges + .map { e => + // Random partitioning based on the source vertex id. + // val part: Pid = edgePartitionFunction1D(e.src, e.dst, numPartitions) + val part: Pid = edgePartitionFunction2D(e.src, e.dst, numPartitions, ceilSqrt) + //val part: Pid = canonicalEdgePartitionFunction2D(e.src, e.dst, numPartitions, ceilSqrt) + + // Should we be using 3-tuple or an optimized class + MessageToPartition(part, (e.src, e.dst, e.data)) } - .partitionBy(edgePartitioner) - .mapPartitions({ part => - part.map { message => (message.data._1, message.data._2) } - }, preservesPartitioning = true) + .partitionBy(new HashPartitioner(numPartitions)) + .mapPartitionsWithIndex({ (pid, iter) => + val builder = new EdgePartitionBuilder[ED] + iter.foreach { message => + val data = message.data + builder.add(data._1, data._2, data._3) + } + Iterator((pid, builder.toEdgePartition)) + }, preservesPartitioning = true).indexed() } -} -object GraphImpl { + protected def createVid2Pid[ED: ClassManifest]( + eTable: IndexedRDD[Pid, EdgePartition[ED]], + vTableIndex: RDDIndex[Vid]): IndexedRDD[Vid, Pid] = { + eTable.mapPartitions { iter => + val (pid, edgePartition) = iter.next() + val vSet = new VertexSet + edgePartition.foreach(e => {vSet.add(e.src); vSet.add(e.dst)}) + vSet.iterator.map { vid => (vid.toLong, pid) } + }.indexed(vTableIndex) + } protected def edgePartitionFunction1D(src: Vid, dst: Vid, numParts: Pid): Pid = { @@ -500,70 +640,44 @@ object GraphImpl { } - /** - * Create the edge table RDD, which is much more efficient for Java heap storage than the - * normal edges data structure (RDD[(Vid, Vid, ED)]). - * - * The edge table contains multiple partitions, and each partition contains only one RDD - * key-value pair: the key is the partition id, and the value is an EdgePartition object - * containing all the edges in a partition. - */ - protected def createETable[ED: ClassManifest](edges: RDD[Edge[ED]], numPartitions: Int) - : RDD[(Pid, EdgePartition[ED])] = { - val ceilSqrt: Pid = math.ceil(math.sqrt(numPartitions)).toInt - edges - .map { e => - // Random partitioning based on the source vertex id. - // val part: Pid = edgePartitionFunction1D(e.src, e.dst, numPartitions) - val part: Pid = edgePartitionFunction2D(e.src, e.dst, numPartitions, ceilSqrt) - //val part: Pid = canonicalEdgePartitionFunction2D(e.src, e.dst, numPartitions, ceilSqrt) - // Should we be using 3-tuple or an optimized class - MessageToPartition(part, (e.src, e.dst, e.data)) - // (math.abs(e.src) % numPartitions, (e.src, e.dst, e.data)) - } - .partitionBy(new HashPartitioner(numPartitions)) - .mapPartitionsWithIndex({ (pid, iter) => - val edgePartition = new EdgePartition[ED] - iter.foreach { message => - val data = message.data - edgePartition.add(data._1, data._2, data._3) - } - edgePartition.trim() - Iterator((pid, edgePartition)) - }, preservesPartitioning = true) - } - protected def createVTable[VD: ClassManifest, ED: ClassManifest]( - vertices: RDD[Vertex[VD]], - eTable: RDD[(Pid, EdgePartition[ED])], - numPartitions: Int) - : RDD[(Vid, (VD, Array[Pid]))] = { - val partitioner = new HashPartitioner(numPartitions) + // protected def createVTable[VD: ClassManifest, ED: ClassManifest]( + // eTable: IndexedRDD[Pid, EdgePartition[ED]], + // vid2pid: Index + // vertices: RDD[Vertex[VD]], - // A key-value RDD. The key is a vertex id, and the value is a list of - // partitions that contains edges referencing the vertex. - val vid2pid : RDD[(Vid, Seq[Pid])] = eTable.mapPartitions { iter => - val (pid, edgePartition) = iter.next() - val vSet = new VertexSet - var i = 0 - while (i < edgePartition.srcIds.size) { - vSet.add(edgePartition.srcIds.getLong(i)) - vSet.add(edgePartition.dstIds.getLong(i)) - i += 1 - } - vSet.iterator.map { vid => (vid.toLong, pid) } - }.groupByKey(partitioner) - - vertices - .map { v => (v.id, v.data) } - .partitionBy(partitioner) - .leftOuterJoin(vid2pid) - .mapValues { - case (vdata, None) => (vdata, Array.empty[Pid]) - case (vdata, Some(pids)) => (vdata, pids.toArray) - } - } + // default: VD) : IndexedRDD[Vid, VD] = { + + // // Compute all the vertices in the edge table. + // val vid2pid = createVid2Pid(eTable) + + // // Compute all the + // vertices.map(v => (v.id, v.data)).cogroup(vids) + + // // A key-value RDD. The key is a vertex id, and the value is a list of + // // partitions that contains edges referencing the vertex. + // val vid2pid : RDD[(Vid, Seq[Pid])] = eTable.mapPartitions { iter => + // val (pid, edgePartition) = iter.next() + // val vSet = new VertexSet + // var i = 0 + // while (i < edgePartition.srcIds.size) { + // vSet.add(edgePartition.srcIds.getLong(i)) + // vSet.add(edgePartition.dstIds.getLong(i)) + // i += 1 + // } + // vSet.iterator.map { vid => (vid.toLong, pid) } + // }.groupByKey(partitioner) + + // vertices + // .map { v => (v.id, v.data) } + // .partitionBy(partitioner) + // .leftOuterJoin(vid2pid) + // .mapValues { + // case (vdata, None) => (vdata, Array.empty[Pid]) + // case (vdata, Some(pids)) => (vdata, pids.toArray) + // } + // } } diff --git a/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala b/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala index d0583c48a8597..01a04e9c39f7e 100644 --- a/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala +++ b/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala @@ -37,7 +37,7 @@ object GraphGenerators { val host = "local[4]" val sc = new SparkContext(host, "Lognormal graph generator") - val lnGraph = lognormalGraph(sc, 10000) + val lnGraph = logNormalGraph(sc, 10000) val rmat = rmatGraph(sc, 1000, 3000) @@ -69,19 +69,21 @@ object GraphGenerators { // Right now it just generates a bunch of edges where // the edge data is the weight (default 1) - def lognormalGraph(sc: SparkContext, numVertices: Int): GraphImpl[Int, Int] = { + def logNormalGraph(sc: SparkContext, numVertices: Int): GraphImpl[Int, Int] = { // based on Pregel settings val mu = 4 val sigma = 1.3 //val vertsAndEdges = (0 until numVertices).flatMap { src => { - val vertices = (0 until numVertices).flatMap { src => - Array(Vertex(src, sampleLogNormal(mu, sigma, numVertices))) } - val edges = vertices.flatMap( { v => - generateRandomEdges(v.id.toInt, v.data, numVertices) }) - + val vertices: RDD[(Vid, Int)] = sc.parallelize(0 until numVertices).map{ + src => (src, sampleLogNormal(mu, sigma, numVertices)) + } - new GraphImpl[Int, Int](sc.parallelize(vertices), sc.parallelize(edges)) + val edges = vertices.flatMap{ + v => generateRandomEdges(v._1.toInt, v._2, numVertices) + } + + GraphImpl(vertices, edges) //println("Vertices:") //for (v <- vertices) { // println(v.id) @@ -161,8 +163,8 @@ object GraphGenerators { val vertices = edges.flatMap { edge => List((edge.src, 1)) } .reduceByKey(_ + _) - .map{ case (vid, degree) => Vertex(vid, degree) } - new GraphImpl[Int, ED](vertices, edges) + .map{ case (vid, degree) => (vid, degree) } + GraphImpl(vertices, edges) } /** From 141c22e28c04aa77377e525dc8dde58b628d4cca Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Sun, 13 Oct 2013 20:43:23 -0700 Subject: [PATCH 102/531] merging in master changes --- conf/slaves | 12 ++---------- conf/spark-env.sh.template | 20 +++++++++++--------- 2 files changed, 13 insertions(+), 19 deletions(-) diff --git a/conf/slaves b/conf/slaves index 728d22ac2ebb6..da0a01343d20a 100644 --- a/conf/slaves +++ b/conf/slaves @@ -1,10 +1,2 @@ -ec2-23-20-12-62.compute-1.amazonaws.com -ec2-54-205-173-19.compute-1.amazonaws.com -ec2-54-225-4-124.compute-1.amazonaws.com -ec2-23-22-209-112.compute-1.amazonaws.com -ec2-50-16-69-88.compute-1.amazonaws.com -ec2-54-205-163-126.compute-1.amazonaws.com -ec2-54-242-235-95.compute-1.amazonaws.com -ec2-54-211-169-232.compute-1.amazonaws.com -ec2-54-237-31-30.compute-1.amazonaws.com -ec2-54-235-15-124.compute-1.amazonaws.com +# A Spark Worker will be started on each of the machines listed below. +localhost \ No newline at end of file diff --git a/conf/spark-env.sh.template b/conf/spark-env.sh.template index b8936314ecce2..0a35ee7c79b12 100755 --- a/conf/spark-env.sh.template +++ b/conf/spark-env.sh.template @@ -1,19 +1,21 @@ #!/usr/bin/env bash # This file contains environment variables required to run Spark. Copy it as -# spark-env.sh and edit that to configure Spark for your site. At a minimum, -# the following two variables should be set: -# - SCALA_HOME, to point to your Scala installation, or SCALA_LIBRARY_PATH to -# point to the directory for Scala library JARs (if you install Scala as a -# Debian or RPM package, these are in a separate path, often /usr/share/java) +# spark-env.sh and edit that to configure Spark for your site. +# +# The following variables can be set in this file: +# - SPARK_LOCAL_IP, to set the IP address Spark binds to on this node # - MESOS_NATIVE_LIBRARY, to point to your libmesos.so if you use Mesos +# - SPARK_JAVA_OPTS, to set node-specific JVM options for Spark. Note that +# we recommend setting app-wide options in the application's driver program. +# Examples of node-specific options : -Dspark.local.dir, GC options +# Examples of app-wide options : -Dspark.serializer # -# If using the standalone deploy mode, you can also set variables for it: -# - SPARK_MASTER_IP, to bind the master to a different IP address +# If using the standalone deploy mode, you can also set variables for it here: +# - SPARK_MASTER_IP, to bind the master to a different IP address or hostname # - SPARK_MASTER_PORT / SPARK_MASTER_WEBUI_PORT, to use non-default ports # - SPARK_WORKER_CORES, to set the number of cores to use on this machine # - SPARK_WORKER_MEMORY, to set how much memory to use (e.g. 1000m, 2g) # - SPARK_WORKER_PORT / SPARK_WORKER_WEBUI_PORT -# - SPARK_WORKER_INSTANCES, to set the number of worker instances/processes -# to be spawned on every slave machine +# - SPARK_WORKER_INSTANCES, to set the number of worker processes per node From f89e6e5cbf648163dc6e2f255eaf9a0cd3c52666 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Sun, 13 Oct 2013 20:45:01 -0700 Subject: [PATCH 103/531] removing benchmark code --- conf/core-site.xml | 43 ------------------------------------------- 1 file changed, 43 deletions(-) delete mode 100644 conf/core-site.xml diff --git a/conf/core-site.xml b/conf/core-site.xml deleted file mode 100644 index eefd875fc819b..0000000000000 --- a/conf/core-site.xml +++ /dev/null @@ -1,43 +0,0 @@ - - - - - - - - - hadoop.tmp.dir - /mnt/ephemeral-hdfs - - - - fs.default.name - hdfs://ec2-50-17-7-68.compute-1.amazonaws.com:9000 - - - - io.file.buffer.size - 65536 - - - - dfs.client.read.shortcircuit - false - - - - dfs.client.read.shortcircuit.skip.checksum - false - - - - dfs.domain.socket.path - /var/run/hadoop-hdfs/dn._PORT - - - - dfs.client.file-block-storage-locations.timeout - 3000 - - - From bff223454aa128a673ae835f6104460a6a70dbc6 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Sun, 13 Oct 2013 22:27:10 -0700 Subject: [PATCH 104/531] trying to address issues with GraphImpl being caught in closures. --- .../apache/spark/graph/impl/GraphImpl.scala | 65 +++++++++++-------- 1 file changed, 38 insertions(+), 27 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index 71f360bf045cc..871694b06d9f4 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -24,9 +24,9 @@ import org.apache.spark.graph.impl.MessageToPartitionRDDFunctions._ * A Graph RDD that supports computation on graphs. */ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( - val vTable: IndexedRDD[Vid, VD], - val vid2pid: IndexedRDD[Vid, Pid], - val eTable: IndexedRDD[Pid, EdgePartition[ED]]) + @transient val vTable: IndexedRDD[Vid, VD], + @transient val vid2pid: IndexedRDD[Vid, Pid], + @transient val eTable: IndexedRDD[Pid, EdgePartition[ED]]) extends Graph[VD, ED] { @@ -34,24 +34,9 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( * The vTableReplicated is a version of the vertex data after it is * replicated. */ - val vTableReplicated: IndexedRDD[Pid, VertexHashMap[VD]] = { - // Join vid2pid and vTable, generate a shuffle dependency on the joined - // result, and get the shuffle id so we can use it on the slave. - vTable.cogroup(vid2pid) - .flatMap { case (vid, (vdatas, pids)) => - pids.iterator.map { - pid => MessageToPartition(pid, (vid, vdatas.head)) - } - } - .partitionBy(eTable.partitioner.get) //@todo assert edge table has partitioner - .mapPartitionsWithIndex( (pid, iter) => { - // Build the hashmap for each partition - val vmap = new VertexHashMap[VD] - for( msg <- iter ) { vmap.put(msg.data._1, msg.data._2) } - Array((pid, vmap)).iterator - }, preservesPartitioning = true) - .indexed(eTable.index) - } + @transient val vTableReplicated: IndexedRDD[Pid, VertexHashMap[VD]] = + createVTableReplicated(vTable, vid2pid, eTable) + @@ -348,9 +333,12 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( ClosureCleaner.clean(mapFunc) ClosureCleaner.clean(reduceFunc) + val mapfun = mapFunc + val redfun = reduceFunc + val newVTable: RDD[(Vid, A)] = vTableReplicated.join(eTable).flatMap{ - case (pid, (vmap, edgePartition)) => + case (pid, (vmap, edgePartition)) => val aggMap = new VertexHashMap[A] val et = new EdgeTriplet[VD, ED] et.src = new Vertex[VD] @@ -361,9 +349,9 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( et.src.data = vmap(e.src) et.dst.id = e.dst et.dst.data = vmap(e.dst) - mapFunc(et).foreach{case (vid, a) => + mapfun(et).foreach{case (vid, a) => if(aggMap.containsKey(vid)) { - aggMap.put(vid, reduceFunc(aggMap.get(vid), a)) + aggMap.put(vid, redfun(aggMap.get(vid), a)) } else { aggMap.put(vid, a) } } } @@ -372,7 +360,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( entry => (entry.getLongKey(), entry.getValue()) } } - .indexed(vTable.index).reduceByKey(reduceFunc) + .indexed(vTable.index).reduceByKey(redfun) newVTable } @@ -525,9 +513,9 @@ def apply[VD: ClassManifest, ED: ClassManifest]( edges .map { e => // Random partitioning based on the source vertex id. - // val part: Pid = edgePartitionFunction1D(e.src, e.dst, numPartitions) + val part: Pid = edgePartitionFunction1D(e.src, e.dst, numPartitions) // val part: Pid = edgePartitionFunction2D(e.src, e.dst, numPartitions, ceilSqrt) - val part: Pid = randomVertexCut(e.src, e.dst, numPartitions) + // val part: Pid = randomVertexCut(e.src, e.dst, numPartitions) //val part: Pid = canonicalEdgePartitionFunction2D(e.src, e.dst, numPartitions, ceilSqrt) // Should we be using 3-tuple or an optimized class @@ -557,6 +545,29 @@ def apply[VD: ClassManifest, ED: ClassManifest]( } + protected def createVTableReplicated[VD: ClassManifest, ED: ClassManifest]( + vTable: IndexedRDD[Vid, VD], vid2pid: IndexedRDD[Vid, Pid], + eTable: IndexedRDD[Pid, EdgePartition[ED]]): + IndexedRDD[Pid, VertexHashMap[VD]] = { + // Join vid2pid and vTable, generate a shuffle dependency on the joined + // result, and get the shuffle id so we can use it on the slave. + vTable.cogroup(vid2pid) + .flatMap { case (vid, (vdatas, pids)) => + pids.iterator.map { + pid => MessageToPartition(pid, (vid, vdatas.head)) + } + } + .partitionBy(eTable.partitioner.get) //@todo assert edge table has partitioner + .mapPartitionsWithIndex( (pid, iter) => { + // Build the hashmap for each partition + val vmap = new VertexHashMap[VD] + for( msg <- iter ) { vmap.put(msg.data._1, msg.data._2) } + Array((pid, vmap)).iterator + }, preservesPartitioning = true) + .indexed(eTable.index) + } + + protected def edgePartitionFunction1D(src: Vid, dst: Vid, numParts: Pid): Pid = { val mixingPrime: Vid = 1125899906842597L (math.abs(src) * mixingPrime).toInt % numParts From 67bb39c54b25e1b13edad01c25f7183e95f8a400 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Mon, 14 Oct 2013 18:49:05 -0700 Subject: [PATCH 105/531] Removing extraneous code --- .../scala/org/apache/spark/graph/Pregel.scala | 7 -- .../spark/graph/impl/EdgeTripletRDD.scala | 112 ------------------ 2 files changed, 119 deletions(-) delete mode 100644 graph/src/main/scala/org/apache/spark/graph/impl/EdgeTripletRDD.scala diff --git a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala index 93c9c09ee3d0d..93545b6ea58ce 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala @@ -19,13 +19,6 @@ object Pregel { def mapF(vid: Vid, edge: EdgeTriplet[VD,ED]) = sendMsg(edge.otherVertex(vid).id, edge) - def runProg(id: Vid, data: (VD, Option[A]) ): VD = { - val (vData, msg) = data - msg match { - case Some(m) => vprog(id, vData, m) - case None => vData - } - } // Receive the first set of messages g.mapVertices( (vid, vdata) => vprog(vid, vdata, initialMsg)) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/EdgeTripletRDD.scala b/graph/src/main/scala/org/apache/spark/graph/impl/EdgeTripletRDD.scala deleted file mode 100644 index 6779f4aa093ff..0000000000000 --- a/graph/src/main/scala/org/apache/spark/graph/impl/EdgeTripletRDD.scala +++ /dev/null @@ -1,112 +0,0 @@ -// package org.apache.spark.graph.impl - -// import scala.collection.mutable - -// import org.apache.spark.Aggregator -// import org.apache.spark.Partition -// import org.apache.spark.SparkEnv -// import org.apache.spark.TaskContext -// import org.apache.spark.rdd.RDD -// import org.apache.spark.Dependency -// import org.apache.spark.OneToOneDependency -// import org.apache.spark.ShuffleDependency -// import org.apache.spark.SparkContext._ -// import org.apache.spark.graph._ - - -// private[graph] -// class EdgeTripletPartition(idx: Int, val vPart: Partition, val ePart: Partition) -// extends Partition { -// override val index: Int = idx -// override def hashCode(): Int = idx -// } - - -// /** -// * A RDD that brings together edge data with its associated vertex data. -// */ -// private[graph] -// class EdgeTripletRDD[VD: ClassManifest, ED: ClassManifest]( -// vTableReplicated: IndexedRDD[Pid, VertexHashMap[VD]], -// eTable: IndexedRDD[Pid, EdgePartition[ED]]) -// extends RDD[(VertexHashMap[VD], Iterator[EdgeTriplet[VD, ED]])](eTable.context, Nil) { - -// //println("ddshfkdfhds" + vTableReplicated.partitioner.get.numPartitions) -// //println("9757984589347598734549" + eTable.partitioner.get.numPartitions) - -// assert(vTableReplicated.partitioner == eTable.partitioner) - -// override def getDependencies: List[Dependency[_]] = { -// List(new OneToOneDependency(eTable), new OneToOneDependency(vTableReplicated)) -// } - -// override def getPartitions = Array.tabulate[Partition](eTable.partitions.size) { -// i => new EdgeTripletPartition(i, eTable.partitions(i), vTableReplicated.partitions(i)) -// } - -// override val partitioner = eTable.partitioner - -// override def getPreferredLocations(s: Partition) = -// eTable.preferredLocations(s.asInstanceOf[EdgeTripletPartition].ePart) - -// override def compute(s: Partition, context: TaskContext) -// : Iterator[(VertexHashMap[VD], Iterator[EdgeTriplet[VD, ED]])] = { - -// val split = s.asInstanceOf[EdgeTripletPartition] - -// // Fetch the vertices and put them in a hashmap. -// // TODO: use primitive hashmaps for primitive VD types. -// val vmap = new VertexHashMap[VD]//(1000000) -// vTableReplicated.iterator(split.vPart, context).foreach { v => vmap.put(v._1, v._2) } - -// val (pid, edgePartition) = eTable.iterator(split.ePart, context).next() -// .asInstanceOf[(Pid, EdgePartition[ED])] - -// // Return an iterator that looks up the hash map to find matching vertices for each edge. -// val iter = new Iterator[EdgeTriplet[VD, ED]] { -// private var pos = 0 -// private val e = new EdgeTriplet[VD, ED] -// e.src = new Vertex[VD] -// e.dst = new Vertex[VD] - -// override def hasNext: Boolean = pos < edgePartition.size -// override def next() = { -// e.src.id = edgePartition.srcIds.getLong(pos) -// // assert(vmap.containsKey(e.src.id)) -// e.src.data = vmap.get(e.src.id) - -// e.dst.id = edgePartition.dstIds.getLong(pos) -// // assert(vmap.containsKey(e.dst.id)) -// e.dst.data = vmap.get(e.dst.id) - -// //println("Iter called: " + pos) -// e.data = edgePartition.data(pos) -// pos += 1 -// e -// } - -// override def toList: List[EdgeTriplet[VD, ED]] = { -// val lb = new mutable.ListBuffer[EdgeTriplet[VD,ED]] -// for (i <- (0 until edgePartition.size)) { -// val currentEdge = new EdgeTriplet[VD, ED] -// currentEdge.src = new Vertex[VD] -// currentEdge.dst = new Vertex[VD] -// currentEdge.src.id = edgePartition.srcIds.getLong(i) -// // assert(vmap.containsKey(e.src.id)) -// currentEdge.src.data = vmap.get(currentEdge.src.id) - -// currentEdge.dst.id = edgePartition.dstIds.getLong(i) -// // assert(vmap.containsKey(e.dst.id)) -// currentEdge.dst.data = vmap.get(currentEdge.dst.id) - -// currentEdge.data = edgePartition.data(i) -// //println("Iter: " + pos + " " + e.src.id + " " + e.dst.id + " " + e.data) -// //println("List: " + i + " " + currentEdge.src.id + " " + currentEdge.dst.id + " " + currentEdge.data) -// lb += currentEdge -// } -// lb.toList -// } -// } -// Iterator((vmap, iter)) -// } -// } From 11a44d0ec98a746d252980cb745962ac2c079336 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Mon, 14 Oct 2013 19:37:52 -0700 Subject: [PATCH 106/531] Introducing indexedrdd The rest of indexed rdd --- .../scala/org/apache/spark/SparkContext.scala | 2 +- .../apache/spark/api/java/JavaPairRDD.scala | 78 ++- .../org/apache/spark/rdd/IndexedRDD.scala | 256 ++++++++++ .../spark/rdd/IndexedRDDFunctions.scala | 295 +++++++++++ .../apache/spark/rdd/PairRDDFunctions.scala | 66 ++- .../main/scala/org/apache/spark/rdd/RDD.scala | 17 + .../apache/spark/rdd/IndexedRDDSuite.scala | 461 ++++++++++++++++++ 7 files changed, 1139 insertions(+), 36 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/rdd/IndexedRDD.scala create mode 100644 core/src/main/scala/org/apache/spark/rdd/IndexedRDDFunctions.scala create mode 100644 core/src/test/scala/org/apache/spark/rdd/IndexedRDDSuite.scala diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index b7a8179786a6c..f3723a4f9d121 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -928,7 +928,7 @@ object SparkContext { // TODO: Add AccumulatorParams for other types, e.g. lists and strings implicit def rddToPairRDDFunctions[K: ClassManifest, V: ClassManifest](rdd: RDD[(K, V)]) = - new PairRDDFunctions(rdd) + rdd.pairRDDFunctions implicit def rddToSequenceFileRDDFunctions[K <% Writable: ClassManifest, V <% Writable: ClassManifest]( rdd: RDD[(K, V)]) = diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index a6518abf456d3..2f94ae5fa8ee6 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala @@ -264,8 +264,11 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif * the merging locally on each mapper before sending results to a reducer, similarly to a * "combiner" in MapReduce. */ - def join[W](other: JavaPairRDD[K, W], partitioner: Partitioner): JavaPairRDD[K, (V, W)] = + def join[W](other: JavaPairRDD[K, W], partitioner: Partitioner): JavaPairRDD[K, (V, W)] = { + implicit val wm: ClassManifest[W] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] fromRDD(rdd.join(other, partitioner)) + } /** * Perform a left outer join of `this` and `other`. For each element (k, v) in `this`, the @@ -275,6 +278,8 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif */ def leftOuterJoin[W](other: JavaPairRDD[K, W], partitioner: Partitioner) : JavaPairRDD[K, (V, Optional[W])] = { + implicit val wm: ClassManifest[W] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] val joinResult = rdd.leftOuterJoin(other, partitioner) fromRDD(joinResult.mapValues{case (v, w) => (v, JavaUtils.optionToOptional(w))}) } @@ -287,6 +292,8 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif */ def rightOuterJoin[W](other: JavaPairRDD[K, W], partitioner: Partitioner) : JavaPairRDD[K, (Optional[V], W)] = { + implicit val wm: ClassManifest[W] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] val joinResult = rdd.rightOuterJoin(other, partitioner) fromRDD(joinResult.mapValues{case (v, w) => (JavaUtils.optionToOptional(v), w)}) } @@ -325,16 +332,22 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and * (k, v2) is in `other`. Performs a hash join across the cluster. */ - def join[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (V, W)] = + def join[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (V, W)] = { + implicit val wm: ClassManifest[W] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] fromRDD(rdd.join(other)) + } /** * Return an RDD containing all pairs of elements with matching keys in `this` and `other`. Each * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and * (k, v2) is in `other`. Performs a hash join across the cluster. */ - def join[W](other: JavaPairRDD[K, W], numPartitions: Int): JavaPairRDD[K, (V, W)] = + def join[W](other: JavaPairRDD[K, W], numPartitions: Int): JavaPairRDD[K, (V, W)] = { + implicit val wm: ClassManifest[W] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] fromRDD(rdd.join(other, numPartitions)) + } /** * Perform a left outer join of `this` and `other`. For each element (k, v) in `this`, the @@ -343,6 +356,8 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif * using the existing partitioner/parallelism level. */ def leftOuterJoin[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (V, Optional[W])] = { + implicit val wm: ClassManifest[W] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] val joinResult = rdd.leftOuterJoin(other) fromRDD(joinResult.mapValues{case (v, w) => (v, JavaUtils.optionToOptional(w))}) } @@ -354,6 +369,8 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif * into `numPartitions` partitions. */ def leftOuterJoin[W](other: JavaPairRDD[K, W], numPartitions: Int): JavaPairRDD[K, (V, Optional[W])] = { + implicit val wm: ClassManifest[W] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] val joinResult = rdd.leftOuterJoin(other, numPartitions) fromRDD(joinResult.mapValues{case (v, w) => (v, JavaUtils.optionToOptional(w))}) } @@ -365,6 +382,8 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif * RDD using the existing partitioner/parallelism level. */ def rightOuterJoin[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (Optional[V], W)] = { + implicit val wm: ClassManifest[W] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] val joinResult = rdd.rightOuterJoin(other) fromRDD(joinResult.mapValues{case (v, w) => (JavaUtils.optionToOptional(v), w)}) } @@ -376,6 +395,8 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif * RDD into the given number of partitions. */ def rightOuterJoin[W](other: JavaPairRDD[K, W], numPartitions: Int): JavaPairRDD[K, (Optional[V], W)] = { + implicit val wm: ClassManifest[W] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] val joinResult = rdd.rightOuterJoin(other, numPartitions) fromRDD(joinResult.mapValues{case (v, w) => (JavaUtils.optionToOptional(v), w)}) } @@ -412,55 +433,86 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif * list of values for that key in `this` as well as `other`. */ def cogroup[W](other: JavaPairRDD[K, W], partitioner: Partitioner) - : JavaPairRDD[K, (JList[V], JList[W])] = + : JavaPairRDD[K, (JList[V], JList[W])] = { + implicit val wm: ClassManifest[W] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] fromRDD(cogroupResultToJava(rdd.cogroup(other, partitioner))) + } /** * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a * tuple with the list of values for that key in `this`, `other1` and `other2`. */ def cogroup[W1, W2](other1: JavaPairRDD[K, W1], other2: JavaPairRDD[K, W2], partitioner: Partitioner) - : JavaPairRDD[K, (JList[V], JList[W1], JList[W2])] = + : JavaPairRDD[K, (JList[V], JList[W1], JList[W2])] = { + implicit val w1m: ClassManifest[W1] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W1]] + implicit val w2m: ClassManifest[W2] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W2]] fromRDD(cogroupResult2ToJava(rdd.cogroup(other1, other2, partitioner))) + } /** * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the * list of values for that key in `this` as well as `other`. */ - def cogroup[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (JList[V], JList[W])] = + def cogroup[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (JList[V], JList[W])] = { + implicit val wm: ClassManifest[W] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] fromRDD(cogroupResultToJava(rdd.cogroup(other))) + } /** * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a * tuple with the list of values for that key in `this`, `other1` and `other2`. */ def cogroup[W1, W2](other1: JavaPairRDD[K, W1], other2: JavaPairRDD[K, W2]) - : JavaPairRDD[K, (JList[V], JList[W1], JList[W2])] = + : JavaPairRDD[K, (JList[V], JList[W1], JList[W2])] = { + implicit val w1m: ClassManifest[W1] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W1]] + implicit val w2m: ClassManifest[W2] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W2]] fromRDD(cogroupResult2ToJava(rdd.cogroup(other1, other2))) + } /** * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the * list of values for that key in `this` as well as `other`. */ - def cogroup[W](other: JavaPairRDD[K, W], numPartitions: Int): JavaPairRDD[K, (JList[V], JList[W])] - = fromRDD(cogroupResultToJava(rdd.cogroup(other, numPartitions))) - + def cogroup[W](other: JavaPairRDD[K, W], numPartitions: Int): JavaPairRDD[K, (JList[V], JList[W])] = { + implicit val wm: ClassManifest[W] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] + fromRDD(cogroupResultToJava(rdd.cogroup(other, numPartitions))) + } /** * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a * tuple with the list of values for that key in `this`, `other1` and `other2`. */ def cogroup[W1, W2](other1: JavaPairRDD[K, W1], other2: JavaPairRDD[K, W2], numPartitions: Int) - : JavaPairRDD[K, (JList[V], JList[W1], JList[W2])] = + : JavaPairRDD[K, (JList[V], JList[W1], JList[W2])] = { + implicit val w1m: ClassManifest[W1] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W1]] + implicit val w2m: ClassManifest[W2] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W2]] fromRDD(cogroupResult2ToJava(rdd.cogroup(other1, other2, numPartitions))) + } /** Alias for cogroup. */ - def groupWith[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (JList[V], JList[W])] = + def groupWith[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (JList[V], JList[W])] = { + implicit val wm: ClassManifest[W] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] fromRDD(cogroupResultToJava(rdd.groupWith(other))) + } /** Alias for cogroup. */ def groupWith[W1, W2](other1: JavaPairRDD[K, W1], other2: JavaPairRDD[K, W2]) - : JavaPairRDD[K, (JList[V], JList[W1], JList[W2])] = + : JavaPairRDD[K, (JList[V], JList[W1], JList[W2])] = { + implicit val w1m: ClassManifest[W1] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W1]] + implicit val w2m: ClassManifest[W2] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W2]] fromRDD(cogroupResult2ToJava(rdd.groupWith(other1, other2))) + } /** * Return the list of values in the RDD for key `key`. This operation is done efficiently if the diff --git a/core/src/main/scala/org/apache/spark/rdd/IndexedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/IndexedRDD.scala new file mode 100644 index 0000000000000..8d2e9782c2936 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/rdd/IndexedRDD.scala @@ -0,0 +1,256 @@ +/* + * 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.rdd + +import java.nio.ByteBuffer + + +import java.util.{HashMap => JHashMap, BitSet => JBitSet, HashSet => JHashSet} + +import scala.collection.JavaConversions._ +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark._ +import org.apache.spark.rdd._ +import org.apache.spark.SparkContext._ +import org.apache.spark.Partitioner._ + +import org.apache.spark.storage.StorageLevel + + + + + + +/** + * The BlockIndex is the internal map structure used inside the index + * of the IndexedRDD. + */ +class BlockIndex[@specialized K: ClassManifest] extends JHashMap[K,Int] + + +/** + * The RDDIndex is an opaque type used to represent the organization + * of values in an RDD + */ +class RDDIndex[@specialized K: ClassManifest](private[spark] val rdd: RDD[BlockIndex[K]]) { + def persist(newLevel: StorageLevel): RDDIndex[K] = { + rdd.persist(newLevel) + return this + } + + def partitioner: Partitioner = rdd.partitioner.get +} + + + + + +/** + * An IndexedRDD[K,V] extends the RDD[(K,V)] by pre-indexing the keys and + * organizing the values to enable faster join operations. + * + * In addition to providing the basic RDD[(K,V)] functionality the IndexedRDD + * exposes an index member which can be used to "key" other IndexedRDDs + * + */ +class IndexedRDD[K: ClassManifest, V: ClassManifest]( + @transient val index: RDDIndex[K], + @transient val valuesRDD: RDD[ Seq[Seq[V]] ]) + extends RDD[(K, V)](index.rdd.context, + List(new OneToOneDependency(index.rdd), new OneToOneDependency(valuesRDD)) ) { + + /** + * An internal representation which joins the block indices with the values + */ + protected[spark] val tuples = new ZippedRDD(index.rdd.context, index.rdd, valuesRDD) + + + /** + * The partitioner is defined by the index + */ + override val partitioner = index.rdd.partitioner + + + + + + /** + * The actual partitions are defined by the tuples. + */ + override def getPartitions: Array[Partition] = tuples.getPartitions + + /** + * The preferred locations are computed based on the preferred locations of the tuples. + */ + override def getPreferredLocations(s: Partition): Seq[String] = + tuples.getPreferredLocations(s) + + + /** + * Caching an IndexedRDD causes the index and values to be cached separately. + */ + override def persist(newLevel: StorageLevel): RDD[(K,V)] = { + index.persist(newLevel) + valuesRDD.persist(newLevel) + return this + } + + + override def pairRDDFunctions[K1, V1]( + implicit t: (K, V) <:< (K1,V1), k: ClassManifest[K1], v: ClassManifest[V1]): + PairRDDFunctions[K1, V1] = { + new IndexedRDDFunctions[K1,V1](this.asInstanceOf[IndexedRDD[K1,V1]]) + } + + + + + + /** + * Provide the RDD[(K,V)] equivalent output. + */ + override def compute(part: Partition, context: TaskContext): Iterator[(K, V)] = { + tuples.compute(part, context).flatMap { case (indexMap, values) => + // Walk the index to construct the key, value pairs + indexMap.iterator + // Extract rows with key value pairs and indicators + .map{ case (k, ind) => (k, values(ind)) } + // Remove tuples that aren't actually present in the array + .filter{ case (_, valar) => valar != null && !valar.isEmpty()} + // Extract the pair (removing the indicator from the tuple) + .flatMap{ case (k, valar) => valar.map(v => (k,v))} + } + } + +} // End of IndexedRDD + + + + +object IndexedRDD { + def apply[K: ClassManifest, V: ClassManifest]( + tbl: RDD[(K,V)], + existingIndex: RDDIndex[K] = null ): IndexedRDD[K, V] = { + + if (existingIndex == null) { + // Shuffle the table (if necessary) + val shuffledTbl = + if (tbl.partitioner.isEmpty) { + new ShuffledRDD[K, V, (K,V)](tbl, Partitioner.defaultPartitioner(tbl)) + } else { tbl } + + val groups = shuffledTbl.mapPartitions( iter => { + val indexMap = new BlockIndex[K]() + val values = new ArrayBuffer[Seq[V]]() + for ((k,v) <- iter){ + if(!indexMap.contains(k)) { + val ind = indexMap.size + indexMap.put(k, ind) + values.append(ArrayBuffer.empty[V]) + } + val ind = indexMap.get(k) + values(ind).asInstanceOf[ArrayBuffer[V]].append(v) + } + List((indexMap, values.toSeq)).iterator + }, true).cache + // extract the index and the values + val index = groups.mapPartitions(_.map{ case (kMap,vAr) => kMap }, true) + val values = groups.mapPartitions(_.map{ case (kMap,vAr) => vAr }, true) + new IndexedRDD[K,V](new RDDIndex(index), values) + } else { + val index = existingIndex + val partitioner = index.rdd.partitioner match { + case Some(p) => p + case None => throw new SparkException("An index must have a partitioner.") + } + + // Shuffle the table according to the index (if necessary) + val shuffledTbl = + if (tbl.partitioner == Some(partitioner)) { + tbl + } else { + new ShuffledRDD[K, V, (K,V)](tbl, partitioner) + } + + // Use the index to build the new values table + val values = index.rdd.zipPartitions(shuffledTbl)( + (indexIter, tblIter) => { + // There is only one map + val index = indexIter.next() + assert(!indexIter.hasNext()) + val values = new Array[Seq[V]](index.size) + for ((k,v) <- tblIter) { + if (!index.contains(k)) { + throw new SparkException("Error: Trying to bind an external index " + + "to an RDD which contains keys that are not in the index.") + } + val ind = index(k) + if (values(ind) == null) { + values(ind) = ArrayBuffer.empty[V] + } + values(ind).asInstanceOf[ArrayBuffer[V]].append(v) + } + List(values.toSeq).iterator + }) + + new IndexedRDD[K,V](index, values) + } + } + + /** + * Construct and index of the unique values in a given RDD. + */ + def makeIndex[K: ClassManifest](keys: RDD[K], + partitioner: Option[Partitioner] = None): RDDIndex[K] = { + // @todo: I don't need the boolean its only there to be the second type since I want to shuffle a single RDD + // Ugly hack :-(. In order to partition the keys they must have values. + val tbl = keys.mapPartitions(_.map(k => (k, false)), true) + // Shuffle the table (if necessary) + val shuffledTbl = partitioner match { + case None => { + if (tbl.partitioner.isEmpty) { + // @todo: I don't need the boolean its only there to be the second type of the shuffle. + new ShuffledRDD[K, Boolean, (K, Boolean)](tbl, Partitioner.defaultPartitioner(tbl)) + } else { tbl } + } + case Some(partitioner) => + tbl.partitionBy(partitioner) +// new ShuffledRDD[K, Boolean](tbl, partitioner) + } + + + val index = shuffledTbl.mapPartitions( iter => { + val indexMap = new BlockIndex[K]() + for ( (k,_) <- iter ){ + if(!indexMap.contains(k)){ + val ind = indexMap.size + indexMap.put(k, ind) + } + } + List(indexMap).iterator + }, true).cache + new RDDIndex(index) + } + +} + + + + + diff --git a/core/src/main/scala/org/apache/spark/rdd/IndexedRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/IndexedRDDFunctions.scala new file mode 100644 index 0000000000000..358ab57b0c9e5 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/rdd/IndexedRDDFunctions.scala @@ -0,0 +1,295 @@ +/* + * 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.rdd + +import java.util.{HashMap => JHashMap, BitSet => JBitSet, HashSet => JHashSet} + +import scala.collection.JavaConversions._ +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark._ + + + +class IndexedRDDFunctions[K: ClassManifest, V: ClassManifest](self: IndexedRDD[K,V]) + extends PairRDDFunctions[K,V](self) { + + /** + * Construct a new IndexedRDD that is indexed by only the keys in the RDD + */ + def reindex(): IndexedRDD[K,V] = IndexedRDD(self) + + + /** + * Pass each value in the key-value pair RDD through a map function without changing the keys; + * this also retains the original RDD's partitioning. + */ + override def mapValues[U: ClassManifest](f: V => U): RDD[(K, U)] = { + val cleanF = self.index.rdd.context.clean(f) + val newValues = self.valuesRDD.mapPartitions(_.map(values => values.map{ + case null => null + case row => row.map(x => f(x)) + }), true) + new IndexedRDD[K,U](self.index, newValues) + } + + + /** + * Pass each value in the key-value pair RDD through a map function without changing the keys; + * this also retains the original RDD's partitioning. + */ + override def mapValuesWithKeys[U: ClassManifest](f: (K, V) => U): RDD[(K, U)] = { + val cleanF = self.index.rdd.context.clean(f) + val newValues = self.index.rdd.zipPartitions(self.valuesRDD){ (keysIter, valuesIter) => + val index = keysIter.next() + assert(keysIter.hasNext() == false) + val oldValues = valuesIter.next() + assert(valuesIter.hasNext() == false) + // Allocate the array to store the results into + val newValues: Array[Seq[U]] = new Array[Seq[U]](oldValues.size) + // Populate the new Values + for( (k,i) <- index ) { + if(oldValues(i) != null) { + newValues(i) = oldValues(i).map( v => f(k,v) ) + } + } + Array(newValues.toSeq).iterator + } + new IndexedRDD[K,U](self.index, newValues) + } + + + + /** + * Pass each value in the key-value pair RDD through a flatMap function without changing the + * keys; this also retains the original RDD's partitioning. + */ + override def flatMapValues[U: ClassManifest](f: V => TraversableOnce[U]): RDD[(K,U)] = { + val cleanF = self.index.rdd.context.clean(f) + val newValues = self.valuesRDD.mapPartitions(_.map(values => values.map{ + case null => null + case row => row.flatMap(x => f(x)) + }), true) + new IndexedRDD[K,U](self.index, newValues) + } + + + /** + * Generic function to combine the elements for each key using a custom set of aggregation + * functions. Turns an RDD[(K, V)] into a result of type RDD[(K, C)], for a "combined type" C + * Note that V and C can be different -- for example, one might group an RDD of type + * (Int, Int) into an RDD of type (Int, Seq[Int]). Users provide three functions: + * + * - `createCombiner`, which turns a V into a C (e.g., creates a one-element list) + * - `mergeValue`, to merge a V into a C (e.g., adds it to the end of a list) + * - `mergeCombiners`, to combine two C's into a single one. + */ + override def combineByKey[C: ClassManifest](createCombiner: V => C, + mergeValue: (C, V) => C, + mergeCombiners: (C, C) => C, + partitioner: Partitioner, + mapSideCombine: Boolean = true, + serializerClass: String = null): RDD[(K, C)] = { + val newValues = self.valuesRDD.mapPartitions( + _.map{ groups: Seq[Seq[V]] => + groups.map{ group: Seq[V] => + if (group != null && !group.isEmpty) { + val c: C = createCombiner(group.head) + val sum: C = group.tail.foldLeft(c)(mergeValue) + Seq(sum) + } else { + null + } + } + }, true) + new IndexedRDD[K,C](self.index, newValues) + } + + + + /** + * Group the values for each key in the RDD into a single sequence. Hash-partitions the + * resulting RDD with the existing partitioner/parallelism level. + */ + override def groupByKey(partitioner: Partitioner): RDD[(K, Seq[V])] = { + val newValues = self.valuesRDD.mapPartitions(_.map{ar => ar.map{s => Seq(s)} }, true) + new IndexedRDD[K, Seq[V]](self.index, newValues) + } + + + /** + * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the + * list of values for that key in `this` as well as `other`. + */ + override def cogroup[W: ClassManifest](other: RDD[(K, W)], partitioner: Partitioner): + IndexedRDD[K, (Seq[V], Seq[W])] = { + //RDD[(K, (Seq[V], Seq[W]))] = { + other match { + case other: IndexedRDD[_, _] if self.index == other.index => { + // if both RDDs share exactly the same index and therefore the same super set of keys + // then we simply merge the value RDDs. + // However it is possible that both RDDs are missing a value for a given key in + // which case the returned RDD should have a null value + val newValues = + self.valuesRDD.zipPartitions(other.valuesRDD)( + (thisIter, otherIter) => { + val thisValues: Seq[Seq[V]] = thisIter.next() + assert(!thisIter.hasNext) + val otherValues: Seq[Seq[W]] = otherIter.next() + assert(!otherIter.hasNext) + // Zip the values and if both arrays are null then the key is not present and + // so the resulting value must be null (not a tuple of empty sequences) + val tmp: Seq[Seq[(Seq[V], Seq[W])]] = thisValues.view.zip(otherValues).map{ + case (null, null) => null // The key is not present in either RDD + case (a, null) => Seq((a, Seq.empty[W])) + case (null, b) => Seq((Seq.empty[V], b)) + case (a, b) => Seq((a,b)) + }.toSeq + List(tmp).iterator + }) + new IndexedRDD[K, (Seq[V], Seq[W])](self.index, newValues) + } + case other: IndexedRDD[_, _] + if self.index.rdd.partitioner == other.index.rdd.partitioner => { + // If both RDDs are indexed using different indices but with the same partitioners + // then we we need to first merge the indicies and then use the merged index to + // merge the values. + val newIndex = + self.index.rdd.zipPartitions(other.index.rdd)( + (thisIter, otherIter) => { + val thisIndex = thisIter.next() + assert(!thisIter.hasNext) + val otherIndex = otherIter.next() + assert(!otherIter.hasNext) + val newIndex = new BlockIndex[K]() + // @todo Merge only the keys that correspond to non-null values + // Merge the keys + newIndex.putAll(thisIndex) + newIndex.putAll(otherIndex) + // We need to rekey the index + var ctr = 0 + for (e <- newIndex.entrySet) { + e.setValue(ctr) + ctr += 1 + } + List(newIndex).iterator + }).cache() + // Use the new index along with the this and the other indices to merge the values + val newValues = + newIndex.zipPartitions(self.tuples, other.tuples)( + (newIndexIter, thisTuplesIter, otherTuplesIter) => { + // Get the new index for this partition + val newIndex = newIndexIter.next() + assert(!newIndexIter.hasNext) + // Get the corresponding indicies and values for this and the other IndexedRDD + val (thisIndex, thisValues) = thisTuplesIter.next() + assert(!thisTuplesIter.hasNext) + val (otherIndex, otherValues) = otherTuplesIter.next() + assert(!otherTuplesIter.hasNext) + // Preallocate the new Values array + val newValues = new Array[Seq[(Seq[V],Seq[W])]](newIndex.size) + // Lookup the sequences in both submaps + for ((k,ind) <- newIndex) { + val thisSeq = if (thisIndex.contains(k)) thisValues(thisIndex.get(k)) else null + val otherSeq = if (otherIndex.contains(k)) otherValues(otherIndex.get(k)) else null + // if either of the sequences is not null then the key was in one of the two tables + // and so the value should appear in the returned table + newValues(ind) = (thisSeq, otherSeq) match { + case (null, null) => null + case (a, null) => Seq( (a, Seq.empty[W]) ) + case (null, b) => Seq( (Seq.empty[V], b) ) + case (a, b) => Seq( (a,b) ) + } + } + List(newValues.toSeq).iterator + }) + new IndexedRDD(new RDDIndex(newIndex), newValues) + } + case _ => { + // Get the partitioner from the index + val partitioner = self.index.rdd.partitioner match { + case Some(p) => p + case None => throw new SparkException("An index must have a partitioner.") + } + // Shuffle the other RDD using the partitioner for this index + val otherShuffled = + if (other.partitioner == Some(partitioner)) { + other + } else { + new ShuffledRDD[K, W, (K,W)](other, partitioner) + } + // Join the other RDD with this RDD building a new valueset and new index on the fly + val groups = + self.tuples.zipPartitions(otherShuffled)( + (thisTuplesIter, otherTuplesIter) => { + // Get the corresponding indicies and values for this IndexedRDD + val (thisIndex, thisValues) = thisTuplesIter.next() + assert(!thisTuplesIter.hasNext()) + // Construct a new index + val newIndex = thisIndex.clone().asInstanceOf[BlockIndex[K]] + // Construct a new array Buffer to store the values + val newValues = ArrayBuffer.fill[(Seq[V], Seq[W])](thisValues.size)(null) + // populate the newValues with the values in this IndexedRDD + for ((k,i) <- thisIndex) { + if (thisValues(i) != null) { + newValues(i) = (thisValues(i), ArrayBuffer.empty[W]) + } + } + // Now iterate through the other tuples updating the map + for ((k,w) <- otherTuplesIter){ + if (!newIndex.contains(k)) { + // update the index + val ind = newIndex.size + newIndex.put(k, ind) + // Update the values + newValues.append( (Seq.empty[V], ArrayBuffer(w) ) ) + } else { + val ind = newIndex.get(k) + if(newValues(ind) == null) { + // If the other key was in the index but not in the values + // of this indexed RDD then create a new values entry for it + newValues(ind) = (Seq.empty[V], ArrayBuffer(w)) + } else { + newValues(ind)._2.asInstanceOf[ArrayBuffer[W]].append(w) + } + } + } + // Finalize the new values array + val newValuesArray: Seq[Seq[(Seq[V],Seq[W])]] = + newValues.view.map{ + case null => null + case (s, ab) => Seq((s, ab.toSeq)) + }.toSeq + List( (newIndex, newValuesArray) ).iterator + }).cache() + + // Extract the index and values from the above RDD + val newIndex = groups.mapPartitions(_.map{ case (kMap,vAr) => kMap }, true) + val newValues = groups.mapPartitions(_.map{ case (kMap,vAr) => vAr }, true) + + new IndexedRDD[K, (Seq[V], Seq[W])](new RDDIndex(newIndex), newValues) + } + } + } + + +} + +//(self: IndexedRDD[K, V]) extends PairRDDFunctions(self) { } + + diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index a47c5122759db..569d74ae7a6a1 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -68,7 +68,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) * In addition, users can control the partitioning of the output RDD, and whether to perform * map-side aggregation (if a mapper can produce multiple items with the same key). */ - def combineByKey[C](createCombiner: V => C, + def combineByKey[C: ClassManifest](createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => C, partitioner: Partitioner, @@ -102,7 +102,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) /** * Simplified version of combineByKey that hash-partitions the output RDD. */ - def combineByKey[C](createCombiner: V => C, + def combineByKey[C: ClassManifest](createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => C, numPartitions: Int): RDD[(K, C)] = { @@ -247,7 +247,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and * (k, v2) is in `other`. Uses the given Partitioner to partition the output RDD. */ - def join[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, W))] = { + def join[W: ClassManifest](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, W))] = { this.cogroup(other, partitioner).flatMapValues { case (vs, ws) => for (v <- vs.iterator; w <- ws.iterator) yield (v, w) } @@ -259,7 +259,9 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) * pair (k, (v, None)) if no elements in `other` have key k. Uses the given Partitioner to * partition the output RDD. */ - def leftOuterJoin[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, Option[W]))] = { + + def leftOuterJoin[W: ClassManifest](other: RDD[(K, W)], partitioner: Partitioner): + RDD[(K, (V, Option[W]))] = { this.cogroup(other, partitioner).flatMapValues { case (vs, ws) => if (ws.isEmpty) { vs.iterator.map(v => (v, None)) @@ -275,7 +277,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) * pair (k, (None, w)) if no elements in `this` have key k. Uses the given Partitioner to * partition the output RDD. */ - def rightOuterJoin[W](other: RDD[(K, W)], partitioner: Partitioner) + def rightOuterJoin[W: ClassManifest](other: RDD[(K, W)], partitioner: Partitioner) : RDD[(K, (Option[V], W))] = { this.cogroup(other, partitioner).flatMapValues { case (vs, ws) => if (vs.isEmpty) { @@ -290,7 +292,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) * Simplified version of combineByKey that hash-partitions the resulting RDD using the * existing partitioner/parallelism level. */ - def combineByKey[C](createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => C) + def combineByKey[C: ClassManifest](createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => C) : RDD[(K, C)] = { combineByKey(createCombiner, mergeValue, mergeCombiners, defaultPartitioner(self)) } @@ -318,7 +320,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and * (k, v2) is in `other`. Performs a hash join across the cluster. */ - def join[W](other: RDD[(K, W)]): RDD[(K, (V, W))] = { + def join[W: ClassManifest](other: RDD[(K, W)]): RDD[(K, (V, W))] = { join(other, defaultPartitioner(self, other)) } @@ -327,7 +329,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and * (k, v2) is in `other`. Performs a hash join across the cluster. */ - def join[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (V, W))] = { + def join[W: ClassManifest](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (V, W))] = { join(other, new HashPartitioner(numPartitions)) } @@ -337,7 +339,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) * pair (k, (v, None)) if no elements in `other` have key k. Hash-partitions the output * using the existing partitioner/parallelism level. */ - def leftOuterJoin[W](other: RDD[(K, W)]): RDD[(K, (V, Option[W]))] = { + def leftOuterJoin[W: ClassManifest](other: RDD[(K, W)]): RDD[(K, (V, Option[W]))] = { leftOuterJoin(other, defaultPartitioner(self, other)) } @@ -347,7 +349,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) * pair (k, (v, None)) if no elements in `other` have key k. Hash-partitions the output * into `numPartitions` partitions. */ - def leftOuterJoin[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (V, Option[W]))] = { + def leftOuterJoin[W: ClassManifest](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (V, Option[W]))] = { leftOuterJoin(other, new HashPartitioner(numPartitions)) } @@ -357,7 +359,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) * pair (k, (None, w)) if no elements in `this` have key k. Hash-partitions the resulting * RDD using the existing partitioner/parallelism level. */ - def rightOuterJoin[W](other: RDD[(K, W)]): RDD[(K, (Option[V], W))] = { + def rightOuterJoin[W: ClassManifest](other: RDD[(K, W)]): RDD[(K, (Option[V], W))] = { rightOuterJoin(other, defaultPartitioner(self, other)) } @@ -367,7 +369,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) * pair (k, (None, w)) if no elements in `this` have key k. Hash-partitions the resulting * RDD into the given number of partitions. */ - def rightOuterJoin[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (Option[V], W))] = { + def rightOuterJoin[W: ClassManifest](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (Option[V], W))] = { rightOuterJoin(other, new HashPartitioner(numPartitions)) } @@ -386,16 +388,25 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) * Pass each value in the key-value pair RDD through a map function without changing the keys; * this also retains the original RDD's partitioning. */ - def mapValues[U](f: V => U): RDD[(K, U)] = { + def mapValues[U: ClassManifest](f: V => U): RDD[(K, U)] = { val cleanF = self.context.clean(f) new MappedValuesRDD(self, cleanF) } + + /** + * Pass each value in the key-value pair RDD through a map function without changing the keys; + * this also retains the original RDD's partitioning. + */ + def mapValuesWithKeys[U: ClassManifest](f: (K, V) => U): RDD[(K, U)] = { + self.map{ case (k,v) => (k, f(k,v)) } + } + /** * Pass each value in the key-value pair RDD through a flatMap function without changing the * keys; this also retains the original RDD's partitioning. */ - def flatMapValues[U](f: V => TraversableOnce[U]): RDD[(K, U)] = { + def flatMapValues[U: ClassManifest](f: V => TraversableOnce[U]): RDD[(K, U)] = { val cleanF = self.context.clean(f) new FlatMappedValuesRDD(self, cleanF) } @@ -404,7 +415,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the * list of values for that key in `this` as well as `other`. */ - def cogroup[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (Seq[V], Seq[W]))] = { + def cogroup[W: ClassManifest](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (Seq[V], Seq[W]))] = { if (partitioner.isInstanceOf[HashPartitioner] && getKeyClass().isArray) { throw new SparkException("Default partitioner cannot partition array keys.") } @@ -419,7 +430,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a * tuple with the list of values for that key in `this`, `other1` and `other2`. */ - def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)], partitioner: Partitioner) + def cogroup[W1: ClassManifest, W2: ClassManifest](other1: RDD[(K, W1)], other2: RDD[(K, W2)], partitioner: Partitioner) : RDD[(K, (Seq[V], Seq[W1], Seq[W2]))] = { if (partitioner.isInstanceOf[HashPartitioner] && getKeyClass().isArray) { throw new SparkException("Default partitioner cannot partition array keys.") @@ -435,7 +446,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the * list of values for that key in `this` as well as `other`. */ - def cogroup[W](other: RDD[(K, W)]): RDD[(K, (Seq[V], Seq[W]))] = { + def cogroup[W: ClassManifest](other: RDD[(K, W)]): RDD[(K, (Seq[V], Seq[W]))] = { cogroup(other, defaultPartitioner(self, other)) } @@ -443,7 +454,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a * tuple with the list of values for that key in `this`, `other1` and `other2`. */ - def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)]) + def cogroup[W1: ClassManifest, W2: ClassManifest](other1: RDD[(K, W1)], other2: RDD[(K, W2)]) : RDD[(K, (Seq[V], Seq[W1], Seq[W2]))] = { cogroup(other1, other2, defaultPartitioner(self, other1, other2)) } @@ -452,7 +463,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the * list of values for that key in `this` as well as `other`. */ - def cogroup[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (Seq[V], Seq[W]))] = { + def cogroup[W: ClassManifest](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (Seq[V], Seq[W]))] = { cogroup(other, new HashPartitioner(numPartitions)) } @@ -460,18 +471,18 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a * tuple with the list of values for that key in `this`, `other1` and `other2`. */ - def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)], numPartitions: Int) + def cogroup[W1: ClassManifest, W2: ClassManifest](other1: RDD[(K, W1)], other2: RDD[(K, W2)], numPartitions: Int) : RDD[(K, (Seq[V], Seq[W1], Seq[W2]))] = { cogroup(other1, other2, new HashPartitioner(numPartitions)) } /** Alias for cogroup. */ - def groupWith[W](other: RDD[(K, W)]): RDD[(K, (Seq[V], Seq[W]))] = { + def groupWith[W: ClassManifest](other: RDD[(K, W)]): RDD[(K, (Seq[V], Seq[W]))] = { cogroup(other, defaultPartitioner(self, other)) } /** Alias for cogroup. */ - def groupWith[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)]) + def groupWith[W1: ClassManifest, W2: ClassManifest](other1: RDD[(K, W1)], other2: RDD[(K, W2)]) : RDD[(K, (Seq[V], Seq[W1], Seq[W2]))] = { cogroup(other1, other2, defaultPartitioner(self, other1, other2)) } @@ -692,6 +703,17 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) */ def values: RDD[V] = self.map(_._2) + + def indexed(numPartitions: Int): IndexedRDD[K,V] = + IndexedRDD(self.partitionBy(new HashPartitioner(numPartitions))) + + def indexed(partitioner: Partitioner): IndexedRDD[K,V] = + IndexedRDD(self.partitionBy(partitioner)) + + + def indexed(existingIndex: RDDIndex[K] = null): IndexedRDD[K,V] = + IndexedRDD(self, existingIndex) + private[spark] def getKeyClass() = implicitly[ClassManifest[K]].erasure private[spark] def getValueClass() = implicitly[ClassManifest[V]].erasure diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 1893627ee2ce3..6776220835fe4 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -792,6 +792,23 @@ abstract class RDD[T: ClassManifest]( return buf.toArray } + + + /** + * For RDD[(K,V)] this function returns a pair-functions object for this RDD + */ + def pairRDDFunctions[K, V]( + implicit t: T <:< (K, V), k: ClassManifest[K], v: ClassManifest[V]): + PairRDDFunctions[K, V] = { + new PairRDDFunctions(this.asInstanceOf[RDD[(K,V)]]) + } + + + def makeIndex(partitioner: Option[Partitioner] = None): RDDIndex[T] = + IndexedRDD.makeIndex(this, partitioner) + + + /** * Return the first element in this RDD. */ diff --git a/core/src/test/scala/org/apache/spark/rdd/IndexedRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/IndexedRDDSuite.scala new file mode 100644 index 0000000000000..3a2ce4e4da4c6 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/rdd/IndexedRDDSuite.scala @@ -0,0 +1,461 @@ +/* + * 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.rdd + + +import org.scalatest.FunSuite +import org.scalatest.prop.Checkers +import org.scalacheck.Arbitrary._ +import org.scalacheck.Gen +import org.scalacheck.Prop._ + +import com.google.common.io.Files + +import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.HashSet + +import org.apache.spark.rdd.RDD +import org.apache.spark.rdd.ShuffledRDD +import org.apache.spark.rdd.IndexedRDD + +import org.apache.spark.SparkContext._ +import org.apache.spark._ + + + +class IndexedRDDSuite extends FunSuite with SharedSparkContext { + + def lineage(rdd: RDD[_]): collection.mutable.HashSet[RDD[_]] = { + val set = new collection.mutable.HashSet[RDD[_]] + def visit(rdd: RDD[_]) { + for (dep <- rdd.dependencies) { + set += dep.rdd + visit(dep.rdd) + } + } + visit(rdd) + set + } + + test("groupByKey") { + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1))).indexed() + val groups = pairs.groupByKey().collect() + assert(groups.size === 2) + val valuesFor1 = groups.find(_._1 == 1).get._2 + assert(valuesFor1.toList.sorted === List(1, 2, 3)) + val valuesFor2 = groups.find(_._1 == 2).get._2 + assert(valuesFor2.toList.sorted === List(1)) + } + + test("groupByKey with duplicates") { + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))).indexed() + val groups = pairs.groupByKey().collect() + assert(groups.size === 2) + val valuesFor1 = groups.find(_._1 == 1).get._2 + assert(valuesFor1.toList.sorted === List(1, 1, 2, 3)) + val valuesFor2 = groups.find(_._1 == 2).get._2 + assert(valuesFor2.toList.sorted === List(1)) + } + + test("groupByKey with negative key hash codes") { + val pairs = sc.parallelize(Array((-1, 1), (-1, 2), (-1, 3), (2, 1))).indexed() + val groups = pairs.groupByKey().collect() + assert(groups.size === 2) + val valuesForMinus1 = groups.find(_._1 == -1).get._2 + assert(valuesForMinus1.toList.sorted === List(1, 2, 3)) + val valuesFor2 = groups.find(_._1 == 2).get._2 + assert(valuesFor2.toList.sorted === List(1)) + } + + test("groupByKey with many output partitions") { + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1))).indexed(10) + val groups = pairs.groupByKey().collect() + assert(groups.size === 2) + val valuesFor1 = groups.find(_._1 == 1).get._2 + assert(valuesFor1.toList.sorted === List(1, 2, 3)) + val valuesFor2 = groups.find(_._1 == 2).get._2 + assert(valuesFor2.toList.sorted === List(1)) + } + + test("reduceByKey") { + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))).indexed() + val sums = pairs.reduceByKey(_+_).collect() + assert(sums.toSet === Set((1, 7), (2, 1))) + } + + test("reduceByKey with collectAsMap") { + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))).indexed() + val sums = pairs.reduceByKey(_+_).collectAsMap() + assert(sums.size === 2) + assert(sums(1) === 7) + assert(sums(2) === 1) + } + + test("reduceByKey with many output partitons") { + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))).indexed(10) + val sums = pairs.reduceByKey(_+_).collect() + assert(sums.toSet === Set((1, 7), (2, 1))) + } + + test("reduceByKey with partitioner") { + val p = new Partitioner() { + def numPartitions = 2 + def getPartition(key: Any) = key.asInstanceOf[Int] + } + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 1), (0, 1))).indexed(p) + val sums = pairs.reduceByKey(_+_) + assert(sums.collect().toSet === Set((1, 4), (0, 1))) + assert(sums.partitioner === Some(p)) + // count the dependencies to make sure there is only 1 ShuffledRDD + val deps = lineage(sums) + + assert(deps.filter(_.isInstanceOf[ShuffledRDD[_,_,_]]).size === 1) // ShuffledRDD, ParallelCollection + } + + + + test("joinIndexVsPair") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed() + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) + val joined = rdd1.join(rdd2).collect() + assert(joined.size === 4) + assert(joined.toSet === Set( + (1, (1, 'x')), + (1, (2, 'x')), + (2, (1, 'y')), + (2, (1, 'z')) + )) + } + + test("joinIndexVsIndex") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed() + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).indexed() + val joined = rdd1.join(rdd2).collect() + assert(joined.size === 4) + assert(joined.toSet === Set( + (1, (1, 'x')), + (1, (2, 'x')), + (2, (1, 'y')), + (2, (1, 'z')) + )) + } + + test("joinSharedIndex") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1), (4,-4), (4, 4) )).indexed() + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).indexed(rdd1.index) + val joined = rdd1.join(rdd2).collect() + assert(joined.size === 6) + assert(joined.toSet === Set( + (1, (1, 'x')), + (1, (2, 'x')), + (2, (1, 'y')), + (2, (1, 'z')), + (4, (-4, 'w')), + (4, (4, 'w')) + )) + } + + + test("join all-to-all") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (1, 3))).indexed() + val rdd2 = sc.parallelize(Array((1, 'x'), (1, 'y'))).indexed(rdd1.index) + val joined = rdd1.join(rdd2).collect() + assert(joined.size === 6) + assert(joined.toSet === Set( + (1, (1, 'x')), + (1, (1, 'y')), + (1, (2, 'x')), + (1, (2, 'y')), + (1, (3, 'x')), + (1, (3, 'y')) + )) + } + + test("leftOuterJoinIndex") { + val index = sc.parallelize( 1 to 6 ).makeIndex() + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed(index) + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) + val joined = rdd1.leftOuterJoin(rdd2).collect() + assert(joined.size === 5) + assert(joined.toSet === Set( + (1, (1, Some('x'))), + (1, (2, Some('x'))), + (2, (1, Some('y'))), + (2, (1, Some('z'))), + (3, (1, None)) + )) + } + + test("leftOuterJoinIndextoIndex") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed() + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).indexed() + val joined = rdd1.leftOuterJoin(rdd2).collect() + assert(joined.size === 5) + assert(joined.toSet === Set( + (1, (1, Some('x'))), + (1, (2, Some('x'))), + (2, (1, Some('y'))), + (2, (1, Some('z'))), + (3, (1, None)) + )) + } + + test("leftOuterJoinIndextoSharedIndex") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1), (4, -4))).indexed() + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).indexed(rdd1.index) + val joined = rdd1.leftOuterJoin(rdd2).collect() + assert(joined.size === 6) + assert(joined.toSet === Set( + (1, (1, Some('x'))), + (1, (2, Some('x'))), + (2, (1, Some('y'))), + (2, (1, Some('z'))), + (4, (-4, Some('w'))), + (3, (1, None)) + )) + } + +test("leftOuterJoinIndextoIndexExternal") { + val index = sc.parallelize( 1 to 6 ).makeIndex() + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed(index) + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).indexed(index) + val joined = rdd1.leftOuterJoin(rdd2).collect() + assert(joined.size === 5) + assert(joined.toSet === Set( + (1, (1, Some('x'))), + (1, (2, Some('x'))), + (2, (1, Some('y'))), + (2, (1, Some('z'))), + (3, (1, None)) + )) + } + + + test("rightOuterJoin") { + val index = sc.parallelize( 1 to 6 ).makeIndex() + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed(index) + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) + val joined = rdd1.rightOuterJoin(rdd2).collect() + assert(joined.size === 5) + assert(joined.toSet === Set( + (1, (Some(1), 'x')), + (1, (Some(2), 'x')), + (2, (Some(1), 'y')), + (2, (Some(1), 'z')), + (4, (None, 'w')) + )) + } + + test("rightOuterJoinIndex2Index") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed() + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).indexed() + val joined = rdd1.rightOuterJoin(rdd2).collect() + assert(joined.size === 5) + assert(joined.toSet === Set( + (1, (Some(1), 'x')), + (1, (Some(2), 'x')), + (2, (Some(1), 'y')), + (2, (Some(1), 'z')), + (4, (None, 'w')) + )) + } + + + test("rightOuterJoinIndex2Indexshared") { + val index = sc.parallelize( 1 to 6 ).makeIndex() + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed(index) + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).indexed(index) + val joined = rdd1.rightOuterJoin(rdd2).collect() + assert(joined.size === 5) + assert(joined.toSet === Set( + (1, (Some(1), 'x')), + (1, (Some(2), 'x')), + (2, (Some(1), 'y')), + (2, (Some(1), 'z')), + (4, (None, 'w')) + )) + } + + + test("join with no matches index") { + val index = IndexedRDD.makeIndex( sc.parallelize( 1 to 6 ) ) + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed(index) + val rdd2 = sc.parallelize(Array((4, 'x'), (5, 'y'), (5, 'z'), (6, 'w'))) + val joined = rdd1.join(rdd2).collect() + assert(joined.size === 0) + } + + test("join with no matches shared index") { + val index = IndexedRDD.makeIndex( sc.parallelize( 1 to 6 ) ) + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed(index) + val rdd2 = sc.parallelize(Array((4, 'x'), (5, 'y'), (5, 'z'), (6, 'w'))).indexed(index) + val joined = rdd1.join(rdd2).collect() + assert(joined.size === 0) + } + + + test("join with many output partitions") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed(10) + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) + val joined = rdd1.join(rdd2).collect() + assert(joined.size === 4) + assert(joined.toSet === Set( + (1, (1, 'x')), + (1, (2, 'x')), + (2, (1, 'y')), + (2, (1, 'z')) + )) + } + + test("join with many output partitions and two indices") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed(10) + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).indexed(20) + val joined = rdd1.join(rdd2).collect() + assert(joined.size === 4) + assert(joined.toSet === Set( + (1, (1, 'x')), + (1, (2, 'x')), + (2, (1, 'y')), + (2, (1, 'z')) + )) + } + + + test("groupWith") { + val index = IndexedRDD.makeIndex( sc.parallelize( 1 to 6 ) ) + + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed(index) + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).indexed(index) + val joined = rdd1.groupWith(rdd2).collect() + assert(joined.size === 4) + assert(joined.toSet === Set( + (1, (ArrayBuffer(1, 2), ArrayBuffer('x'))), + (2, (ArrayBuffer(1), ArrayBuffer('y', 'z'))), + (3, (ArrayBuffer(1), ArrayBuffer())), + (4, (ArrayBuffer(), ArrayBuffer('w'))) + )) + } + + test("zero-partition RDD") { + val emptyDir = Files.createTempDir() + val file = sc.textFile(emptyDir.getAbsolutePath) + assert(file.partitions.size == 0) + assert(file.collect().toList === Nil) + // Test that a shuffle on the file works, because this used to be a bug + assert(file.map(line => (line, 1)).reduceByKey(_ + _).collect().toList === Nil) + } + + test("keys and values") { + val rdd = sc.parallelize(Array((1, "a"), (2, "b"))).indexed() + assert(rdd.keys.collect().toList === List(1, 2)) + assert(rdd.values.collect().toList === List("a", "b")) + } + + test("default partitioner uses partition size") { + // specify 2000 partitions + val a = sc.makeRDD(Array(1, 2, 3, 4), 2000) + // do a map, which loses the partitioner + val b = a.map(a => (a, (a * 2).toString)) + // then a group by, and see we didn't revert to 2 partitions + val c = b.groupByKey() + assert(c.partitions.size === 2000) + } + + // test("default partitioner uses largest partitioner indexed to indexed") { + // val a = sc.makeRDD(Array((1, "a"), (2, "b")), 2).indexed() + // val b = sc.makeRDD(Array((1, "a"), (2, "b")), 2000).indexed() + // val c = a.join(b) + // assert(c.partitions.size === 2000) + // } + + + + test("subtract") { + val a = sc.parallelize(Array(1, 2, 3), 2) + val b = sc.parallelize(Array(2, 3, 4), 4) + val c = a.subtract(b) + assert(c.collect().toSet === Set(1)) + assert(c.partitions.size === a.partitions.size) + } + + test("subtract with narrow dependency") { + // use a deterministic partitioner + val p = new Partitioner() { + def numPartitions = 5 + def getPartition(key: Any) = key.asInstanceOf[Int] + } + // partitionBy so we have a narrow dependency + val a = sc.parallelize(Array((1, "a"), (2, "b"), (3, "c"))).indexed(p) + // more partitions/no partitioner so a shuffle dependency + val b = sc.parallelize(Array((2, "b"), (3, "cc"), (4, "d")), 4) + val c = a.subtract(b) + assert(c.collect().toSet === Set((1, "a"), (3, "c"))) + // Ideally we could keep the original partitioner... + assert(c.partitioner === None) + } + + test("subtractByKey") { + + val a = sc.parallelize(Array((1, "a"), (1, "a"), (2, "b"), (3, "c")), 2).indexed() + val b = sc.parallelize(Array((2, 20), (3, 30), (4, 40)), 4) + val c = a.subtractByKey(b) + assert(c.collect().toSet === Set((1, "a"), (1, "a"))) + assert(c.partitions.size === a.partitions.size) + } + + // test("subtractByKey with narrow dependency") { + // // use a deterministic partitioner + // val p = new Partitioner() { + // def numPartitions = 5 + // def getPartition(key: Any) = key.asInstanceOf[Int] + // } + + // val index = sc.parallelize( 1 to 6 ).makeIndex(Some(p)) + // // partitionBy so we have a narrow dependency + // val a = sc.parallelize(Array((1, "a"), (1, "a"), (2, "b"), (3, "c"))).indexed(index) + // // more partitions/no partitioner so a shuffle dependency + // val b = sc.parallelize(Array((2, "b"), (3, "cc"), (4, "d")), 4).indexed(index) + // val c = a.subtractByKey(b) + // assert(c.collect().toSet === Set((1, "a"), (1, "a"))) + // assert(c.partitioner.get === p) + // } + + test("foldByKey") { + val index = IndexedRDD.makeIndex( sc.parallelize( 1 to 6 ) ) + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))).indexed(index) + val sums = pairs.foldByKey(0)(_+_).collect() + assert(sums.toSet === Set((1, 7), (2, 1))) + } + + test("foldByKey with mutable result type") { + val index = IndexedRDD.makeIndex( sc.parallelize( 1 to 6 ) ) + + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))).indexed(index) + val bufs = pairs.mapValues(v => ArrayBuffer(v)).cache() + // Fold the values using in-place mutation + val sums = bufs.foldByKey(new ArrayBuffer[Int])(_ ++= _).collect() + assert(sums.toSet === Set((1, ArrayBuffer(1, 2, 3, 1)), (2, ArrayBuffer(1)))) + // Check that the mutable objects in the original RDD were not changed + assert(bufs.collect().toSet === Set( + (1, ArrayBuffer(1)), + (1, ArrayBuffer(2)), + (1, ArrayBuffer(3)), + (1, ArrayBuffer(1)), + (2, ArrayBuffer(1)))) + } +} From bf059691f07b59d81b061033a01f1173cc7b2458 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Mon, 14 Oct 2013 19:59:11 -0700 Subject: [PATCH 107/531] Adding a few extra comments. --- .../org/apache/spark/rdd/IndexedRDD.scala | 25 +++++++++++-------- .../main/scala/org/apache/spark/rdd/RDD.scala | 4 +++ 2 files changed, 18 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/IndexedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/IndexedRDD.scala index 8d2e9782c2936..a881ee3a1da35 100644 --- a/core/src/main/scala/org/apache/spark/rdd/IndexedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/IndexedRDD.scala @@ -59,8 +59,6 @@ class RDDIndex[@specialized K: ClassManifest](private[spark] val rdd: RDD[BlockI - - /** * An IndexedRDD[K,V] extends the RDD[(K,V)] by pre-indexing the keys and * organizing the values to enable faster join operations. @@ -75,10 +73,12 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( extends RDD[(K, V)](index.rdd.context, List(new OneToOneDependency(index.rdd), new OneToOneDependency(valuesRDD)) ) { + /** * An internal representation which joins the block indices with the values */ - protected[spark] val tuples = new ZippedRDD(index.rdd.context, index.rdd, valuesRDD) + protected[spark] val tuples = + new ZippedRDD(index.rdd.context, index.rdd, valuesRDD) /** @@ -87,14 +87,12 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( override val partitioner = index.rdd.partitioner - - - /** * The actual partitions are defined by the tuples. */ override def getPartitions: Array[Partition] = tuples.getPartitions + /** * The preferred locations are computed based on the preferred locations of the tuples. */ @@ -112,6 +110,9 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( } + /** + * The IndexedRDD has its own optimized version of the pairRDDFunctions. + */ override def pairRDDFunctions[K1, V1]( implicit t: (K, V) <:< (K1,V1), k: ClassManifest[K1], v: ClassManifest[V1]): PairRDDFunctions[K1, V1] = { @@ -119,9 +120,6 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( } - - - /** * Provide the RDD[(K,V)] equivalent output. */ @@ -144,11 +142,16 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( object IndexedRDD { + + /** + * Construct an IndexedRDD from a regular RDD[(K,V)] using an existing index + * if one is provided. + */ def apply[K: ClassManifest, V: ClassManifest]( tbl: RDD[(K,V)], existingIndex: RDDIndex[K] = null ): IndexedRDD[K, V] = { - if (existingIndex == null) { + if (existingIndex == null) { // If no index was provided // Shuffle the table (if necessary) val shuffledTbl = if (tbl.partitioner.isEmpty) { @@ -248,7 +251,7 @@ object IndexedRDD { new RDDIndex(index) } -} +} // end of object IndexedRDD diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 6776220835fe4..f8a969cf103e7 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -804,6 +804,10 @@ abstract class RDD[T: ClassManifest]( } + /** + * Construct an index over the unique elements in this RDD. The + * index can then be used to organize a RDD[(T,V)]. + */ def makeIndex(partitioner: Option[Partitioner] = None): RDDIndex[T] = IndexedRDD.makeIndex(this, partitioner) From 4755f42d7835a5e584580167fc12644b12530aaa Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Mon, 14 Oct 2013 23:13:27 -0700 Subject: [PATCH 108/531] moving indexedrdd to the correct location --- .../scala/org/apache/spark/{ => rdd}/IndexedRDDFunctions.scala | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename core/src/main/scala/org/apache/spark/{ => rdd}/IndexedRDDFunctions.scala (100%) diff --git a/core/src/main/scala/org/apache/spark/IndexedRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/IndexedRDDFunctions.scala similarity index 100% rename from core/src/main/scala/org/apache/spark/IndexedRDDFunctions.scala rename to core/src/main/scala/org/apache/spark/rdd/IndexedRDDFunctions.scala From 6700ccd7d5faa1f3147c72b3bba026d6dfae1773 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Mon, 14 Oct 2013 19:37:52 -0700 Subject: [PATCH 109/531] Introducing indexedrdd The rest of indexed rdd --- .../apache/spark/rdd/IndexedRDDSuite.scala | 461 ++++++++++++++++++ 1 file changed, 461 insertions(+) create mode 100644 core/src/test/scala/org/apache/spark/rdd/IndexedRDDSuite.scala diff --git a/core/src/test/scala/org/apache/spark/rdd/IndexedRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/IndexedRDDSuite.scala new file mode 100644 index 0000000000000..3a2ce4e4da4c6 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/rdd/IndexedRDDSuite.scala @@ -0,0 +1,461 @@ +/* + * 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.rdd + + +import org.scalatest.FunSuite +import org.scalatest.prop.Checkers +import org.scalacheck.Arbitrary._ +import org.scalacheck.Gen +import org.scalacheck.Prop._ + +import com.google.common.io.Files + +import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.HashSet + +import org.apache.spark.rdd.RDD +import org.apache.spark.rdd.ShuffledRDD +import org.apache.spark.rdd.IndexedRDD + +import org.apache.spark.SparkContext._ +import org.apache.spark._ + + + +class IndexedRDDSuite extends FunSuite with SharedSparkContext { + + def lineage(rdd: RDD[_]): collection.mutable.HashSet[RDD[_]] = { + val set = new collection.mutable.HashSet[RDD[_]] + def visit(rdd: RDD[_]) { + for (dep <- rdd.dependencies) { + set += dep.rdd + visit(dep.rdd) + } + } + visit(rdd) + set + } + + test("groupByKey") { + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1))).indexed() + val groups = pairs.groupByKey().collect() + assert(groups.size === 2) + val valuesFor1 = groups.find(_._1 == 1).get._2 + assert(valuesFor1.toList.sorted === List(1, 2, 3)) + val valuesFor2 = groups.find(_._1 == 2).get._2 + assert(valuesFor2.toList.sorted === List(1)) + } + + test("groupByKey with duplicates") { + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))).indexed() + val groups = pairs.groupByKey().collect() + assert(groups.size === 2) + val valuesFor1 = groups.find(_._1 == 1).get._2 + assert(valuesFor1.toList.sorted === List(1, 1, 2, 3)) + val valuesFor2 = groups.find(_._1 == 2).get._2 + assert(valuesFor2.toList.sorted === List(1)) + } + + test("groupByKey with negative key hash codes") { + val pairs = sc.parallelize(Array((-1, 1), (-1, 2), (-1, 3), (2, 1))).indexed() + val groups = pairs.groupByKey().collect() + assert(groups.size === 2) + val valuesForMinus1 = groups.find(_._1 == -1).get._2 + assert(valuesForMinus1.toList.sorted === List(1, 2, 3)) + val valuesFor2 = groups.find(_._1 == 2).get._2 + assert(valuesFor2.toList.sorted === List(1)) + } + + test("groupByKey with many output partitions") { + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1))).indexed(10) + val groups = pairs.groupByKey().collect() + assert(groups.size === 2) + val valuesFor1 = groups.find(_._1 == 1).get._2 + assert(valuesFor1.toList.sorted === List(1, 2, 3)) + val valuesFor2 = groups.find(_._1 == 2).get._2 + assert(valuesFor2.toList.sorted === List(1)) + } + + test("reduceByKey") { + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))).indexed() + val sums = pairs.reduceByKey(_+_).collect() + assert(sums.toSet === Set((1, 7), (2, 1))) + } + + test("reduceByKey with collectAsMap") { + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))).indexed() + val sums = pairs.reduceByKey(_+_).collectAsMap() + assert(sums.size === 2) + assert(sums(1) === 7) + assert(sums(2) === 1) + } + + test("reduceByKey with many output partitons") { + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))).indexed(10) + val sums = pairs.reduceByKey(_+_).collect() + assert(sums.toSet === Set((1, 7), (2, 1))) + } + + test("reduceByKey with partitioner") { + val p = new Partitioner() { + def numPartitions = 2 + def getPartition(key: Any) = key.asInstanceOf[Int] + } + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 1), (0, 1))).indexed(p) + val sums = pairs.reduceByKey(_+_) + assert(sums.collect().toSet === Set((1, 4), (0, 1))) + assert(sums.partitioner === Some(p)) + // count the dependencies to make sure there is only 1 ShuffledRDD + val deps = lineage(sums) + + assert(deps.filter(_.isInstanceOf[ShuffledRDD[_,_,_]]).size === 1) // ShuffledRDD, ParallelCollection + } + + + + test("joinIndexVsPair") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed() + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) + val joined = rdd1.join(rdd2).collect() + assert(joined.size === 4) + assert(joined.toSet === Set( + (1, (1, 'x')), + (1, (2, 'x')), + (2, (1, 'y')), + (2, (1, 'z')) + )) + } + + test("joinIndexVsIndex") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed() + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).indexed() + val joined = rdd1.join(rdd2).collect() + assert(joined.size === 4) + assert(joined.toSet === Set( + (1, (1, 'x')), + (1, (2, 'x')), + (2, (1, 'y')), + (2, (1, 'z')) + )) + } + + test("joinSharedIndex") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1), (4,-4), (4, 4) )).indexed() + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).indexed(rdd1.index) + val joined = rdd1.join(rdd2).collect() + assert(joined.size === 6) + assert(joined.toSet === Set( + (1, (1, 'x')), + (1, (2, 'x')), + (2, (1, 'y')), + (2, (1, 'z')), + (4, (-4, 'w')), + (4, (4, 'w')) + )) + } + + + test("join all-to-all") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (1, 3))).indexed() + val rdd2 = sc.parallelize(Array((1, 'x'), (1, 'y'))).indexed(rdd1.index) + val joined = rdd1.join(rdd2).collect() + assert(joined.size === 6) + assert(joined.toSet === Set( + (1, (1, 'x')), + (1, (1, 'y')), + (1, (2, 'x')), + (1, (2, 'y')), + (1, (3, 'x')), + (1, (3, 'y')) + )) + } + + test("leftOuterJoinIndex") { + val index = sc.parallelize( 1 to 6 ).makeIndex() + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed(index) + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) + val joined = rdd1.leftOuterJoin(rdd2).collect() + assert(joined.size === 5) + assert(joined.toSet === Set( + (1, (1, Some('x'))), + (1, (2, Some('x'))), + (2, (1, Some('y'))), + (2, (1, Some('z'))), + (3, (1, None)) + )) + } + + test("leftOuterJoinIndextoIndex") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed() + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).indexed() + val joined = rdd1.leftOuterJoin(rdd2).collect() + assert(joined.size === 5) + assert(joined.toSet === Set( + (1, (1, Some('x'))), + (1, (2, Some('x'))), + (2, (1, Some('y'))), + (2, (1, Some('z'))), + (3, (1, None)) + )) + } + + test("leftOuterJoinIndextoSharedIndex") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1), (4, -4))).indexed() + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).indexed(rdd1.index) + val joined = rdd1.leftOuterJoin(rdd2).collect() + assert(joined.size === 6) + assert(joined.toSet === Set( + (1, (1, Some('x'))), + (1, (2, Some('x'))), + (2, (1, Some('y'))), + (2, (1, Some('z'))), + (4, (-4, Some('w'))), + (3, (1, None)) + )) + } + +test("leftOuterJoinIndextoIndexExternal") { + val index = sc.parallelize( 1 to 6 ).makeIndex() + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed(index) + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).indexed(index) + val joined = rdd1.leftOuterJoin(rdd2).collect() + assert(joined.size === 5) + assert(joined.toSet === Set( + (1, (1, Some('x'))), + (1, (2, Some('x'))), + (2, (1, Some('y'))), + (2, (1, Some('z'))), + (3, (1, None)) + )) + } + + + test("rightOuterJoin") { + val index = sc.parallelize( 1 to 6 ).makeIndex() + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed(index) + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) + val joined = rdd1.rightOuterJoin(rdd2).collect() + assert(joined.size === 5) + assert(joined.toSet === Set( + (1, (Some(1), 'x')), + (1, (Some(2), 'x')), + (2, (Some(1), 'y')), + (2, (Some(1), 'z')), + (4, (None, 'w')) + )) + } + + test("rightOuterJoinIndex2Index") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed() + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).indexed() + val joined = rdd1.rightOuterJoin(rdd2).collect() + assert(joined.size === 5) + assert(joined.toSet === Set( + (1, (Some(1), 'x')), + (1, (Some(2), 'x')), + (2, (Some(1), 'y')), + (2, (Some(1), 'z')), + (4, (None, 'w')) + )) + } + + + test("rightOuterJoinIndex2Indexshared") { + val index = sc.parallelize( 1 to 6 ).makeIndex() + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed(index) + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).indexed(index) + val joined = rdd1.rightOuterJoin(rdd2).collect() + assert(joined.size === 5) + assert(joined.toSet === Set( + (1, (Some(1), 'x')), + (1, (Some(2), 'x')), + (2, (Some(1), 'y')), + (2, (Some(1), 'z')), + (4, (None, 'w')) + )) + } + + + test("join with no matches index") { + val index = IndexedRDD.makeIndex( sc.parallelize( 1 to 6 ) ) + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed(index) + val rdd2 = sc.parallelize(Array((4, 'x'), (5, 'y'), (5, 'z'), (6, 'w'))) + val joined = rdd1.join(rdd2).collect() + assert(joined.size === 0) + } + + test("join with no matches shared index") { + val index = IndexedRDD.makeIndex( sc.parallelize( 1 to 6 ) ) + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed(index) + val rdd2 = sc.parallelize(Array((4, 'x'), (5, 'y'), (5, 'z'), (6, 'w'))).indexed(index) + val joined = rdd1.join(rdd2).collect() + assert(joined.size === 0) + } + + + test("join with many output partitions") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed(10) + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) + val joined = rdd1.join(rdd2).collect() + assert(joined.size === 4) + assert(joined.toSet === Set( + (1, (1, 'x')), + (1, (2, 'x')), + (2, (1, 'y')), + (2, (1, 'z')) + )) + } + + test("join with many output partitions and two indices") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed(10) + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).indexed(20) + val joined = rdd1.join(rdd2).collect() + assert(joined.size === 4) + assert(joined.toSet === Set( + (1, (1, 'x')), + (1, (2, 'x')), + (2, (1, 'y')), + (2, (1, 'z')) + )) + } + + + test("groupWith") { + val index = IndexedRDD.makeIndex( sc.parallelize( 1 to 6 ) ) + + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed(index) + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).indexed(index) + val joined = rdd1.groupWith(rdd2).collect() + assert(joined.size === 4) + assert(joined.toSet === Set( + (1, (ArrayBuffer(1, 2), ArrayBuffer('x'))), + (2, (ArrayBuffer(1), ArrayBuffer('y', 'z'))), + (3, (ArrayBuffer(1), ArrayBuffer())), + (4, (ArrayBuffer(), ArrayBuffer('w'))) + )) + } + + test("zero-partition RDD") { + val emptyDir = Files.createTempDir() + val file = sc.textFile(emptyDir.getAbsolutePath) + assert(file.partitions.size == 0) + assert(file.collect().toList === Nil) + // Test that a shuffle on the file works, because this used to be a bug + assert(file.map(line => (line, 1)).reduceByKey(_ + _).collect().toList === Nil) + } + + test("keys and values") { + val rdd = sc.parallelize(Array((1, "a"), (2, "b"))).indexed() + assert(rdd.keys.collect().toList === List(1, 2)) + assert(rdd.values.collect().toList === List("a", "b")) + } + + test("default partitioner uses partition size") { + // specify 2000 partitions + val a = sc.makeRDD(Array(1, 2, 3, 4), 2000) + // do a map, which loses the partitioner + val b = a.map(a => (a, (a * 2).toString)) + // then a group by, and see we didn't revert to 2 partitions + val c = b.groupByKey() + assert(c.partitions.size === 2000) + } + + // test("default partitioner uses largest partitioner indexed to indexed") { + // val a = sc.makeRDD(Array((1, "a"), (2, "b")), 2).indexed() + // val b = sc.makeRDD(Array((1, "a"), (2, "b")), 2000).indexed() + // val c = a.join(b) + // assert(c.partitions.size === 2000) + // } + + + + test("subtract") { + val a = sc.parallelize(Array(1, 2, 3), 2) + val b = sc.parallelize(Array(2, 3, 4), 4) + val c = a.subtract(b) + assert(c.collect().toSet === Set(1)) + assert(c.partitions.size === a.partitions.size) + } + + test("subtract with narrow dependency") { + // use a deterministic partitioner + val p = new Partitioner() { + def numPartitions = 5 + def getPartition(key: Any) = key.asInstanceOf[Int] + } + // partitionBy so we have a narrow dependency + val a = sc.parallelize(Array((1, "a"), (2, "b"), (3, "c"))).indexed(p) + // more partitions/no partitioner so a shuffle dependency + val b = sc.parallelize(Array((2, "b"), (3, "cc"), (4, "d")), 4) + val c = a.subtract(b) + assert(c.collect().toSet === Set((1, "a"), (3, "c"))) + // Ideally we could keep the original partitioner... + assert(c.partitioner === None) + } + + test("subtractByKey") { + + val a = sc.parallelize(Array((1, "a"), (1, "a"), (2, "b"), (3, "c")), 2).indexed() + val b = sc.parallelize(Array((2, 20), (3, 30), (4, 40)), 4) + val c = a.subtractByKey(b) + assert(c.collect().toSet === Set((1, "a"), (1, "a"))) + assert(c.partitions.size === a.partitions.size) + } + + // test("subtractByKey with narrow dependency") { + // // use a deterministic partitioner + // val p = new Partitioner() { + // def numPartitions = 5 + // def getPartition(key: Any) = key.asInstanceOf[Int] + // } + + // val index = sc.parallelize( 1 to 6 ).makeIndex(Some(p)) + // // partitionBy so we have a narrow dependency + // val a = sc.parallelize(Array((1, "a"), (1, "a"), (2, "b"), (3, "c"))).indexed(index) + // // more partitions/no partitioner so a shuffle dependency + // val b = sc.parallelize(Array((2, "b"), (3, "cc"), (4, "d")), 4).indexed(index) + // val c = a.subtractByKey(b) + // assert(c.collect().toSet === Set((1, "a"), (1, "a"))) + // assert(c.partitioner.get === p) + // } + + test("foldByKey") { + val index = IndexedRDD.makeIndex( sc.parallelize( 1 to 6 ) ) + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))).indexed(index) + val sums = pairs.foldByKey(0)(_+_).collect() + assert(sums.toSet === Set((1, 7), (2, 1))) + } + + test("foldByKey with mutable result type") { + val index = IndexedRDD.makeIndex( sc.parallelize( 1 to 6 ) ) + + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))).indexed(index) + val bufs = pairs.mapValues(v => ArrayBuffer(v)).cache() + // Fold the values using in-place mutation + val sums = bufs.foldByKey(new ArrayBuffer[Int])(_ ++= _).collect() + assert(sums.toSet === Set((1, ArrayBuffer(1, 2, 3, 1)), (2, ArrayBuffer(1)))) + // Check that the mutable objects in the original RDD were not changed + assert(bufs.collect().toSet === Set( + (1, ArrayBuffer(1)), + (1, ArrayBuffer(2)), + (1, ArrayBuffer(3)), + (1, ArrayBuffer(1)), + (2, ArrayBuffer(1)))) + } +} From 6a13d0231991753b359aaf4f24f9da0c006c4ca1 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Mon, 14 Oct 2013 19:59:11 -0700 Subject: [PATCH 110/531] Merging chagnes for IndexedRDD branch --- .../org/apache/spark/rdd/IndexedRDD.scala | 25 +++++++++++-------- .../main/scala/org/apache/spark/rdd/RDD.scala | 5 +++- 2 files changed, 18 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/IndexedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/IndexedRDD.scala index 8d2e9782c2936..a881ee3a1da35 100644 --- a/core/src/main/scala/org/apache/spark/rdd/IndexedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/IndexedRDD.scala @@ -59,8 +59,6 @@ class RDDIndex[@specialized K: ClassManifest](private[spark] val rdd: RDD[BlockI - - /** * An IndexedRDD[K,V] extends the RDD[(K,V)] by pre-indexing the keys and * organizing the values to enable faster join operations. @@ -75,10 +73,12 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( extends RDD[(K, V)](index.rdd.context, List(new OneToOneDependency(index.rdd), new OneToOneDependency(valuesRDD)) ) { + /** * An internal representation which joins the block indices with the values */ - protected[spark] val tuples = new ZippedRDD(index.rdd.context, index.rdd, valuesRDD) + protected[spark] val tuples = + new ZippedRDD(index.rdd.context, index.rdd, valuesRDD) /** @@ -87,14 +87,12 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( override val partitioner = index.rdd.partitioner - - - /** * The actual partitions are defined by the tuples. */ override def getPartitions: Array[Partition] = tuples.getPartitions + /** * The preferred locations are computed based on the preferred locations of the tuples. */ @@ -112,6 +110,9 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( } + /** + * The IndexedRDD has its own optimized version of the pairRDDFunctions. + */ override def pairRDDFunctions[K1, V1]( implicit t: (K, V) <:< (K1,V1), k: ClassManifest[K1], v: ClassManifest[V1]): PairRDDFunctions[K1, V1] = { @@ -119,9 +120,6 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( } - - - /** * Provide the RDD[(K,V)] equivalent output. */ @@ -144,11 +142,16 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( object IndexedRDD { + + /** + * Construct an IndexedRDD from a regular RDD[(K,V)] using an existing index + * if one is provided. + */ def apply[K: ClassManifest, V: ClassManifest]( tbl: RDD[(K,V)], existingIndex: RDDIndex[K] = null ): IndexedRDD[K, V] = { - if (existingIndex == null) { + if (existingIndex == null) { // If no index was provided // Shuffle the table (if necessary) val shuffledTbl = if (tbl.partitioner.isEmpty) { @@ -248,7 +251,7 @@ object IndexedRDD { new RDDIndex(index) } -} +} // end of object IndexedRDD diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 6776220835fe4..16eaffb0fcab9 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -804,11 +804,14 @@ abstract class RDD[T: ClassManifest]( } + /** + * Construct an index over the unique elements in this RDD. The + * index can then be used to organize a RDD[(T,V)]. + */ def makeIndex(partitioner: Option[Partitioner] = None): RDDIndex[T] = IndexedRDD.makeIndex(this, partitioner) - /** * Return the first element in this RDD. */ From e7d03200008432f53c398310721389ddd47ff48f Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Tue, 15 Oct 2013 02:20:06 -0700 Subject: [PATCH 111/531] More refactoring and documentating including renaming data to attr for vertex and edge data and eliminating the vertex type. --- .../org/apache/spark/graph/Analytics.scala | 22 +- .../scala/org/apache/spark/graph/Edge.scala | 27 +- .../org/apache/spark/graph/EdgeTriplet.scala | 48 ++-- .../scala/org/apache/spark/graph/Graph.scala | 233 +++++----------- .../spark/graph/GraphKryoRegistrator.scala | 1 - .../org/apache/spark/graph/GraphLab.scala | 29 +- .../org/apache/spark/graph/GraphLoader.scala | 2 +- .../org/apache/spark/graph/GraphOps.scala | 147 +++++++++- .../scala/org/apache/spark/graph/Pregel.scala | 3 +- .../scala/org/apache/spark/graph/Vertex.scala | 15 -- .../spark/graph/impl/EdgePartition.scala | 18 +- .../apache/spark/graph/impl/GraphImpl.scala | 251 ++++-------------- .../spark/graph/util/GraphGenerators.scala | 2 +- 13 files changed, 337 insertions(+), 461 deletions(-) delete mode 100644 graph/src/main/scala/org/apache/spark/graph/Vertex.scala diff --git a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala index 9e2b54541b8ae..92632db491beb 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala @@ -17,24 +17,17 @@ object Analytics extends Logging { (vid, vdata, deg) => (deg.getOrElse(0), 1.0) } - println("Vertex Replication: " + pagerankGraph.replication) + println(pagerankGraph.statistics) - val edgeCounts = pagerankGraph.balance - - println("Edge Balance: " + (edgeCounts.max.toDouble / edgeCounts.min ) ) - println("Min edge block: " + edgeCounts.min) - println("Max edge block: " + edgeCounts.max) - - - Pregel.iterate[(Int, Double), ED, Double](pagerankGraph)( (vid, data, a: Double) => (data._1, (resetProb + (1.0 - resetProb) * a)), // apply - (me_id, edge) => Some(edge.src.data._2 / edge.src.data._1), // gather + (me_id, edge) => Some(edge.srcAttr._2 / edge.srcAttr._1), // gather (a: Double, b: Double) => a + b, // merge 1.0, numIter).mapVertices{ case (id, (outDeg, r)) => r } } + /** * Compute the PageRank of a graph returning the pagerank of each vertex as an RDD */ @@ -47,14 +40,13 @@ object Analytics extends Logging { (id, data, degIter) => (degIter.sum, 1.0, 1.0) } - // Run PageRank GraphLab.iterate(pagerankGraph)( - (me_id, edge) => edge.src.data._2 / edge.src.data._1, // gather + (me_id, edge) => edge.srcAttr._2 / edge.srcAttr._1, // gather (a: Double, b: Double) => a + b, (id, data, a: Option[Double]) => (data._1, (resetProb + (1.0 - resetProb) * a.getOrElse(0.0)), data._2), // apply - (me_id, edge) => math.abs(edge.src.data._3 - edge.src.data._2) > tol, // scatter + (me_id, edge) => math.abs(edge.srcAttr._3 - edge.srcAttr._2) > tol, // scatter maxIter).mapVertices { case (vid, data) => data._2 } } @@ -68,10 +60,10 @@ object Analytics extends Logging { def connectedComponents[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], numIter: Int) = { val ccGraph = graph.mapVertices { case (vid, _) => vid } GraphLab.iterate(ccGraph)( - (me_id, edge) => edge.otherVertex(me_id).data, // gather + (me_id, edge) => edge.otherVertexAttr(me_id), // gather (a: Vid, b: Vid) => math.min(a, b), // merge (id, data, a: Option[Vid]) => math.min(data, a.getOrElse(Long.MaxValue)), // apply - (me_id, edge) => (edge.vertex(me_id).data < edge.otherVertex(me_id).data), // scatter + (me_id, edge) => (edge.vertexAttr(me_id) < edge.otherVertexAttr(me_id)), // scatter numIter, gatherDirection = EdgeDirection.Both, scatterDirection = EdgeDirection.Both ) diff --git a/graph/src/main/scala/org/apache/spark/graph/Edge.scala b/graph/src/main/scala/org/apache/spark/graph/Edge.scala index 20539b8af05b8..67b64540177fc 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Edge.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Edge.scala @@ -8,6 +8,27 @@ package org.apache.spark.graph * @tparam ED type of the edge attribute */ case class Edge[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED] ( - var src: Vid = 0, - var dst: Vid = 0, - var data: ED = nullValue[ED]) + var srcId: Vid = 0, + var dstId: Vid = 0, + var attr: ED = nullValue[ED]) { + + /** + * Given one vertex in the edge return the other vertex. + * + * @param vid the id one of the two vertices on the edge. + * @return the id of the other vertex on the edge. + */ + def otherVertexId(vid: Vid): Vid = + if (srcId == vid) dstId else { assert(dstId == vid); srcId } + + + /** + * Return the relative direction of the edge to the corresponding vertex. + * + * @param vid the id of one of the two vertices in the edge. + * @return the relative direction of the edge to the corresponding vertex. + */ + def relativeDirection(vid: Vid): EdgeDirection = + if (vid == srcId) EdgeDirection.Out else { assert(vid == dstId); EdgeDirection.In } + +} diff --git a/graph/src/main/scala/org/apache/spark/graph/EdgeTriplet.scala b/graph/src/main/scala/org/apache/spark/graph/EdgeTriplet.scala index 4ade1d7333d81..3833b38abe672 100644 --- a/graph/src/main/scala/org/apache/spark/graph/EdgeTriplet.scala +++ b/graph/src/main/scala/org/apache/spark/graph/EdgeTriplet.scala @@ -5,49 +5,51 @@ package org.apache.spark.graph * * @tparam VD the type of the vertex attribute. * @tparam ED the type of the edge attribute + * + * @todo specialize edge triplet for basic types, though when I last tried + * specializing I got a warning about inherenting from a type that is not + * a trait. */ -class EdgeTriplet[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) VD, - @specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED] { +class EdgeTriplet[VD: ClassManifest, ED: ClassManifest] extends Edge[ED] { +// class EdgeTriplet[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) VD: ClassManifest, +// @specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED: ClassManifest] extends Edge[ED] { + + /** - * The vertex (id and attribute) corresponding to the source vertex. + * The source vertex attribute */ - var src: Vertex[VD] = _ + var srcAttr: VD = nullValue[VD] /** - * The vertex (id and attribute) corresponding to the target vertex. + * The destination vertex attribute */ - var dst: Vertex[VD] = _ + var dstAttr: VD = nullValue[VD] /** - * The attribute associated with the edge. + * Set the edge properties of this triplet. */ - var data: ED = _ + protected[spark] def set(other: Edge[ED]) { + srcId = other.srcId + dstId = other.dstId + attr = other.attr + } /** * Given one vertex in the edge return the other vertex. * * @param vid the id one of the two vertices on the edge. - * @return the other vertex on the edge. + * @return the attribute for the other vertex on the edge. */ - def otherVertex(vid: Vid): Vertex[VD] = - if (src.id == vid) dst else { assert(dst.id == vid); src } + def otherVertexAttr(vid: Vid): VD = + if (srcId == vid) dstAttr else { assert(dstId == vid); srcAttr } /** * Get the vertex object for the given vertex in the edge. * * @param vid the id of one of the two vertices on the edge - * @return the vertex object with that id. - */ - def vertex(vid: Vid): Vertex[VD] = - if (src.id == vid) src else { assert(dst.id == vid); dst } - - /** - * Return the relative direction of the edge to the corresponding vertex. - * - * @param vid the id of one of the two vertices in the edge. - * @return the relative direction of the edge to the corresponding vertex. + * @return the attr for the vertex with that id. */ - def relativeDirection(vid: Vid): EdgeDirection = - if (vid == src.id) EdgeDirection.Out else { assert(vid == dst.id); EdgeDirection.In } + def vertexAttr(vid: Vid): VD = + if (srcId == vid) srcAttr else { assert(dstId == vid); dstAttr } } diff --git a/graph/src/main/scala/org/apache/spark/graph/Graph.scala b/graph/src/main/scala/org/apache/spark/graph/Graph.scala index 39c699ce8bae2..d9cb2c037deae 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Graph.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Graph.scala @@ -20,19 +20,20 @@ import org.apache.spark.util.ClosureCleaner abstract class Graph[VD: ClassManifest, ED: ClassManifest] { - def replication: Double - - def balance: Array[Int] + /** + * Compute statistics describing the graph representation. + */ + def statistics: Map[String, Any] /** * Get the vertices and their data. * + * @note vertex ids are unique. * @return An RDD containing the vertices in this graph * * @see Vertex for the vertex type. * - * @todo should vertices return tuples instead of vertex objects? */ def vertices: RDD[(Vid,VD)] @@ -98,7 +99,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * val rawGraph: Graph[(), ()] = Graph.textFile("hdfs://file") * val root = 42 * var bfsGraph = rawGraph - * .mapVertices[Int](v => if(v.id == 0) 0 else Math.MaxValue) + * .mapVertices[Int]((vid, data) => if(vid == root) 0 else Math.MaxValue) * }}} * */ @@ -108,7 +109,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * Construct a new graph where each the value of each edge is transformed by * the map operation. This function is not passed the vertex value for the * vertices adjacent to the edge. If vertex values are desired use the - * mapEdgesWithVertices function. + * mapTriplets function. * * @note This graph is not changed and that the new graph has the same * structure. As a consequence the underlying index structures can be @@ -141,7 +142,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * on the attributes associated with each vertex. * {{{ * val rawGraph: Graph[Int, Int] = someLoadFunction() - * val graph = rawGraph.mapEdgesWithVertices[Int]( edge => + * val graph = rawGraph.mapTriplets[Int]( edge => * edge.src.data - edge.dst.data) * }}} * @@ -150,14 +151,6 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { map: EdgeTriplet[VD, ED] => ED2): Graph[VD, ED2] - // /** - // * Remove edges conntecting vertices that are not in the graph. - // * - // * @todo remove this function and ensure that for a graph G=(V,E): - // * if (u,v) in E then u in V and v in V - // */ - // def correctEdges(): Graph[VD, ED] - /** * Construct a new graph with all the edges reversed. If this graph contains * an edge from a to b then the returned graph contains an edge from b to a. @@ -188,123 +181,60 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { vpred: (Vid, VD) => Boolean = ((v,d) => true) ): Graph[VD, ED] - // /** - // * Combine the attrributes of edges connecting the same vertices. - // * - // * @todo Do we want to support this function - // */ - // def combineEdges(reduce: (ED, ED) => ED): Graph[VD, ED] + /** + * @todo document function + */ def groupEdgeTriplets[ED2: ClassManifest](f: Iterator[EdgeTriplet[VD,ED]] => ED2 ): Graph[VD,ED2] - def groupEdges[ED2: ClassManifest](f: Iterator[Edge[ED]] => ED2 ): Graph[VD,ED2] - - - - def mapReduceTriplets[A: ClassManifest]( - mapFunc: EdgeTriplet[VD, ED] => Array[(Vid, A)], - reduceFunc: (A, A) => A) - : RDD[(Vid, A)] + /** + * @todo document function + */ + def groupEdges[ED2: ClassManifest](f: Iterator[Edge[ED]] => ED2 ): Graph[VD,ED2] - // /** - // * This function is used to compute a statistic for the neighborhood of each - // * vertex. - // * - // * This is one of the core functions in the Graph API in that enables - // * neighborhood level computation. For example this function can be used to - // * count neighbors satisfying a predicate or implement PageRank. - // * - // * @note The returned RDD may contain fewer entries than their are vertices - // * in the graph. This is because some vertices may not have neighbors or the - // * map function may return None for all neighbors. - // * - // * @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 mergeFunc 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 VD2 The returned type of the aggregation operation. - // * - // * @return A Spark.RDD containing tuples of vertex identifiers and thee - // * resulting value. Note that the returned RDD may contain fewer vertices - // * than in the original graph since some vertices may not have neighbors or - // * the map function could return None for all neighbors. - // * - // * @example We can use this function to compute the average follower age for - // * each user - // * {{{ - // * val graph: Graph[Int,Int] = loadGraph() - // * val averageFollowerAge: RDD[(Int, Int)] = - // * graph.aggregateNeighbors[(Int,Double)]( - // * (vid, edge) => (edge.otherVertex(vid).data, 1), - // * (a, b) => (a._1 + b._1, a._2 + b._2), - // * EdgeDirection.In) - // * .mapValues{ case (sum,followers) => sum.toDouble / followers} - // * }}} - // * - // */ - // def aggregateNeighbors[A: ClassManifest]( - // mapFunc: (Vid, EdgeTriplet[VD, ED]) => Option[A], - // mergeFunc: (A, A) => A, - // direction: EdgeDirection) - // : Graph[(VD, Option[A]), ED] /** - * This function is used to compute a statistic for the neighborhood of each - * vertex and returns a value for all vertices (including those without - * neighbors). - * - * This is one of the core functions in the Graph API in that enables - * neighborhood level computation. For example this function can be used to - * count neighbors satisfying a predicate or implement PageRank. - * - * @note Because the a default value is provided all vertices will have a - * corresponding entry in the returned RDD. - * - * @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 default the default value to use for each vertex if it has no - * neighbors or the map function repeatedly evaluates to none - * @param direction the direction of edges to consider (e.g., In, Out, Both). - * @tparam VD2 The returned type of the aggregation operation. - * - * @return A Spark.RDD containing tuples of vertex identifiers and - * their resulting value. There will be exactly one entry for ever vertex in - * the original graph. - * - * @example We can use this function to compute the average follower age - * for each user + * The mapReduceTriplets function is used to compute statistics about + * 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. + * + * @tparam A the type of "message" to be sent to each vertex + * + * @param mapFunc the user defined map function which returns 0 or + * 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 of + * the map phase. + * + * @example We can use this function to compute the inDegree of each + * vertex * {{{ - * val graph: Graph[Int,Int] = loadGraph() - * val averageFollowerAge: RDD[(Int, Int)] = - * graph.aggregateNeighbors[(Int,Double)]( - * (vid, edge) => (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} + * val rawGraph: Graph[(),()] = Graph.textFile("twittergraph") + * val inDeg: RDD[(Vid, Int)] = + * mapReduceTriplets[Int](et => Array((et.dst.id, 1)), _ + _) * }}} * - * @todo Should this return a graph with the new vertex values? - * + * @note By expressing computation at the edge level we achieve maximum + * parallelism. This is one of the core functions in the Graph API in that enables + * neighborhood level computation. For example this function can be used to + * count neighbors satisfying a predicate or implement PageRank. + * */ - def aggregateNeighbors[A: ClassManifest]( - mapFunc: (Vid, EdgeTriplet[VD, ED]) => Option[A], - reduceFunc: (A, A) => A, - direction: EdgeDirection) - : RDD[(Vid, A)] + def mapReduceTriplets[A: ClassManifest]( + mapFunc: EdgeTriplet[VD, ED] => Array[(Vid, A)], + reduceFunc: (A, A) => A) + : RDD[(Vid, A)] /** * Join the vertices with an RDD and then apply a function from the the - * vertex and RDD entry to a new vertex value and type. The input table should - * contain at most one entry for each vertex. If no entry is provided the - * map function is invoked passing none. + * vertex and RDD entry to a new vertex value and type. + * The input table should contain at most one entry for each vertex. + * If no entry is provided the map function is invoked passing none. * * @tparam U the type of entry in the table of updates * @tparam VD2 the new vertex value type @@ -320,64 +250,17 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * vertex record * {{{ * val rawGraph: Graph[(),()] = Graph.textFile("webgraph") - * val outDeg: RDD[(Int, Int)] = rawGraph.outDegrees() - * val graph = rawGraph.leftJoinVertices[Int,Int](outDeg, - * (v, deg) => deg.getOrElse(0) ) + * val outDeg: RDD[(Vid, Int)] = rawGraph.outDegrees() + * val graph = rawGraph.outerJoinVertices(outDeg) { + * (vid, data, optDeg) => optDeg.getOrElse(0) + * } * }}} * - * @todo Should this function be curried to enable type inference? For - * example - * {{{ - * graph.leftJoinVertices(tbl)( (v, row) => row.getOrElse(0) ) - * }}} - * @todo Is leftJoinVertices the right name? */ def outerJoinVertices[U: ClassManifest, VD2: ClassManifest](table: RDD[(Vid, U)]) (mapFunc: (Vid, VD, Option[U]) => VD2) : Graph[VD2, ED] - /** - * 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 should - * contain at most one entry for each vertex. If no entry is provided the - * map function is skipped and the old value is used. - * - * @tparam U the type of entry in the table of updates - * @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 only for vertices with a 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") - * .mapVertices(v => 0) - * val outDeg: RDD[(Int, Int)] = rawGraph.outDegrees() - * val graph = rawGraph.leftJoinVertices[Int,Int](outDeg, - * (v, deg) => deg ) - * }}} - * - * @todo Should this function be curried to enable type inference? For - * example - * {{{ - * graph.joinVertices(tbl)( (v, row) => row ) - * }}} - */ - def joinVertices[U: ClassManifest](table: RDD[(Vid, U)])(mapFunc: (Vid, VD, U) => VD) - : Graph[VD, ED] = { - ClosureCleaner.clean(mapFunc) - def uf(id: Vid, data: VD, o: Option[U]): VD = o match { - case Some(u) => mapFunc(id, data, u) - case None => data - } - outerJoinVertices(table)(uf) - } // 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. @@ -385,11 +268,24 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { } +/** + * The Graph Singleton contains basic routines to create graphs + */ object Graph { import org.apache.spark.graph.impl._ import org.apache.spark.SparkContext._ + /** + * Construct a graph from a list of Edges. + * + * @param rawEdges a collection of edges in (src,dst) form. + * @param uniqueEdges if multiple identical edges are found they are combined + * and the edge attribute is set to the sum. Otherwise duplicate edges are + * treated as separate. + * + * + */ def apply(rawEdges: RDD[(Vid, Vid)], uniqueEdges: Boolean = true): Graph[Int, Int] = { // Reduce to unique edges. val edges: RDD[Edge[Int]] = @@ -399,6 +295,7 @@ object Graph { rawEdges.map { case (s, t) => Edge(s, t, 1) } } // Determine unique vertices + /** @todo Should this reduceByKey operation be indexed? */ val vertices: RDD[(Vid, Int)] = edges.flatMap{ case Edge(s, t, cnt) => Array((s, 1), (t, 1)) }.reduceByKey(_ + _) diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphKryoRegistrator.scala b/graph/src/main/scala/org/apache/spark/graph/GraphKryoRegistrator.scala index 2d72789878c1a..29ea38ec67fdf 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphKryoRegistrator.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphKryoRegistrator.scala @@ -9,7 +9,6 @@ import org.apache.spark.graph.impl._ class GraphKryoRegistrator extends KryoRegistrator { def registerClasses(kryo: Kryo) { - kryo.register(classOf[Vertex[Object]]) kryo.register(classOf[Edge[Object]]) kryo.register(classOf[MutableTuple2[Object, Object]]) kryo.register(classOf[MessageToPartition[Object]]) diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala b/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala index ccb1bd8e5dab4..8ba708ba3290b 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala @@ -51,12 +51,12 @@ object GraphLab { // The gather function wrapper strips the active attribute and // only invokes the gather function on active vertices def gather(vid: Vid, e: EdgeTriplet[(Boolean, VD), ED]): Option[A] = { - if (e.vertex(vid).data._1) { - val edge = new EdgeTriplet[VD,ED] - edge.src = Vertex(e.src.id, e.src.data._2) - edge.dst = Vertex(e.dst.id, e.dst.data._2) - edge.data = e.data - Some(gatherFunc(vid, edge)) + if (e.vertexAttr(vid)._1) { + val edgeTriplet = new EdgeTriplet[VD,ED] + edgeTriplet.set(e) + edgeTriplet.srcAttr = e.srcAttr._2 + edgeTriplet.dstAttr = e.dstAttr._2 + Some(gatherFunc(vid, edgeTriplet)) } else { None } @@ -73,16 +73,13 @@ object GraphLab { // The scatter function wrapper strips the vertex of the active attribute // and only invokes the scatter function on active vertices def scatter(rawVid: Vid, e: EdgeTriplet[(Boolean, VD), ED]): Option[Boolean] = { - val vid = e.otherVertex(rawVid).id - if (e.vertex(vid).data._1) { - val edge = new EdgeTriplet[VD,ED] - edge.src = Vertex(e.src.id, e.src.data._2) - edge.dst = Vertex(e.dst.id, e.dst.data._2) - edge.data = e.data -// val src = Vertex(e.src.id, e.src.data._2) -// val dst = Vertex(e.dst.id, e.dst.data._2) -// val edge = new EdgeTriplet[VD,ED](src, dst, e.data) - Some(scatterFunc(vid, edge)) + val vid = e.otherVertexId(rawVid) + if (e.vertexAttr(vid)._1) { + val edgeTriplet = new EdgeTriplet[VD,ED] + edgeTriplet.set(e) + edgeTriplet.srcAttr = e.srcAttr._2 + edgeTriplet.dstAttr = e.dstAttr._2 + Some(scatterFunc(vid, edgeTriplet)) } else { None } diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala b/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala index 53e7d0a7bb5fb..052f9acdeb6b3 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala @@ -46,7 +46,7 @@ object GraphLoader { } def fromEdges[ED: ClassManifest](edges: RDD[Edge[ED]]): GraphImpl[Int, ED] = { - val vertices = edges.flatMap { edge => List((edge.src, 1), (edge.dst, 1)) } + val vertices = edges.flatMap { edge => List((edge.srcId, 1), (edge.dstId, 1)) } .reduceByKey(_ + _) .map{ case (vid, degree) => (vid, degree) } GraphImpl(vertices, edges) diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala b/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala index 23c783ba3a80f..92198a4995c8e 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala @@ -2,12 +2,16 @@ package org.apache.spark.graph import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext._ +import org.apache.spark.util.ClosureCleaner -class GraphOps[VD: ClassManifest, ED: ClassManifest](g: Graph[VD, ED]) { - lazy val numEdges: Long = g.edges.count() +class GraphOps[VD: ClassManifest, ED: ClassManifest](graph: Graph[VD, ED]) { - lazy val numVertices: Long = g.vertices.count() + + + lazy val numEdges: Long = graph.edges.count() + + lazy val numVertices: Long = graph.vertices.count() lazy val inDegrees: RDD[(Vid, Int)] = degreesRDD(EdgeDirection.In) @@ -15,19 +19,148 @@ class GraphOps[VD: ClassManifest, ED: ClassManifest](g: Graph[VD, ED]) { lazy val degrees: RDD[(Vid, Int)] = degreesRDD(EdgeDirection.Both) + + /** + * This function is used to compute a statistic for the neighborhood of each + * vertex and returns a value for all vertices (including those without + * neighbors). + * + * @note Because the a default value is provided all vertices will have a + * corresponding entry in the returned RDD. + * + * @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 default the default value to use for each vertex if it has no + * neighbors or the map function repeatedly evaluates to none + * @param direction the direction of edges to consider (e.g., In, Out, Both). + * @tparam VD2 The returned type of the aggregation operation. + * + * @return A Spark.RDD containing tuples of vertex identifiers and + * their resulting value. There will be exactly one entry for ever vertex in + * the original graph. + * + * @example We can use this function to compute the average follower age + * for each user + * {{{ + * val graph: Graph[Int,Int] = loadGraph() + * val averageFollowerAge: RDD[(Int, Int)] = + * graph.aggregateNeighbors[(Int,Double)]( + * (vid, edge) => (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} + * }}} + * + * @todo Should this return a graph with the new vertex values? + * + */ + def aggregateNeighbors[A: ClassManifest]( + mapFunc: (Vid, EdgeTriplet[VD, ED]) => Option[A], + reduceFunc: (A, A) => A, + dir: EdgeDirection) + : RDD[(Vid, A)] = { + + ClosureCleaner.clean(mapFunc) + ClosureCleaner.clean(reduceFunc) + + // 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) => Array.empty[(Vid, A)] + case (Some(srcA),None) => Array((et.srcId, srcA)) + case (None, Some(dstA)) => Array((et.dstId, dstA)) + case (Some(srcA), Some(dstA)) => + Array((et.srcId, srcA), (et.dstId, dstA)) + } + } + + ClosureCleaner.clean(mf) + graph.mapReduceTriplets(mf, reduceFunc) + } // end of aggregateNeighbors + + + + + + + def collectNeighborIds(edgeDirection: EdgeDirection) : RDD[(Vid, Array[Vid])] = { - val nbrs = g.aggregateNeighbors[Array[Vid]]( - (vid, edge) => Some(Array(edge.otherVertex(vid).id)), + val nbrs = graph.aggregateNeighbors[Array[Vid]]( + (vid, edge) => Some(Array(edge.otherVertexId(vid))), (a, b) => a ++ b, edgeDirection) - g.vertices.leftOuterJoin(nbrs).mapValues{ + graph.vertices.leftOuterJoin(nbrs).mapValues{ case (_, Some(nbrs)) => nbrs case (_, None) => Array.empty[Vid] } } + private def degreesRDD(edgeDirection: EdgeDirection): RDD[(Vid, Int)] = { - g.aggregateNeighbors((vid, edge) => Some(1), _+_, edgeDirection) + graph.aggregateNeighbors((vid, edge) => Some(1), _+_, edgeDirection) } + + + /** + * 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 should + * contain at most one entry for each vertex. If no entry is provided the + * map function is skipped and the old value is used. + * + * @tparam U the type of entry in the table of updates + * @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 only for vertices with a 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") + * .mapVertices(v => 0) + * val outDeg: RDD[(Int, Int)] = rawGraph.outDegrees() + * val graph = rawGraph.leftJoinVertices[Int,Int](outDeg, + * (v, deg) => deg ) + * }}} + * + * @todo Should this function be curried to enable type inference? For + * example + * {{{ + * graph.joinVertices(tbl)( (v, row) => row ) + * }}} + */ + def joinVertices[U: ClassManifest](table: RDD[(Vid, U)])(mapFunc: (Vid, VD, U) => VD) + : Graph[VD, ED] = { + ClosureCleaner.clean(mapFunc) + val uf = (id: Vid, data: VD, o: Option[U]) => { + o match { + case Some(u) => mapFunc(id, data, u) + case None => data + } + } + ClosureCleaner.clean(uf) + graph.outerJoinVertices(table)(uf) + } + } diff --git a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala index 93545b6ea58ce..065d196ff651d 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala @@ -17,8 +17,7 @@ object Pregel { //var g = graph.cache() var i = 0 - def mapF(vid: Vid, edge: EdgeTriplet[VD,ED]) = sendMsg(edge.otherVertex(vid).id, edge) - + def mapF(vid: Vid, edge: EdgeTriplet[VD,ED]) = sendMsg(edge.otherVertexId(vid), edge) // Receive the first set of messages g.mapVertices( (vid, vdata) => vprog(vid, vdata, initialMsg)) diff --git a/graph/src/main/scala/org/apache/spark/graph/Vertex.scala b/graph/src/main/scala/org/apache/spark/graph/Vertex.scala deleted file mode 100644 index c8671b7f13024..0000000000000 --- a/graph/src/main/scala/org/apache/spark/graph/Vertex.scala +++ /dev/null @@ -1,15 +0,0 @@ -package org.apache.spark.graph - -/** - * A graph vertex consists of a vertex id and attribute. - * - * @tparam VD the type of the vertex attribute. - */ -case class Vertex[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) VD] ( - var id: Vid = 0, - var data: VD = nullValue[VD]) { - - def this(tuple: (Vid, VD)) = this(tuple._1, tuple._2) - - def tuple = (id, data) -} diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala b/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala index f0d9080d97597..3d6724ba0f266 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala @@ -26,9 +26,9 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) val newData = new Array[ED2](data.size) val edge = new Edge[ED]() for(i <- 0 until data.size){ - edge.src = srcIds(i) - edge.dst = dstIds(i) - edge.data = data(i) + edge.srcId = srcIds(i) + edge.dstId = dstIds(i) + edge.attr = data(i) newData(i) = f(edge) } new EdgePartition(srcIds, dstIds, newData) @@ -37,9 +37,9 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) def foreach(f: Edge[ED] => Unit) { val edge = new Edge[ED] for(i <- 0 until data.size){ - edge.src = srcIds(i) - edge.dst = dstIds(i) - edge.data = data(i) + edge.srcId = srcIds(i) + edge.dstId = dstIds(i) + edge.attr = data(i) f(edge) } } @@ -54,9 +54,9 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) override def hasNext: Boolean = pos < EdgePartition.this.size override def next(): Edge[ED] = { - edge.src = srcIds(pos) - edge.dst = dstIds(pos) - edge.data = data(pos) + edge.srcId = srcIds(pos) + edge.dstId = dstIds(pos) + edge.attr = data(pos) pos += 1 edge } diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index 871694b06d9f4..8f4a4452b8527 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -39,32 +39,24 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( + override def statistics: Map[String, Any] = { + val numVertices = this.numVertices + val numEdges = this.numEdges + val replicationRatio = + vid2pid.groupByKey().map(kv => kv._2.size).sum / vTable.count + val loadArray = + eTable.map{ case (pid, epart) => epart.data.size }.collect.map(x => x / numEdges) + val minLoad = loadArray.min + val maxLoad = loadArray.max + Map( + "Num Vertices" -> numVertices, "Num Edges" -> numEdges, + "Replication" -> replicationRatio, "Load Array" -> loadArray, + "Min Load" -> minLoad, "Max Load" -> maxLoad) + } - // def this(vertices: RDD[Vertex[VD]], edges: RDD[Edge[ED]]) = { - // this(vertices.partitions.size, edges.partitions.size, vertices, edges, null, null) - // } - - // def withPartitioner(numVertexPartitions: Int, numEdgePartitions: Int): Graph[VD, ED] = { - // if (_cached) { - // new GraphImpl(numVertexPartitions, numEdgePartitions, null, null, _rawVTable, _rawETable) - // .cache() - // } else { - // new GraphImpl(numVertexPartitions, numEdgePartitions, _rawVertices, _rawEdges, null, null) - // } - // } - - // def withVertexPartitioner(numVertexPartitions: Int) = { - // withPartitioner(numVertexPartitions, numEdgePartitions) - // } - - // def withEdgePartitioner(numEdgePartitions: Int) = { - // withPartitioner(numVertexPartitions, numEdgePartitions) - // } - - override def cache(): Graph[VD, ED] = { eTable.cache() @@ -76,14 +68,6 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( } - override def replication(): Double = { - val rep = vid2pid.groupByKey().map(kv => kv._2.size).sum - rep / vTable.count - } - - override def balance(): Array[Int] = { - eTable.map{ case (pid, epart) => epart.data.size }.collect - } override def reverse: Graph[VD, ED] = { val etable = eTable.mapValues( _.reverse ).asInstanceOf[IndexedRDD[Pid, EdgePartition[ED]]] @@ -109,46 +93,42 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( // vertices for each edge. new Iterator[EdgeTriplet[VD, ED]] { private var pos = 0 - private val e = new EdgeTriplet[VD, ED] - e.src = new Vertex[VD] - e.dst = new Vertex[VD] - + private val et = new EdgeTriplet[VD, ED] + override def hasNext: Boolean = pos < edgePartition.size override def next() = { - e.src.id = edgePartition.srcIds(pos) + et.srcId = edgePartition.srcIds(pos) // assert(vmap.containsKey(e.src.id)) - e.src.data = vmap.get(e.src.id) - e.dst.id = edgePartition.dstIds(pos) + et.srcAttr = vmap.get(et.srcId) + et.dstId = edgePartition.dstIds(pos) // assert(vmap.containsKey(e.dst.id)) - e.dst.data = vmap.get(e.dst.id) + et.dstAttr = vmap.get(et.dstId) //println("Iter called: " + pos) - e.data = edgePartition.data(pos) + et.attr = edgePartition.data(pos) pos += 1 - e + et } override def toList: List[EdgeTriplet[VD, ED]] = { val lb = new mutable.ListBuffer[EdgeTriplet[VD,ED]] for (i <- (0 until edgePartition.size)) { val currentEdge = new EdgeTriplet[VD, ED] - currentEdge.src = new Vertex[VD] - currentEdge.dst = new Vertex[VD] - currentEdge.src.id = edgePartition.srcIds(i) + currentEdge.srcId = edgePartition.srcIds(i) // assert(vmap.containsKey(e.src.id)) - currentEdge.src.data = vmap.get(currentEdge.src.id) + currentEdge.srcAttr = vmap.get(currentEdge.srcId) - currentEdge.dst.id = edgePartition.dstIds(i) + currentEdge.dstId = edgePartition.dstIds(i) // assert(vmap.containsKey(e.dst.id)) - currentEdge.dst.data = vmap.get(currentEdge.dst.id) + currentEdge.dstAttr = vmap.get(currentEdge.dstId) - currentEdge.data = edgePartition.data(i) + currentEdge.attr = edgePartition.data(i) lb += currentEdge } lb.toList } } // end of iterator } // end of map partition - } + } // end of triplets override def mapVertices[VD2: ClassManifest](f: (Vid, VD) => VD2): Graph[VD2, ED] = { val newVTable = vTable.mapValuesWithKeys((vid, data) => f(vid, data)) @@ -167,16 +147,11 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( Graph[VD, ED2] = { val newETable = eTable.join(vTableReplicated).mapValues{ case (edgePartition, vmap) => - val et = new EdgeTriplet[VD, ED] - et.src = new Vertex[VD] - et.dst = new Vertex[VD] - + val et = new EdgeTriplet[VD, ED] edgePartition.map{e => - et.data = e.data - et.src.id = e.src - et.src.data = vmap(e.src) - et.dst.id = e.dst - et.dst.data = vmap(e.dst) + et.set(e) + et.srcAttr = vmap(e.srcId) + et.dstAttr = vmap(e.dstId) f(et) } }.asInstanceOf[IndexedRDD[Pid, EdgePartition[ED2]]] @@ -215,9 +190,9 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( // Restrict the set of edges to those that satisfy the vertex and the edge predicate. val newETable = createETable( triplets.filter( - t => vpred( t.src.id, t.src.data ) && vpred( t.dst.id, t.dst.data ) && epred(t) + t => vpred( t.srcId, t.srcAttr ) && vpred( t.dstId, t.dstAttr ) && epred(t) ) - .map( t => Edge(t.src.id, t.dst.id, t.data) ), + .map( t => Edge(t.srcId, t.dstId, t.attr) ), eTable.index.partitioner.numPartitions ) @@ -264,7 +239,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( // ET groups will be complete. .groupBy { t: EdgeTriplet[VD, ED] => //println("(" + t.src.id + ", " + t.dst.id + ", " + t.data + ")") - (t.src.id, t.dst.id) } + (t.srcId, t.dstId) } //.groupBy { e => (e.src, e.dst) } // Apply the user supplied supplied edge group function to // each group of edges @@ -305,8 +280,8 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( val newEdges: RDD[Edge[ED2]] = edges.mapPartitions { partIter => partIter.toList .groupBy { e: Edge[ED] => - println(e.src + " " + e.dst) - (e.src, e.dst) } + println(e.srcId + " " + e.dstId) + (e.srcId, e.dstId) } .mapValues { ts => f(ts.toIterator) } .toList .toIterator @@ -333,25 +308,18 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( ClosureCleaner.clean(mapFunc) ClosureCleaner.clean(reduceFunc) - val mapfun = mapFunc - val redfun = reduceFunc - val newVTable: RDD[(Vid, A)] = vTableReplicated.join(eTable).flatMap{ case (pid, (vmap, edgePartition)) => val aggMap = new VertexHashMap[A] val et = new EdgeTriplet[VD, ED] - et.src = new Vertex[VD] - et.dst = new Vertex[VD] edgePartition.foreach{e => - et.data = e.data - et.src.id = e.src - et.src.data = vmap(e.src) - et.dst.id = e.dst - et.dst.data = vmap(e.dst) - mapfun(et).foreach{case (vid, a) => + et.set(e) + et.srcAttr = vmap(e.srcId) + et.dstAttr = vmap(e.dstId) + mapFunc(et).foreach{case (vid, a) => if(aggMap.containsKey(vid)) { - aggMap.put(vid, redfun(aggMap.get(vid), a)) + aggMap.put(vid, reduceFunc(aggMap.get(vid), a)) } else { aggMap.put(vid, a) } } } @@ -360,47 +328,11 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( entry => (entry.getLongKey(), entry.getValue()) } } - .indexed(vTable.index).reduceByKey(redfun) + .indexed(vTable.index).reduceByKey(reduceFunc) newVTable } - def aggregateNeighbors[A: ClassManifest]( - mapFunc: (Vid, EdgeTriplet[VD, ED]) => Option[A], - reduceFunc: (A, A) => A, - dir: EdgeDirection) - : RDD[(Vid, A)] = { - - ClosureCleaner.clean(mapFunc) - ClosureCleaner.clean(reduceFunc) - - // Define a new map function over edge triplets - def mf(et: EdgeTriplet[VD,ED]): Array[(Vid, A)] = { - // Compute the message to the dst vertex - val dstA = - if (dir == EdgeDirection.In || dir == EdgeDirection.Both) { - mapFunc(et.dst.id, et) - } else { Option.empty[A] } - // Compute the message to the source vertex - val srcA = - if (dir == EdgeDirection.Out || dir == EdgeDirection.Both) { - mapFunc(et.src.id, et) - } else { Option.empty[A] } - // construct the return array - (srcA, dstA) match { - case (None, None) => Array.empty[(Vid, A)] - case (Some(src),None) => Array((et.src.id, src)) - case (None, Some(dst)) => Array((et.dst.id, dst)) - case (Some(src), Some(dst)) => - Array((et.src.id, src), (et.dst.id, dst)) - } - } - - mapReduceTriplets(mf, reduceFunc) - } - - - override def outerJoinVertices[U: ClassManifest, VD2: ClassManifest] (updates: RDD[(Vid, U)])(updateF: (Vid, VD, Option[U]) => VD2) @@ -415,49 +347,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( } - ////////////////////////////////////////////////////////////////////////////////////////////////// - // Internals hidden from callers - ////////////////////////////////////////////////////////////////////////////////////////////////// - - - - - - // /** Create a new graph but keep the current partitioning scheme. */ - // protected def newGraph[VD2: ClassManifest, ED2: ClassManifest]( - // vertices: RDD[Vertex[VD2]], edges: RDD[Edge[ED2]]): Graph[VD2, ED2] = { - // (new GraphImpl[VD2, ED2](vertices, edges)).withPartitioner(numVertexPartitions, numEdgePartitions) - // } - - // protected lazy val eTable: RDD[(Pid, EdgePartition[ED])] = { - // if (_rawETable == null) { - // createETable(_rawEdges, numEdgePartitions) - // } else { - // _rawETable - // } - // } - - // protected lazy val vTable: RDD[(Vid, (VD, Array[Pid]))] = { - // if (_rawVTable == null) { - // createVTable(_rawVertices, eTable, numVertexPartitions) - // } else { - // _rawVTable - // } - // } - - // protected lazy val vTableReplicated: RDD[(Vid, VD)] = { - // // Join vid2pid and vTable, generate a shuffle dependency on the joined result, and get - // // the shuffle id so we can use it on the slave. - // vTable - // .flatMap { case (vid, (vdata, pids)) => - // pids.iterator.map { pid => MessageToPartition(pid, (vid, vdata)) } - // } - // .partitionBy(edgePartitioner) - // .mapPartitions({ part => - // part.map { message => (message.data._1, message.data._2) } - // }, preservesPartitioning = true) - // } -} +} // end of class GraphImpl @@ -513,13 +403,13 @@ def apply[VD: ClassManifest, ED: ClassManifest]( edges .map { e => // Random partitioning based on the source vertex id. - val part: Pid = edgePartitionFunction1D(e.src, e.dst, numPartitions) - // val part: Pid = edgePartitionFunction2D(e.src, e.dst, numPartitions, ceilSqrt) - // val part: Pid = randomVertexCut(e.src, e.dst, numPartitions) - //val part: Pid = canonicalEdgePartitionFunction2D(e.src, e.dst, numPartitions, ceilSqrt) + // val part: Pid = edgePartitionFunction1D(e.srcId, e.dstId, numPartitions) + // val part: Pid = edgePartitionFunction2D(e.srcId, e.dstId, numPartitions, ceilSqrt) + val part: Pid = randomVertexCut(e.srcId, e.dstId, numPartitions) + //val part: Pid = canonicalEdgePartitionFunction2D(e.srcId, e.dstId, numPartitions, ceilSqrt) // Should we be using 3-tuple or an optimized class - MessageToPartition(part, (e.src, e.dst, e.data)) + MessageToPartition(part, (e.srcId, e.dstId, e.attr)) } .partitionBy(new HashPartitioner(numPartitions)) .mapPartitionsWithIndex({ (pid, iter) => @@ -539,7 +429,7 @@ def apply[VD: ClassManifest, ED: ClassManifest]( eTable.mapPartitions { iter => val (pid, edgePartition) = iter.next() val vSet = new VertexSet - edgePartition.foreach(e => {vSet.add(e.src); vSet.add(e.dst)}) + edgePartition.foreach(e => {vSet.add(e.srcId); vSet.add(e.dstId)}) vSet.iterator.map { vid => (vid.toLong, pid) } }.indexed(vTableIndex) } @@ -651,44 +541,5 @@ def apply[VD: ClassManifest, ED: ClassManifest]( (col * ceilSqrtNumParts + row) % numParts } - - - - // protected def createVTable[VD: ClassManifest, ED: ClassManifest]( - // eTable: IndexedRDD[Pid, EdgePartition[ED]], - // vid2pid: Index - // vertices: RDD[Vertex[VD]], - - // default: VD) : IndexedRDD[Vid, VD] = { - - // // Compute all the vertices in the edge table. - // val vid2pid = createVid2Pid(eTable) - - // // Compute all the - // vertices.map(v => (v.id, v.data)).cogroup(vids) - - // // A key-value RDD. The key is a vertex id, and the value is a list of - // // partitions that contains edges referencing the vertex. - // val vid2pid : RDD[(Vid, Seq[Pid])] = eTable.mapPartitions { iter => - // val (pid, edgePartition) = iter.next() - // val vSet = new VertexSet - // var i = 0 - // while (i < edgePartition.srcIds.size) { - // vSet.add(edgePartition.srcIds.getLong(i)) - // vSet.add(edgePartition.dstIds.getLong(i)) - // i += 1 - // } - // vSet.iterator.map { vid => (vid.toLong, pid) } - // }.groupByKey(partitioner) - - // vertices - // .map { v => (v.id, v.data) } - // .partitionBy(partitioner) - // .leftOuterJoin(vid2pid) - // .mapValues { - // case (vdata, None) => (vdata, Array.empty[Pid]) - // case (vdata, Some(pids)) => (vdata, pids.toArray) - // } - // } -} +} // end of object GraphImpl diff --git a/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala b/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala index aba3a7d9421be..b2be9b89f826a 100644 --- a/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala +++ b/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala @@ -158,7 +158,7 @@ object GraphGenerators { def outDegreeFromEdges[ED: ClassManifest](edges: RDD[Edge[ED]]): GraphImpl[Int, ED] = { - val vertices = edges.flatMap { edge => List((edge.src, 1)) } + val vertices = edges.flatMap { edge => List((edge.srcId, 1)) } .reduceByKey(_ + _) .map{ case (vid, degree) => (vid, degree) } GraphImpl(vertices, edges) From b64337ec40fd82d6082052d49a534dc5542663b6 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Tue, 15 Oct 2013 13:02:17 -0700 Subject: [PATCH 112/531] Trying to resolve issues with closure capture. --- .../scala/org/apache/spark/graph/Graph.scala | 21 +-- .../apache/spark/graph/impl/GraphImpl.scala | 154 +++++++++--------- .../spark/graph/util/GraphGenerators.scala | 1 - 3 files changed, 92 insertions(+), 84 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/Graph.scala b/graph/src/main/scala/org/apache/spark/graph/Graph.scala index d9cb2c037deae..50a44e51e5d2d 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Graph.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Graph.scala @@ -19,13 +19,6 @@ import org.apache.spark.util.ClosureCleaner */ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { - - /** - * Compute statistics describing the graph representation. - */ - def statistics: Map[String, Any] - - /** * Get the vertices and their data. * @@ -35,7 +28,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * @see Vertex for the vertex type. * */ - def vertices: RDD[(Vid,VD)] + val vertices: RDD[(Vid,VD)] /** * Get the Edges and their data as an RDD. The entries in the RDD contain @@ -50,7 +43,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * @todo Should edges return 3 tuples instead of Edge objects? In this case * we could rename EdgeTriplet to Edge? */ - def edges: RDD[Edge[ED]] + val edges: RDD[Edge[ED]] /** * Get the edges with the vertex data associated with the adjacent pair of @@ -70,7 +63,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * @see edges() If only the edge data and adjacent vertex ids are required. * */ - def triplets: RDD[EdgeTriplet[VD, ED]] + val triplets: RDD[EdgeTriplet[VD, ED]] /** * Return a graph that is cached when first created. This is used to pin a @@ -81,6 +74,14 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { */ def cache(): Graph[VD, ED] + + /** + * Compute statistics describing the graph representation. + */ + def statistics: Map[String, Any] + + + /** * Construct a new graph where each vertex value has been transformed by the * map function. diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index 8f4a4452b8527..0cbe9bbb95e13 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -19,6 +19,49 @@ import org.apache.spark.graph._ import org.apache.spark.graph.impl.GraphImpl._ import org.apache.spark.graph.impl.MessageToPartitionRDDFunctions._ +/** + * The Iterator type returned when constructing edge triplets + */ +class EdgeTripletIterator[VD: ClassManifest, ED: ClassManifest]( + val vmap: VertexHashMap[VD], + val edgePartition: EdgePartition[ED]) extends Iterator[EdgeTriplet[VD, ED]] { + + private var pos = 0 + private val et = new EdgeTriplet[VD, ED] + + override def hasNext: Boolean = pos < edgePartition.size + override def next() = { + et.srcId = edgePartition.srcIds(pos) + // assert(vmap.containsKey(e.src.id)) + et.srcAttr = vmap.get(et.srcId) + et.dstId = edgePartition.dstIds(pos) + // assert(vmap.containsKey(e.dst.id)) + et.dstAttr = vmap.get(et.dstId) + //println("Iter called: " + pos) + et.attr = edgePartition.data(pos) + pos += 1 + et + } + + override def toList: List[EdgeTriplet[VD, ED]] = { + val lb = new mutable.ListBuffer[EdgeTriplet[VD,ED]] + val currentEdge = new EdgeTriplet[VD, ED] + for (i <- (0 until edgePartition.size)) { + currentEdge.srcId = edgePartition.srcIds(i) + // assert(vmap.containsKey(e.src.id)) + currentEdge.srcAttr = vmap.get(currentEdge.srcId) + currentEdge.dstId = edgePartition.dstIds(i) + // assert(vmap.containsKey(e.dst.id)) + currentEdge.dstAttr = vmap.get(currentEdge.dstId) + currentEdge.attr = edgePartition.data(i) + lb += currentEdge + } + lb.toList + } +} // end of Edge Triplet Iterator + + + /** * A Graph RDD that supports computation on graphs. @@ -39,22 +82,26 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( - override def statistics: Map[String, Any] = { - val numVertices = this.numVertices - val numEdges = this.numEdges - val replicationRatio = - vid2pid.groupByKey().map(kv => kv._2.size).sum / vTable.count - val loadArray = - eTable.map{ case (pid, epart) => epart.data.size }.collect.map(x => x / numEdges) - val minLoad = loadArray.min - val maxLoad = loadArray.max - Map( - "Num Vertices" -> numVertices, "Num Edges" -> numEdges, - "Replication" -> replicationRatio, "Load Array" -> loadArray, - "Min Load" -> minLoad, "Max Load" -> maxLoad) + + /** Return a RDD of vertices. */ + override val vertices: RDD[(Vid, VD)] = vTable + + + /** Return a RDD of edges. */ + override val edges: RDD[Edge[ED]] = { + eTable.mapPartitions { iter => iter.next()._2.iterator } } + /** Return a RDD that brings edges with its source and destination vertices together. */ + override val triplets: RDD[EdgeTriplet[VD, ED]] = + vTableReplicated.join(eTable).mapPartitions( iter => { + val (pid, (vmap, edgePartition)) = iter.next() + //assert(iter.hasNext == false) + // Return an iterator that looks up the hash map to find matching + // vertices for each edge. + new EdgeTripletIterator(vmap, edgePartition) + }) // end of map partition @@ -62,73 +109,33 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( eTable.cache() vid2pid.cache() vTable.cache() - // @todo: should we cache the replicated data? + /** @todo should we cache the replicated data? */ vTableReplicated.cache() this } + override def statistics: Map[String, Any] = { + val numVertices = this.numVertices + val numEdges = this.numEdges + val replicationRatio = + vid2pid.groupByKey().map(kv => kv._2.size).sum / vTable.count + val loadArray = + eTable.map{ case (pid, epart) => epart.data.size }.collect.map(x => x / numEdges) + val minLoad = loadArray.min + val maxLoad = loadArray.max + Map( + "Num Vertices" -> numVertices, "Num Edges" -> numEdges, + "Replication" -> replicationRatio, "Load Array" -> loadArray, + "Min Load" -> minLoad, "Max Load" -> maxLoad) + } + override def reverse: Graph[VD, ED] = { val etable = eTable.mapValues( _.reverse ).asInstanceOf[IndexedRDD[Pid, EdgePartition[ED]]] new GraphImpl(vTable, vid2pid, etable) } - /** Return a RDD of vertices. */ - override def vertices: RDD[(Vid, VD)] = vTable - - - /** Return a RDD of edges. */ - override def edges: RDD[Edge[ED]] = { - eTable.mapPartitions { iter => iter.next()._2.iterator } - } - - /** Return a RDD that brings edges with its source and destination vertices together. */ - override def triplets: RDD[EdgeTriplet[VD, ED]] = { - vTableReplicated.join(eTable) - .mapPartitions{ iter => - val (pid, (vmap, edgePartition)) = iter.next() - assert(iter.hasNext == false) - // Return an iterator that looks up the hash map to find matching - // vertices for each edge. - new Iterator[EdgeTriplet[VD, ED]] { - private var pos = 0 - private val et = new EdgeTriplet[VD, ED] - - override def hasNext: Boolean = pos < edgePartition.size - override def next() = { - et.srcId = edgePartition.srcIds(pos) - // assert(vmap.containsKey(e.src.id)) - et.srcAttr = vmap.get(et.srcId) - et.dstId = edgePartition.dstIds(pos) - // assert(vmap.containsKey(e.dst.id)) - et.dstAttr = vmap.get(et.dstId) - //println("Iter called: " + pos) - et.attr = edgePartition.data(pos) - pos += 1 - et - } - - override def toList: List[EdgeTriplet[VD, ED]] = { - val lb = new mutable.ListBuffer[EdgeTriplet[VD,ED]] - for (i <- (0 until edgePartition.size)) { - val currentEdge = new EdgeTriplet[VD, ED] - currentEdge.srcId = edgePartition.srcIds(i) - // assert(vmap.containsKey(e.src.id)) - currentEdge.srcAttr = vmap.get(currentEdge.srcId) - - currentEdge.dstId = edgePartition.dstIds(i) - // assert(vmap.containsKey(e.dst.id)) - currentEdge.dstAttr = vmap.get(currentEdge.dstId) - - currentEdge.attr = edgePartition.data(i) - lb += currentEdge - } - lb.toList - } - } // end of iterator - } // end of map partition - } // end of triplets override def mapVertices[VD2: ClassManifest](f: (Vid, VD) => VD2): Graph[VD2, ED] = { val newVTable = vTable.mapValuesWithKeys((vid, data) => f(vid, data)) @@ -169,9 +176,10 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( override def subgraph(epred: EdgeTriplet[VD,ED] => Boolean = (x => true), vpred: (Vid, VD) => Boolean = ((a,b) => true) ): Graph[VD, ED] = { - /// @todo: The following code behaves deterministically on each - /// vertex predicate but uses additional space. Should we swithc to - /// this version + /** @todo The following code behaves deterministically on each + * vertex predicate but uses additional space. Should we swithc to + * this version + */ // val predGraph = mapVertices(v => (v.data, vpred(v))) // val newETable = predGraph.triplets.filter(t => // if(v.src.data._2 && v.dst.data._2) { @@ -366,7 +374,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( object GraphImpl { -def apply[VD: ClassManifest, ED: ClassManifest]( + def apply[VD: ClassManifest, ED: ClassManifest]( vertices: RDD[(Vid, VD)], edges: RDD[Edge[ED]]): GraphImpl[VD,ED] = { diff --git a/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala b/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala index b2be9b89f826a..d75a678b26949 100644 --- a/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala +++ b/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala @@ -13,7 +13,6 @@ import org.apache.spark.SparkContext import org.apache.spark.SparkContext._ import org.apache.spark.graph._ import org.apache.spark.graph.Graph -import org.apache.spark.graph.Vertex import org.apache.spark.graph.Edge import org.apache.spark.graph.impl.GraphImpl From 345e1e94ccdc35a7d652b91cf0e54fa3aa7d313f Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Tue, 15 Oct 2013 14:01:38 -0700 Subject: [PATCH 113/531] Still trying to resolve issues with capture. --- .../org/apache/spark/graph/impl/GraphImpl.scala | 16 +++++++++++----- 1 file changed, 11 insertions(+), 5 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index 0cbe9bbb95e13..99badfa09feeb 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -72,6 +72,8 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( @transient val eTable: IndexedRDD[Pid, EdgePartition[ED]]) extends Graph[VD, ED] { +// def this() = this(null,null,null) + /** * The vTableReplicated is a version of the vertex data after it is @@ -84,24 +86,28 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( /** Return a RDD of vertices. */ - override val vertices: RDD[(Vid, VD)] = vTable + @transient override val vertices: RDD[(Vid, VD)] = vTable /** Return a RDD of edges. */ - override val edges: RDD[Edge[ED]] = { + @transient override val edges: RDD[Edge[ED]] = { eTable.mapPartitions { iter => iter.next()._2.iterator } } /** Return a RDD that brings edges with its source and destination vertices together. */ - override val triplets: RDD[EdgeTriplet[VD, ED]] = - vTableReplicated.join(eTable).mapPartitions( iter => { + @transient override val triplets: RDD[EdgeTriplet[VD, ED]] = { + val iterFun = (iter: Iterator[(Pid, (VertexHashMap[VD], EdgePartition[ED]))]) => { val (pid, (vmap, edgePartition)) = iter.next() //assert(iter.hasNext == false) // Return an iterator that looks up the hash map to find matching // vertices for each edge. new EdgeTripletIterator(vmap, edgePartition) - }) // end of map partition + } + ClosureCleaner.clean(iterFun) + vTableReplicated.join(eTable).mapPartitions( iterFun ) // end of map partition + } + From 7241cf1632d990023ad100f26ecff4990770feba Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Tue, 15 Oct 2013 14:18:03 -0700 Subject: [PATCH 114/531] Updating unit tests. --- .../org/apache/spark/graph/GraphSuite.scala | 38 +++---------------- 1 file changed, 6 insertions(+), 32 deletions(-) diff --git a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala index 8c85260c1b520..1b03b8094e379 100644 --- a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala @@ -26,46 +26,20 @@ class GraphSuite extends FunSuite with LocalSparkContext { val indegrees = star.aggregateNeighbors( (vid, edge) => Some(1), (a: Int, b: Int) => a + b, - EdgeDirection.In).vertices.map(v => (v.id, v.data._2.getOrElse(0))) - assert(indegrees.collect().toSet === Set((0, 0), (1, 1), (2, 1), (3, 1))) + EdgeDirection.In)// .map((vid, attr) => (vid, attr._2.getOrElse(0))) + assert(indegrees.collect().toSet === Set((1, 1), (2, 1), (3, 1))) // (0, 0), val outdegrees = star.aggregateNeighbors( (vid, edge) => Some(1), (a: Int, b: Int) => a + b, - EdgeDirection.Out).vertices.map(v => (v.id, v.data._2.getOrElse(0))) - assert(outdegrees.collect().toSet === Set((0, 3), (1, 0), (2, 0), (3, 0))) + EdgeDirection.Out) //.map((vid, attr) => (vid, attr._2.getOrElse(0))) + assert(outdegrees.collect().toSet === Set((0, 3))) //, (1, 0), (2, 0), (3, 0))) val noVertexValues = star.aggregateNeighbors[Int]( (vid: Vid, edge: EdgeTriplet[Int, Int]) => None, (a: Int, b: Int) => throw new Exception("reduceFunc called unexpectedly"), - EdgeDirection.In).vertices.map(v => (v.id, v.data._2)) - assert(noVertexValues.collect().toSet === Set((0, None), (1, None), (2, None), (3, None))) - } - } - - test("groupEdges") { - withSpark(new SparkContext("local", "test")) { sc => - val vertices = sc.parallelize(List(Vertex(6, 1),Vertex(7, 1), Vertex(8,1))) - val edges = sc.parallelize(List( - Edge(6, 7, 4), - Edge(6, 7, 9), - Edge(6, 7, 7), - Edge(7, 6, 25), - Edge(7, 6, 300), - Edge(7, 6, 600), - Edge(8, 7, 11), - Edge(7, 8, 89))) - - val original = Graph(vertices, edges) - val grouped = original.groupEdgeTriplets { iter => - iter.map(_.data).sum } - - assert(grouped.edges.count() === 4) - assert(grouped.edges.collect().toSet === Set( - Edge(6, 7, 20), - Edge(7, 6, 925), - Edge(8, 7, 11), - Edge(7, 8, 89))) + EdgeDirection.In)//.map((vid, attr) => (vid, attr)) + assert(noVertexValues.collect().toSet === Set.empty[(Vid, Int)] ) // ((0, None), (1, None), (2, None), (3, None))) } } From 194bb03d1637f731535f964e1d1661d218380162 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Tue, 15 Oct 2013 15:10:41 -0700 Subject: [PATCH 115/531] Resolved closure capture issues by addressing capture through implicit variables. --- .../org/apache/spark/graph/EdgeTriplet.scala | 2 +- .../apache/spark/graph/impl/GraphImpl.scala | 41 ++++++++++++++----- 2 files changed, 31 insertions(+), 12 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/EdgeTriplet.scala b/graph/src/main/scala/org/apache/spark/graph/EdgeTriplet.scala index 3833b38abe672..7dfb5caa4c529 100644 --- a/graph/src/main/scala/org/apache/spark/graph/EdgeTriplet.scala +++ b/graph/src/main/scala/org/apache/spark/graph/EdgeTriplet.scala @@ -10,7 +10,7 @@ package org.apache.spark.graph * specializing I got a warning about inherenting from a type that is not * a trait. */ -class EdgeTriplet[VD: ClassManifest, ED: ClassManifest] extends Edge[ED] { +class EdgeTriplet[VD, ED] extends Edge[ED] { // class EdgeTriplet[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) VD: ClassManifest, // @specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED: ClassManifest] extends Edge[ED] { diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index 99badfa09feeb..d373d86a5a1fd 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -60,7 +60,22 @@ class EdgeTripletIterator[VD: ClassManifest, ED: ClassManifest]( } } // end of Edge Triplet Iterator +object EdgeTripletBuilder { + def makeTriplets[VD: ClassManifest, ED: ClassManifest]( + vTableReplicated: IndexedRDD[Pid, VertexHashMap[VD]], + eTable: IndexedRDD[Pid, EdgePartition[ED]]): RDD[EdgeTriplet[VD, ED]] = { + val iterFun = (iter: Iterator[(Pid, (VertexHashMap[VD], EdgePartition[ED]))]) => { + val (pid, (vmap, edgePartition)) = iter.next() + //assert(iter.hasNext == false) + // Return an iterator that looks up the hash map to find matching + // vertices for each edge. + new EdgeTripletIterator(vmap, edgePartition) + } + ClosureCleaner.clean(iterFun) + vTableReplicated.join(eTable).mapPartitions( iterFun ) // end of map partition + } +} /** @@ -96,17 +111,21 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( /** Return a RDD that brings edges with its source and destination vertices together. */ - @transient override val triplets: RDD[EdgeTriplet[VD, ED]] = { - val iterFun = (iter: Iterator[(Pid, (VertexHashMap[VD], EdgePartition[ED]))]) => { - val (pid, (vmap, edgePartition)) = iter.next() - //assert(iter.hasNext == false) - // Return an iterator that looks up the hash map to find matching - // vertices for each edge. - new EdgeTripletIterator(vmap, edgePartition) - } - ClosureCleaner.clean(iterFun) - vTableReplicated.join(eTable).mapPartitions( iterFun ) // end of map partition - } + @transient override val triplets: RDD[EdgeTriplet[VD, ED]] = + EdgeTripletBuilder.makeTriplets(vTableReplicated, eTable) + + + // { + // val iterFun = (iter: Iterator[(Pid, (VertexHashMap[VD], EdgePartition[ED]))]) => { + // val (pid, (vmap, edgePartition)) = iter.next() + // //assert(iter.hasNext == false) + // // Return an iterator that looks up the hash map to find matching + // // vertices for each edge. + // new EdgeTripletIterator(vmap, edgePartition) + // } + // ClosureCleaner.clean(iterFun) + // vTableReplicated.join(eTable).mapPartitions( iterFun ) // end of map partition + // } From 9058f261fe6ba7c7889808e124ec2bd8f37f99f5 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Tue, 15 Oct 2013 17:39:09 -0700 Subject: [PATCH 116/531] Addressing issue where statistics are not computed correctly --- .../src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index d373d86a5a1fd..aa0aaaaef4b90 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -146,7 +146,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( val replicationRatio = vid2pid.groupByKey().map(kv => kv._2.size).sum / vTable.count val loadArray = - eTable.map{ case (pid, epart) => epart.data.size }.collect.map(x => x / numEdges) + eTable.map{ case (pid, epart) => epart.data.size }.collect.map(x => x.toDouble / numEdges) val minLoad = loadArray.min val maxLoad = loadArray.max Map( From 3cb6dffce0ee7be237749b9f9e5db6e023e06054 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Tue, 15 Oct 2013 18:55:06 -0700 Subject: [PATCH 117/531] adding indexed reduce by key --- .../org/apache/spark/rdd/IndexedRDD.scala | 43 +++++++++++++++++++ 1 file changed, 43 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/rdd/IndexedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/IndexedRDD.scala index a881ee3a1da35..750a4567151d8 100644 --- a/core/src/main/scala/org/apache/spark/rdd/IndexedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/IndexedRDD.scala @@ -216,6 +216,49 @@ object IndexedRDD { } } + def reduceByKey[K: ClassManifest, V: ClassManifest]( + rdd: RDD[(K,V)], reduceFun: (V, V) => V, index: RDDIndex[K]): IndexedRDD[K,V] = { + // Get the index Partitioner + val partitioner = index.rdd.partitioner match { + case Some(p) => p + case None => throw new SparkException("An index must have a partitioner.") + } + // Preaggregate and shuffle if necessary + val partitioned = + if (rdd.partitioner != Some(partitioner)) { + // Preaggregation. + val aggregator = new Aggregator[K, V, V](v => v, reduceFun, reduceFun) + val combined = rdd.mapPartitions(aggregator.combineValuesByKey, preservesPartitioning = true) + combined.partitionBy(partitioner) //new ShuffledRDD[K, V, (K, V)](combined, partitioner) + } else { + rdd + } + + // Use the index to build the new values table + val values = index.rdd.zipPartitions(partitioned)( (indexIter, tblIter) => { + // There is only one map + val index = indexIter.next() + assert(!indexIter.hasNext()) + val values = new Array[Array[V]](index.size) + for ((k,v) <- tblIter) { + if (!index.contains(k)) { + throw new SparkException("Error: Trying to bind an external index " + + "to an RDD which contains keys that are not in the index.") + } + val ind = index(k) + if (values(ind) == null) { + values(ind) = Array(v) + } else { + values(ind)(0) = reduceFun(values(ind).head, v) + } + } + List(values.view.map(x => if (x != null) x.toSeq else null ).toSeq).iterator + }) + + new IndexedRDD[K,V](index, values) + + } + /** * Construct and index of the unique values in a given RDD. */ From 80e4ec3278c0e63edf0d2c293cf4c650d85eb976 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Wed, 16 Oct 2013 00:16:44 -0700 Subject: [PATCH 118/531] IndexedRDD now only supports unique keys --- .../org/apache/spark/rdd/IndexedRDD.scala | 160 +++++++------- .../spark/rdd/IndexedRDDFunctions.scala | 199 +++++++++--------- .../apache/spark/rdd/PairRDDFunctions.scala | 7 +- 3 files changed, 182 insertions(+), 184 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/IndexedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/IndexedRDD.scala index 750a4567151d8..956e0fe5f9d8d 100644 --- a/core/src/main/scala/org/apache/spark/rdd/IndexedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/IndexedRDD.scala @@ -24,6 +24,7 @@ import java.util.{HashMap => JHashMap, BitSet => JBitSet, HashSet => JHashSet} import scala.collection.JavaConversions._ import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.BitSet import org.apache.spark._ import org.apache.spark.rdd._ @@ -69,7 +70,7 @@ class RDDIndex[@specialized K: ClassManifest](private[spark] val rdd: RDD[BlockI */ class IndexedRDD[K: ClassManifest, V: ClassManifest]( @transient val index: RDDIndex[K], - @transient val valuesRDD: RDD[ Seq[Seq[V]] ]) + @transient val valuesRDD: RDD[ (Array[V], BitSet) ]) extends RDD[(K, V)](index.rdd.context, List(new OneToOneDependency(index.rdd), new OneToOneDependency(valuesRDD)) ) { @@ -120,19 +121,40 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( } + override def filter(f: Tuple2[K,V] => Boolean): RDD[(K,V)] = { + val cleanF = index.rdd.context.clean(f) + val newValues = index.rdd.zipPartitions(valuesRDD){ + (keysIter, valuesIter) => + val index = keysIter.next() + assert(keysIter.hasNext() == false) + val (oldValues, bs) = valuesIter.next() + assert(valuesIter.hasNext() == false) + // Allocate the array to store the results into + val newValues: Array[V] = oldValues.clone().asInstanceOf[Array[V]] + val newBS = new BitSet(oldValues.size) + // Populate the new Values + for( (k,i) <- index ) { + if ( bs(i) && f( (k, oldValues(i)) ) ) { newBS(i) = true } + else { newValues(i) = null.asInstanceOf[V] } + } + Array((newValues, newBS)).iterator + } + new IndexedRDD[K,V](index, newValues) + } + /** * Provide the RDD[(K,V)] equivalent output. */ override def compute(part: Partition, context: TaskContext): Iterator[(K, V)] = { - tuples.compute(part, context).flatMap { case (indexMap, values) => + tuples.compute(part, context).flatMap { case (indexMap, (values, bs) ) => // Walk the index to construct the key, value pairs - indexMap.iterator + indexMap.iterator // Extract rows with key value pairs and indicators - .map{ case (k, ind) => (k, values(ind)) } + .map{ case (k, ind) => (bs(ind), k, ind) } // Remove tuples that aren't actually present in the array - .filter{ case (_, valar) => valar != null && !valar.isEmpty()} + .filter( _._1 ) // Extract the pair (removing the indicator from the tuple) - .flatMap{ case (k, valar) => valar.map(v => (k,v))} + .map( x => (x._2, values(x._3) ) ) } } @@ -143,81 +165,49 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( object IndexedRDD { - /** - * Construct an IndexedRDD from a regular RDD[(K,V)] using an existing index - * if one is provided. - */ - def apply[K: ClassManifest, V: ClassManifest]( - tbl: RDD[(K,V)], - existingIndex: RDDIndex[K] = null ): IndexedRDD[K, V] = { +def apply[K: ClassManifest, V: ClassManifest](rdd: RDD[(K,V)]): IndexedRDD[K,V] = + apply(rdd, (a:V, b:V) => a ) - if (existingIndex == null) { // If no index was provided - // Shuffle the table (if necessary) - val shuffledTbl = - if (tbl.partitioner.isEmpty) { - new ShuffledRDD[K, V, (K,V)](tbl, Partitioner.defaultPartitioner(tbl)) - } else { tbl } + def apply[K: ClassManifest, V: ClassManifest]( + rdd: RDD[(K,V)], reduceFunc: (V, V) => V): IndexedRDD[K,V] = { + // Preaggregate and shuffle if necessary + // Preaggregation. + val aggregator = new Aggregator[K, V, V](v => v, reduceFunc, reduceFunc) + val partitioner = new HashPartitioner(rdd.partitions.size) + val preAgg = rdd.mapPartitions(aggregator.combineValuesByKey).partitionBy(partitioner) - val groups = shuffledTbl.mapPartitions( iter => { - val indexMap = new BlockIndex[K]() - val values = new ArrayBuffer[Seq[V]]() - for ((k,v) <- iter){ - if(!indexMap.contains(k)) { - val ind = indexMap.size - indexMap.put(k, ind) - values.append(ArrayBuffer.empty[V]) - } + val groups = preAgg.mapPartitions( iter => { + val indexMap = new BlockIndex[K]() + val values = new ArrayBuffer[V]() + val bs = new BitSet + for ((k,v) <- iter) { + if(!indexMap.contains(k)) { + val ind = indexMap.size + indexMap.put(k, ind) + values.append(v) + bs(ind) = true + } else { val ind = indexMap.get(k) - values(ind).asInstanceOf[ArrayBuffer[V]].append(v) + values(ind) = reduceFunc(values(ind), v) } - List((indexMap, values.toSeq)).iterator - }, true).cache - // extract the index and the values - val index = groups.mapPartitions(_.map{ case (kMap,vAr) => kMap }, true) - val values = groups.mapPartitions(_.map{ case (kMap,vAr) => vAr }, true) - new IndexedRDD[K,V](new RDDIndex(index), values) - } else { - val index = existingIndex - val partitioner = index.rdd.partitioner match { - case Some(p) => p - case None => throw new SparkException("An index must have a partitioner.") } + List( (indexMap, (values.toArray, bs)) ).iterator + }, true).cache + // extract the index and the values + val index = groups.mapPartitions(_.map{ case (kMap, vAr) => kMap }, true) + val values = groups.mapPartitions(_.map{ case (kMap,vAr) => vAr }, true) + new IndexedRDD[K,V](new RDDIndex(index), values) + } - // Shuffle the table according to the index (if necessary) - val shuffledTbl = - if (tbl.partitioner == Some(partitioner)) { - tbl - } else { - new ShuffledRDD[K, V, (K,V)](tbl, partitioner) - } - // Use the index to build the new values table - val values = index.rdd.zipPartitions(shuffledTbl)( - (indexIter, tblIter) => { - // There is only one map - val index = indexIter.next() - assert(!indexIter.hasNext()) - val values = new Array[Seq[V]](index.size) - for ((k,v) <- tblIter) { - if (!index.contains(k)) { - throw new SparkException("Error: Trying to bind an external index " + - "to an RDD which contains keys that are not in the index.") - } - val ind = index(k) - if (values(ind) == null) { - values(ind) = ArrayBuffer.empty[V] - } - values(ind).asInstanceOf[ArrayBuffer[V]].append(v) - } - List(values.toSeq).iterator - }) - - new IndexedRDD[K,V](index, values) - } - } - def reduceByKey[K: ClassManifest, V: ClassManifest]( - rdd: RDD[(K,V)], reduceFun: (V, V) => V, index: RDDIndex[K]): IndexedRDD[K,V] = { + def apply[K: ClassManifest, V: ClassManifest]( + rdd: RDD[(K,V)], index: RDDIndex[K]): IndexedRDD[K,V] = + apply(rdd, index, (a:V,b:V) => a) + + def apply[K: ClassManifest, V: ClassManifest]( + rdd: RDD[(K,V)], index: RDDIndex[K], + reduceFunc: (V, V) => V): IndexedRDD[K,V] = { // Get the index Partitioner val partitioner = index.rdd.partitioner match { case Some(p) => p @@ -227,9 +217,8 @@ object IndexedRDD { val partitioned = if (rdd.partitioner != Some(partitioner)) { // Preaggregation. - val aggregator = new Aggregator[K, V, V](v => v, reduceFun, reduceFun) - val combined = rdd.mapPartitions(aggregator.combineValuesByKey, preservesPartitioning = true) - combined.partitionBy(partitioner) //new ShuffledRDD[K, V, (K, V)](combined, partitioner) + val aggregator = new Aggregator[K, V, V](v => v, reduceFunc, reduceFunc) + rdd.mapPartitions(aggregator.combineValuesByKey).partitionBy(partitioner) } else { rdd } @@ -239,26 +228,27 @@ object IndexedRDD { // There is only one map val index = indexIter.next() assert(!indexIter.hasNext()) - val values = new Array[Array[V]](index.size) + val values = new Array[V](index.size) + val bs = new BitSet(index.size) for ((k,v) <- tblIter) { if (!index.contains(k)) { throw new SparkException("Error: Trying to bind an external index " + "to an RDD which contains keys that are not in the index.") } val ind = index(k) - if (values(ind) == null) { - values(ind) = Array(v) + if (bs(ind)) { + values(ind) = reduceFunc(values(ind), v) } else { - values(ind)(0) = reduceFun(values(ind).head, v) + values(ind) = v + bs(ind) = true } } - List(values.view.map(x => if (x != null) x.toSeq else null ).toSeq).iterator + List((values, bs)).iterator }) - new IndexedRDD[K,V](index, values) + } // end of apply - } - + /** * Construct and index of the unique values in a given RDD. */ @@ -277,9 +267,7 @@ object IndexedRDD { } case Some(partitioner) => tbl.partitionBy(partitioner) -// new ShuffledRDD[K, Boolean](tbl, partitioner) } - val index = shuffledTbl.mapPartitions( iter => { val indexMap = new BlockIndex[K]() diff --git a/core/src/main/scala/org/apache/spark/rdd/IndexedRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/IndexedRDDFunctions.scala index 358ab57b0c9e5..59e7d7da2b6c7 100644 --- a/core/src/main/scala/org/apache/spark/rdd/IndexedRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/IndexedRDDFunctions.scala @@ -21,6 +21,8 @@ import java.util.{HashMap => JHashMap, BitSet => JBitSet, HashSet => JHashSet} import scala.collection.JavaConversions._ import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.BitSet + import org.apache.spark._ @@ -41,11 +43,15 @@ class IndexedRDDFunctions[K: ClassManifest, V: ClassManifest](self: IndexedRDD[K */ override def mapValues[U: ClassManifest](f: V => U): RDD[(K, U)] = { val cleanF = self.index.rdd.context.clean(f) - val newValues = self.valuesRDD.mapPartitions(_.map(values => values.map{ - case null => null - case row => row.map(x => f(x)) - }), true) - new IndexedRDD[K,U](self.index, newValues) + val newValuesRDD = self.valuesRDD.mapPartitions(iter => iter.map{ + case (values, bs) => + val newValues = new Array[U](values.size) + for ( ind <- bs ) { + newValues(ind) = f(values(ind)) + } + (newValues, bs) + }, preservesPartitioning = true) + new IndexedRDD[K,U](self.index, newValuesRDD) } @@ -55,38 +61,37 @@ class IndexedRDDFunctions[K: ClassManifest, V: ClassManifest](self: IndexedRDD[K */ override def mapValuesWithKeys[U: ClassManifest](f: (K, V) => U): RDD[(K, U)] = { val cleanF = self.index.rdd.context.clean(f) - val newValues = self.index.rdd.zipPartitions(self.valuesRDD){ (keysIter, valuesIter) => + val newValues = self.index.rdd.zipPartitions(self.valuesRDD){ + (keysIter, valuesIter) => val index = keysIter.next() assert(keysIter.hasNext() == false) - val oldValues = valuesIter.next() + val (oldValues, bs) = valuesIter.next() assert(valuesIter.hasNext() == false) - // Allocate the array to store the results into - val newValues: Array[Seq[U]] = new Array[Seq[U]](oldValues.size) + // Allocate the array to store the results into + val newValues: Array[U] = new Array[U](oldValues.size) // Populate the new Values for( (k,i) <- index ) { - if(oldValues(i) != null) { - newValues(i) = oldValues(i).map( v => f(k,v) ) - } + if (bs(i)) { newValues(i) = f(k, oldValues(i)) } } - Array(newValues.toSeq).iterator + Array((newValues, bs)).iterator } new IndexedRDD[K,U](self.index, newValues) } - /** - * Pass each value in the key-value pair RDD through a flatMap function without changing the - * keys; this also retains the original RDD's partitioning. - */ - override def flatMapValues[U: ClassManifest](f: V => TraversableOnce[U]): RDD[(K,U)] = { - val cleanF = self.index.rdd.context.clean(f) - val newValues = self.valuesRDD.mapPartitions(_.map(values => values.map{ - case null => null - case row => row.flatMap(x => f(x)) - }), true) - new IndexedRDD[K,U](self.index, newValues) - } + // /** + // * Pass each value in the key-value pair RDD through a flatMap function without changing the + // * keys; this also retains the original RDD's partitioning. + // */ + // override def flatMapValues[U: ClassManifest](f: V => TraversableOnce[U]): RDD[(K,U)] = { + // val cleanF = self.index.rdd.context.clean(f) + // val newValues = self.valuesRDD.mapPartitions(_.map(values => values.map{ + // case null => null + // case row => row.flatMap(x => f(x)) + // }), true) + // new IndexedRDD[K,U](self.index, newValues) + // } /** @@ -105,31 +110,19 @@ class IndexedRDDFunctions[K: ClassManifest, V: ClassManifest](self: IndexedRDD[K partitioner: Partitioner, mapSideCombine: Boolean = true, serializerClass: String = null): RDD[(K, C)] = { - val newValues = self.valuesRDD.mapPartitions( - _.map{ groups: Seq[Seq[V]] => - groups.map{ group: Seq[V] => - if (group != null && !group.isEmpty) { - val c: C = createCombiner(group.head) - val sum: C = group.tail.foldLeft(c)(mergeValue) - Seq(sum) - } else { - null - } - } - }, true) - new IndexedRDD[K,C](self.index, newValues) + mapValues(createCombiner) } - /** - * Group the values for each key in the RDD into a single sequence. Hash-partitions the - * resulting RDD with the existing partitioner/parallelism level. - */ - override def groupByKey(partitioner: Partitioner): RDD[(K, Seq[V])] = { - val newValues = self.valuesRDD.mapPartitions(_.map{ar => ar.map{s => Seq(s)} }, true) - new IndexedRDD[K, Seq[V]](self.index, newValues) - } + // /** + // * Group the values for each key in the RDD into a single sequence. Hash-partitions the + // * resulting RDD with the existing partitioner/parallelism level. + // */ + // override def groupByKey(partitioner: Partitioner): RDD[(K, Seq[V])] = { + // val newValues = self.valuesRDD.mapPartitions(_.map{ar => ar.map{s => Seq(s)} }, true) + // new IndexedRDD[K, Seq[V]](self.index, newValues) + // } /** @@ -146,23 +139,24 @@ class IndexedRDDFunctions[K: ClassManifest, V: ClassManifest](self: IndexedRDD[K // However it is possible that both RDDs are missing a value for a given key in // which case the returned RDD should have a null value val newValues = - self.valuesRDD.zipPartitions(other.valuesRDD)( - (thisIter, otherIter) => { - val thisValues: Seq[Seq[V]] = thisIter.next() + self.valuesRDD.zipPartitions(other.valuesRDD){ + (thisIter, otherIter) => + val (thisValues, thisBS) = thisIter.next() assert(!thisIter.hasNext) - val otherValues: Seq[Seq[W]] = otherIter.next() - assert(!otherIter.hasNext) - // Zip the values and if both arrays are null then the key is not present and - // so the resulting value must be null (not a tuple of empty sequences) - val tmp: Seq[Seq[(Seq[V], Seq[W])]] = thisValues.view.zip(otherValues).map{ - case (null, null) => null // The key is not present in either RDD - case (a, null) => Seq((a, Seq.empty[W])) - case (null, b) => Seq((Seq.empty[V], b)) - case (a, b) => Seq((a,b)) - }.toSeq - List(tmp).iterator - }) - new IndexedRDD[K, (Seq[V], Seq[W])](self.index, newValues) + val (otherValues, otherBS) = otherIter.next() + assert(!otherIter.hasNext) + + val newValues = new Array[(Seq[V], Seq[W])](thisValues.size) + val newBS = thisBS | otherBS + + for( ind <- newBS ) { + val a = if (thisBS(ind)) Seq(thisValues(ind)) else Seq.empty[V] + val b = if (otherBS(ind)) Seq(otherValues(ind)) else Seq.empty[W] + newValues(ind) = (a, b) + } + List((newValues, newBS)).iterator + } + new IndexedRDD(self.index, newValues) } case other: IndexedRDD[_, _] if self.index.rdd.partitioner == other.index.rdd.partitioner => { @@ -197,26 +191,33 @@ class IndexedRDDFunctions[K: ClassManifest, V: ClassManifest](self: IndexedRDD[K val newIndex = newIndexIter.next() assert(!newIndexIter.hasNext) // Get the corresponding indicies and values for this and the other IndexedRDD - val (thisIndex, thisValues) = thisTuplesIter.next() + val (thisIndex, (thisValues, thisBS)) = thisTuplesIter.next() assert(!thisTuplesIter.hasNext) - val (otherIndex, otherValues) = otherTuplesIter.next() + val (otherIndex, (otherValues, otherBS)) = otherTuplesIter.next() assert(!otherTuplesIter.hasNext) // Preallocate the new Values array - val newValues = new Array[Seq[(Seq[V],Seq[W])]](newIndex.size) + val newValues = new Array[(Seq[V], Seq[W])](newIndex.size) + val newBS = new BitSet(newIndex.size) + // Lookup the sequences in both submaps for ((k,ind) <- newIndex) { - val thisSeq = if (thisIndex.contains(k)) thisValues(thisIndex.get(k)) else null - val otherSeq = if (otherIndex.contains(k)) otherValues(otherIndex.get(k)) else null - // if either of the sequences is not null then the key was in one of the two tables - // and so the value should appear in the returned table - newValues(ind) = (thisSeq, otherSeq) match { - case (null, null) => null - case (a, null) => Seq( (a, Seq.empty[W]) ) - case (null, b) => Seq( (Seq.empty[V], b) ) - case (a, b) => Seq( (a,b) ) + // Get the left key + val a = if (thisIndex.contains(k)) { + val ind = thisIndex.get(k) + if(thisBS(ind)) Seq(thisValues(ind)) else Seq.empty[V] + } else Seq.empty[V] + // Get the right key + val b = if (otherIndex.contains(k)) { + val ind = otherIndex.get(k) + if (otherBS(ind)) Seq(otherValues(ind)) else Seq.empty[W] + } else Seq.empty[W] + // If at least one key was present then we generate a tuple. + if (!a.isEmpty || !b.isEmpty) { + newValues(ind) = (a, b) + newBS(ind) = true } } - List(newValues.toSeq).iterator + List((newValues, newBS)).iterator }) new IndexedRDD(new RDDIndex(newIndex), newValues) } @@ -238,44 +239,48 @@ class IndexedRDDFunctions[K: ClassManifest, V: ClassManifest](self: IndexedRDD[K self.tuples.zipPartitions(otherShuffled)( (thisTuplesIter, otherTuplesIter) => { // Get the corresponding indicies and values for this IndexedRDD - val (thisIndex, thisValues) = thisTuplesIter.next() + val (thisIndex, (thisValues, thisBS)) = thisTuplesIter.next() assert(!thisTuplesIter.hasNext()) // Construct a new index val newIndex = thisIndex.clone().asInstanceOf[BlockIndex[K]] // Construct a new array Buffer to store the values - val newValues = ArrayBuffer.fill[(Seq[V], Seq[W])](thisValues.size)(null) + val newValues = ArrayBuffer.fill[ (Seq[V], Seq[W]) ](thisValues.size)(null) + val newBS = new BitSet(thisValues.size) // populate the newValues with the values in this IndexedRDD for ((k,i) <- thisIndex) { - if (thisValues(i) != null) { - newValues(i) = (thisValues(i), ArrayBuffer.empty[W]) + if (thisBS(i)) { + newValues(i) = (Seq(thisValues(i)), ArrayBuffer.empty[W]) + newBS(i) = true } } // Now iterate through the other tuples updating the map for ((k,w) <- otherTuplesIter){ - if (!newIndex.contains(k)) { - // update the index - val ind = newIndex.size - newIndex.put(k, ind) - // Update the values - newValues.append( (Seq.empty[V], ArrayBuffer(w) ) ) - } else { + if (newIndex.contains(k)) { val ind = newIndex.get(k) - if(newValues(ind) == null) { + if(newBS(ind)) { + newValues(ind)._2.asInstanceOf[ArrayBuffer[W]].append(w) + } else { // If the other key was in the index but not in the values // of this indexed RDD then create a new values entry for it + newBS(ind) = true newValues(ind) = (Seq.empty[V], ArrayBuffer(w)) - } else { - newValues(ind)._2.asInstanceOf[ArrayBuffer[W]].append(w) - } + } + } else { + // update the index + val ind = newIndex.size + newIndex.put(k, ind) + newBS(ind) = true + // Update the values + newValues.append( (Seq.empty[V], ArrayBuffer(w) ) ) } } - // Finalize the new values array - val newValuesArray: Seq[Seq[(Seq[V],Seq[W])]] = - newValues.view.map{ - case null => null - case (s, ab) => Seq((s, ab.toSeq)) - }.toSeq - List( (newIndex, newValuesArray) ).iterator + // // Finalize the new values array + // val newValuesArray: Seq[Seq[(Seq[V],Seq[W])]] = + // newValues.view.map{ + // case null => null + // case (s, ab) => Seq((s, ab.toSeq)) + // }.toSeq + List( (newIndex, (newValues.toArray, newBS)) ).iterator }).cache() // Extract the index and values from the above RDD diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 569d74ae7a6a1..eeb60d3ff9377 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -704,6 +704,9 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) def values: RDD[V] = self.map(_._2) + def indexed(): IndexedRDD[K,V] = IndexedRDD(self) + + def indexed(numPartitions: Int): IndexedRDD[K,V] = IndexedRDD(self.partitionBy(new HashPartitioner(numPartitions))) @@ -711,9 +714,11 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) IndexedRDD(self.partitionBy(partitioner)) - def indexed(existingIndex: RDDIndex[K] = null): IndexedRDD[K,V] = + def indexed(existingIndex: RDDIndex[K]): IndexedRDD[K,V] = IndexedRDD(self, existingIndex) + + private[spark] def getKeyClass() = implicitly[ClassManifest[K]].erasure private[spark] def getValueClass() = implicitly[ClassManifest[V]].erasure From 59700c0c2a9827cf26f08d1b20681f2c26cd4a0d Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Wed, 16 Oct 2013 00:18:37 -0700 Subject: [PATCH 119/531] switched to more efficienct implementation of reduce by key --- .../org/apache/spark/rdd/IndexedRDD.scala | 44 +++++++++++++++++++ .../apache/spark/rdd/PairRDDFunctions.scala | 2 +- .../org/apache/spark/graph/EdgeTriplet.scala | 4 +- .../apache/spark/graph/impl/GraphImpl.scala | 16 +++---- 4 files changed, 55 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/IndexedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/IndexedRDD.scala index a881ee3a1da35..e099669c22cb7 100644 --- a/core/src/main/scala/org/apache/spark/rdd/IndexedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/IndexedRDD.scala @@ -216,6 +216,50 @@ object IndexedRDD { } } + + def reduceByKey[K: ClassManifest, V: ClassManifest]( + rdd: RDD[(K,V)], reduceFun: (V, V) => V, index: RDDIndex[K]): IndexedRDD[K,V] = { + // Get the index Partitioner + val partitioner = index.rdd.partitioner match { + case Some(p) => p + case None => throw new SparkException("An index must have a partitioner.") + } + // Preaggregate and shuffle if necessary + val partitioned = + if (rdd.partitioner != Some(partitioner)) { + // Preaggregation. + val aggregator = new Aggregator[K, V, V](v => v, reduceFun, reduceFun) + val combined = rdd.mapPartitions(aggregator.combineValuesByKey, preservesPartitioning = true) + combined.partitionBy(partitioner) //new ShuffledRDD[K, V, (K, V)](combined, partitioner) + } else { + rdd + } + + // Use the index to build the new values table + val values = index.rdd.zipPartitions(partitioned)( (indexIter, tblIter) => { + // There is only one map + val index = indexIter.next() + assert(!indexIter.hasNext()) + val values = new Array[Array[V]](index.size) + for ((k,v) <- tblIter) { + if (!index.contains(k)) { + throw new SparkException("Error: Trying to bind an external index " + + "to an RDD which contains keys that are not in the index.") + } + val ind = index(k) + if (values(ind) == null) { + values(ind) = Array(v) + } else { + values(ind)(0) = reduceFun(values(ind).head, v) + } + } + List(values.view.map(x => if (x != null) x.toSeq else null ).toSeq).iterator + }) + + new IndexedRDD[K,V](index, values) + + } + /** * Construct and index of the unique values in a given RDD. */ diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 7fadbcf4eccc5..5d00917dabd07 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -245,7 +245,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) if (getKeyClass().isArray && partitioner.isInstanceOf[HashPartitioner]) { throw new SparkException("Default partitioner cannot partition array keys.") } - new ShuffledRDD[K, V, (K, V)](self, partitioner) + new ShuffledRDD[K, V, (K, V)](self, partitioner) } /** diff --git a/graph/src/main/scala/org/apache/spark/graph/EdgeTriplet.scala b/graph/src/main/scala/org/apache/spark/graph/EdgeTriplet.scala index 7dfb5caa4c529..c2ef63d1fd3d8 100644 --- a/graph/src/main/scala/org/apache/spark/graph/EdgeTriplet.scala +++ b/graph/src/main/scala/org/apache/spark/graph/EdgeTriplet.scala @@ -18,12 +18,12 @@ class EdgeTriplet[VD, ED] extends Edge[ED] { /** * The source vertex attribute */ - var srcAttr: VD = nullValue[VD] + var srcAttr: VD = _ //nullValue[VD] /** * The destination vertex attribute */ - var dstAttr: VD = nullValue[VD] + var dstAttr: VD = _ //nullValue[VD] /** * Set the edge properties of this triplet. diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index aa0aaaaef4b90..73538862b11ce 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -341,9 +341,9 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( ClosureCleaner.clean(mapFunc) ClosureCleaner.clean(reduceFunc) - val newVTable: RDD[(Vid, A)] = - vTableReplicated.join(eTable).flatMap{ - case (pid, (vmap, edgePartition)) => + // Map and preaggregate + val preAgg = vTableReplicated.join(eTable).flatMap{ + case (pid, (vmap, edgePartition)) => val aggMap = new VertexHashMap[A] val et = new EdgeTriplet[VD, ED] edgePartition.foreach{e => @@ -353,17 +353,17 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( mapFunc(et).foreach{case (vid, a) => if(aggMap.containsKey(vid)) { aggMap.put(vid, reduceFunc(aggMap.get(vid), a)) - } else { aggMap.put(vid, a) } + } else { aggMap.put(vid, a) } + } } - } // Return the aggregate map aggMap.long2ObjectEntrySet().fastIterator().map{ entry => (entry.getLongKey(), entry.getValue()) } - } - .indexed(vTable.index).reduceByKey(reduceFunc) + }.partitionBy(vTable.index.rdd.partitioner.get) - newVTable + // do the final reduction reusing the index map + IndexedRDD.reduceByKey(preAgg, reduceFunc, vTable.index) } From af8e4618416923c499d5efc0b08538b7fd49e198 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Wed, 16 Oct 2013 19:21:24 -0700 Subject: [PATCH 120/531] Set serialization properties in GraphSuite --- graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala index 1b03b8094e379..145be3c126a38 100644 --- a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala @@ -10,6 +10,9 @@ class GraphSuite extends FunSuite with LocalSparkContext { // val sc = new SparkContext("local[4]", "test") + System.setProperty("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + System.setProperty("spark.kryo.registrator", "org.apache.spark.graph.GraphKryoRegistrator") + test("Graph Creation") { withSpark(new SparkContext("local", "test")) { sc => val rawEdges = (0L to 100L).zip((1L to 99L) :+ 0L) From bc234bf0e1ecc5c9bba2cd629644780c9d5c3511 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Wed, 16 Oct 2013 18:59:54 -0700 Subject: [PATCH 121/531] Split vTableReplicated into two RDDs Previously, (vTableReplicated: IndexedRDD[Pid, VertexHashMap[VD]]) stored one hashmap per partition, taking Vid directly to VD. To take advantage of rxin's new hashmaps (see rxin/incubator-spark@32a79d6d13be8a03071296d251a3e532bf964cc0), this commit splits that data structure into two RDDs: (vTableReplicationMap: IndexedRDD[Pid, VertexIdToIndexMap]) stores a map per partition from vertex ID to the index where that vertex's attribute is stored. This index refers to an array in the same partition in vTableReplicatedValues. (vTableReplicatedValues: IndexedRDD[Pid, Array[VD]]) stores the vertex data and is arranged as described above. --- .../apache/spark/graph/impl/GraphImpl.scala | 114 ++++++++++-------- .../org/apache/spark/graph/package.scala | 2 + 2 files changed, 68 insertions(+), 48 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index b6758b0501e7c..5398631d26ba0 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -24,7 +24,8 @@ import org.apache.spark.graph.impl.MessageToPartitionRDDFunctions._ * The Iterator type returned when constructing edge triplets */ class EdgeTripletIterator[VD: ClassManifest, ED: ClassManifest]( - val vmap: VertexHashMap[VD], + val vidToIndex: VertexIdToIndexMap, + val vertexArray: Array[VD], val edgePartition: EdgePartition[ED]) extends Iterator[EdgeTriplet[VD, ED]] { private var pos = 0 @@ -34,10 +35,10 @@ class EdgeTripletIterator[VD: ClassManifest, ED: ClassManifest]( override def next() = { et.srcId = edgePartition.srcIds(pos) // assert(vmap.containsKey(e.src.id)) - et.srcAttr = vmap.get(et.srcId) + et.srcAttr = vertexArray(vidToIndex(et.srcId)) et.dstId = edgePartition.dstIds(pos) // assert(vmap.containsKey(e.dst.id)) - et.dstAttr = vmap.get(et.dstId) + et.dstAttr = vertexArray(vidToIndex(et.dstId)) //println("Iter called: " + pos) et.attr = edgePartition.data(pos) pos += 1 @@ -50,10 +51,10 @@ class EdgeTripletIterator[VD: ClassManifest, ED: ClassManifest]( for (i <- (0 until edgePartition.size)) { currentEdge.srcId = edgePartition.srcIds(i) // assert(vmap.containsKey(e.src.id)) - currentEdge.srcAttr = vmap.get(currentEdge.srcId) + currentEdge.srcAttr = vertexArray(vidToIndex(currentEdge.srcId)) currentEdge.dstId = edgePartition.dstIds(i) // assert(vmap.containsKey(e.dst.id)) - currentEdge.dstAttr = vmap.get(currentEdge.dstId) + currentEdge.dstAttr = vertexArray(vidToIndex(currentEdge.dstId)) currentEdge.attr = edgePartition.data(i) lb += currentEdge } @@ -63,17 +64,18 @@ class EdgeTripletIterator[VD: ClassManifest, ED: ClassManifest]( object EdgeTripletBuilder { def makeTriplets[VD: ClassManifest, ED: ClassManifest]( - vTableReplicated: IndexedRDD[Pid, VertexHashMap[VD]], + vTableReplicationMap: IndexedRDD[Pid, VertexIdToIndexMap], + vTableReplicatedValues: IndexedRDD[Pid, Array[VD]], eTable: IndexedRDD[Pid, EdgePartition[ED]]): RDD[EdgeTriplet[VD, ED]] = { - val iterFun = (iter: Iterator[(Pid, (VertexHashMap[VD], EdgePartition[ED]))]) => { - val (pid, (vmap, edgePartition)) = iter.next() + val iterFun = (iter: Iterator[(Pid, ((VertexIdToIndexMap, Array[VD]), EdgePartition[ED]))]) => { + val (pid, ((vidToIndex, vertexArray), edgePartition)) = iter.next() //assert(iter.hasNext == false) // Return an iterator that looks up the hash map to find matching // vertices for each edge. - new EdgeTripletIterator(vmap, edgePartition) + new EdgeTripletIterator(vidToIndex, vertexArray, edgePartition) } ClosureCleaner.clean(iterFun) - vTableReplicated.zipJoinRDD(eTable) + vTableReplicationMap.zipJoin(vTableReplicatedValues).zipJoinRDD(eTable) .mapPartitions( iterFun ) // end of map partition } @@ -93,13 +95,16 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( /** - * The vTableReplicated is a version of the vertex data after it is - * replicated. + * (vTableReplicationMap: IndexedRDD[Pid, VertexIdToIndexMap]) is a version of the + * vertex data after it is replicated. Within each partition, it holds a map + * from vertex ID to the index where that vertex's attribute is stored. This + * index refers to an array in the same partition in vTableReplicatedValues. + * + * (vTableReplicatedValues: IndexedRDD[Pid, Array[VD]]) holds the vertex data + * and is arranged as described above. */ - @transient val vTableReplicated: IndexedRDD[Pid, VertexHashMap[VD]] = - createVTableReplicated(vTable, vid2pid, eTable) - - + @transient val (vTableReplicationMap, vTableReplicatedValues) = + createVTableReplicated(vTable, vid2pid, eTable) /** Return a RDD of vertices. */ @@ -114,7 +119,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( /** Return a RDD that brings edges with its source and destination vertices together. */ @transient override val triplets: RDD[EdgeTriplet[VD, ED]] = - EdgeTripletBuilder.makeTriplets(vTableReplicated, eTable) + EdgeTripletBuilder.makeTriplets(vTableReplicationMap, vTableReplicatedValues, eTable) // { @@ -136,8 +141,6 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( eTable.cache() vid2pid.cache() vTable.cache() - /** @todo should we cache the replicated data? */ - vTableReplicated.cache() this } @@ -179,15 +182,15 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( override def mapTriplets[ED2: ClassManifest](f: EdgeTriplet[VD, ED] => ED2): Graph[VD, ED2] = { - val newETable = eTable.join(vTableReplicated).mapValues{ - case (edgePartition, vmap) => - val et = new EdgeTriplet[VD, ED] - edgePartition.map{e => - et.set(e) - et.srcAttr = vmap(e.srcId) - et.dstAttr = vmap(e.dstId) - f(et) - } + val newETable = eTable.zipJoin(vTableReplicationMap).zipJoin(vTableReplicatedValues).mapValues{ + case ((edgePartition, vidToIndex), vertexArray) => + val et = new EdgeTriplet[VD, ED] + edgePartition.map{e => + et.set(e) + et.srcAttr = vertexArray(vidToIndex(e.srcId)) + et.dstAttr = vertexArray(vidToIndex(e.dstId)) + f(et) + } }.asInstanceOf[IndexedRDD[Pid, EdgePartition[ED2]]] new GraphImpl(vTable, vid2pid, newETable) } @@ -344,20 +347,20 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( ClosureCleaner.clean(reduceFunc) // Map and preaggregate - val preAgg = vTableReplicated.zipJoinRDD(eTable).flatMap{ - case (pid, (vmap, edgePartition)) => + val preAgg = vTableReplicationMap.zipJoin(vTableReplicatedValues).zipJoinRDD(eTable).flatMap{ + case (pid, ((vidToIndex, vertexArray), edgePartition)) => val aggMap = new VertexHashMap[A] val et = new EdgeTriplet[VD, ED] edgePartition.foreach{e => et.set(e) - et.srcAttr = vmap(e.srcId) - et.dstAttr = vmap(e.dstId) + et.srcAttr = vertexArray(vidToIndex(e.srcId)) + et.dstAttr = vertexArray(vidToIndex(e.dstId)) mapFunc(et).foreach{case (vid, a) => if(aggMap.containsKey(vid)) { - aggMap.put(vid, reduceFunc(aggMap.get(vid), a)) - } else { aggMap.put(vid, a) } - } + aggMap.put(vid, reduceFunc(aggMap.get(vid), a)) + } else { aggMap.put(vid, a) } } + } // Return the aggregate map aggMap.long2ObjectEntrySet().fastIterator().map{ entry => (entry.getLongKey(), entry.getValue()) @@ -475,21 +478,36 @@ object GraphImpl { protected def createVTableReplicated[VD: ClassManifest, ED: ClassManifest]( vTable: IndexedRDD[Vid, VD], vid2pid: IndexedRDD[Vid, Array[Pid]], eTable: IndexedRDD[Pid, EdgePartition[ED]]): - IndexedRDD[Pid, VertexHashMap[VD]] = { + (IndexedRDD[Pid, VertexIdToIndexMap], IndexedRDD[Pid, Array[VD]]) = { // Join vid2pid and vTable, generate a shuffle dependency on the joined // result, and get the shuffle id so we can use it on the slave. - vTable.zipJoinRDD(vid2pid) - .flatMap { case (vid, (vdata, pids)) => - pids.iterator.map { pid => MessageToPartition(pid, (vid, vdata)) } - } - .partitionBy(eTable.partitioner.get) //@todo assert edge table has partitioner - .mapPartitionsWithIndex( (pid, iter) => { - // Build the hashmap for each partition - val vmap = new VertexHashMap[VD] - for( msg <- iter ) { vmap.put(msg.data._1, msg.data._2) } - Array((pid, vmap)).iterator - }, preservesPartitioning = true) - .indexed(eTable.index) + val msgsByPartition = + vTable.zipJoinRDD(vid2pid) + .flatMap { case (vid, (vdata, pids)) => + pids.iterator.map { pid => MessageToPartition(pid, (vid, vdata)) } + } + .partitionBy(eTable.partitioner.get) //@todo assert edge table has partitioner + + val vTableReplicationMap: IndexedRDD[Pid, VertexIdToIndexMap] = + msgsByPartition.mapPartitionsWithIndex( (pid, iter) => { + val vidToIndex = new VertexIdToIndexMap + var i = 0 + for (msg <- iter) { + vidToIndex.put(msg.data._1, i) + } + Array((pid, vidToIndex)).iterator + }, preservesPartitioning = true).indexed(eTable.index) + + val vTableReplicatedValues: IndexedRDD[Pid, Array[VD]] = + msgsByPartition.mapPartitionsWithIndex( (pid, iter) => { + val vertexArray = new ArrayBuffer[VD] + for (msg <- iter) { + vertexArray += msg.data._2 + } + Array((pid, vertexArray.toArray)).iterator + }, preservesPartitioning = true).indexed(eTable.index) + + (vTableReplicationMap, vTableReplicatedValues) } diff --git a/graph/src/main/scala/org/apache/spark/graph/package.scala b/graph/src/main/scala/org/apache/spark/graph/package.scala index 474ace520f440..47d5acb9e76ba 100644 --- a/graph/src/main/scala/org/apache/spark/graph/package.scala +++ b/graph/src/main/scala/org/apache/spark/graph/package.scala @@ -8,6 +8,8 @@ package object graph { type VertexHashMap[T] = it.unimi.dsi.fastutil.longs.Long2ObjectOpenHashMap[T] type VertexSet = it.unimi.dsi.fastutil.longs.LongOpenHashSet type VertexArrayList = it.unimi.dsi.fastutil.longs.LongArrayList + // @todo replace with rxin's fast hashmap + type VertexIdToIndexMap = scala.collection.mutable.HashMap[Vid, Int] /** * Return the default null-like value for a data type T. From 2282d27cf1da04bfdad5173c412e77a51d7c36e6 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Wed, 16 Oct 2013 23:56:15 -0700 Subject: [PATCH 122/531] Cache msgsByPartition --- .../src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index 5398631d26ba0..a3cdb97ceade4 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -486,7 +486,8 @@ object GraphImpl { .flatMap { case (vid, (vdata, pids)) => pids.iterator.map { pid => MessageToPartition(pid, (vid, vdata)) } } - .partitionBy(eTable.partitioner.get) //@todo assert edge table has partitioner + .partitionBy(eTable.partitioner.get).cache() + // @todo assert edge table has partitioner val vTableReplicationMap: IndexedRDD[Pid, VertexIdToIndexMap] = msgsByPartition.mapPartitionsWithIndex( (pid, iter) => { From bf19aac2b79d43f90d99e6206ba754b1e6b0f58c Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Thu, 17 Oct 2013 13:19:00 -0700 Subject: [PATCH 123/531] Use ArrayBuilder instead of ArrayBuffer ArrayBuilder is specialized for holding primitive VD types. --- .../main/scala/org/apache/spark/graph/impl/GraphImpl.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index a3cdb97ceade4..87fb9dcd2e05f 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -4,6 +4,7 @@ import scala.collection.JavaConversions._ import scala.collection.mutable import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.ArrayBuilder import org.apache.spark.SparkContext._ import org.apache.spark.Partitioner @@ -501,11 +502,11 @@ object GraphImpl { val vTableReplicatedValues: IndexedRDD[Pid, Array[VD]] = msgsByPartition.mapPartitionsWithIndex( (pid, iter) => { - val vertexArray = new ArrayBuffer[VD] + val vertexArray = ArrayBuilder.make[VD] for (msg <- iter) { vertexArray += msg.data._2 } - Array((pid, vertexArray.toArray)).iterator + Array((pid, vertexArray.result)).iterator }, preservesPartitioning = true).indexed(eTable.index) (vTableReplicationMap, vTableReplicatedValues) From 9a03c5fe28dffcc8c498e2a1605e714aca3ad77e Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Thu, 17 Oct 2013 19:01:48 -0700 Subject: [PATCH 124/531] This commit accomplishes three goals: 1) Further simplification of the IndexedRDD operations (eliminating some) 2) Aggressive reuse of HashMaps 3) Pipelining join operations within indexedrdd --- .../org/apache/spark/rdd/IndexedRDD.scala | 180 ++++++++++++--- .../spark/rdd/IndexedRDDFunctions.scala | 66 ++---- .../org/apache/spark/graph/EdgeTriplet.scala | 3 +- .../apache/spark/graph/impl/GraphImpl.scala | 209 ++++++++++++------ .../org/apache/spark/graph/package.scala | 2 +- 5 files changed, 309 insertions(+), 151 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/IndexedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/IndexedRDD.scala index 30566adba37d7..2f1f907c6c746 100644 --- a/core/src/main/scala/org/apache/spark/rdd/IndexedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/IndexedRDD.scala @@ -72,7 +72,7 @@ class RDDIndex[@specialized K: ClassManifest](private[spark] val rdd: RDD[BlockI */ class IndexedRDD[K: ClassManifest, V: ClassManifest]( @transient val index: RDDIndex[K], - @transient val valuesRDD: RDD[ (Array[V], BitSet) ]) + @transient val valuesRDD: RDD[ (Seq[V], BitSet) ]) extends RDD[(K, V)](index.rdd.context, List(new OneToOneDependency(index.rdd), new OneToOneDependency(valuesRDD)) ) { @@ -113,46 +113,160 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( } + /** + * Pass each value in the key-value pair RDD through a map function without changing the keys; + * this also retains the original RDD's partitioning. + */ + def mapValues[U: ClassManifest](f: V => U): IndexedRDD[K, U] = { + val cleanF = index.rdd.context.clean(f) + val newValuesRDD = valuesRDD.mapPartitions(iter => iter.map{ + case (values, bs) => + val newValues = new Array[U](values.size) + for ( ind <- bs ) { + newValues(ind) = f(values(ind)) + } + (newValues.toSeq, bs) + }, preservesPartitioning = true) + new IndexedRDD[K,U](index, newValuesRDD) + } + + /** + * Pass each value in the key-value pair RDD through a map function without changing the keys; + * this also retains the original RDD's partitioning. + */ + def mapValuesWithKeys[U: ClassManifest](f: (K, V) => U): IndexedRDD[K, U] = { + val cleanF = index.rdd.context.clean(f) + val newValues = index.rdd.zipPartitions(valuesRDD){ + (keysIter, valuesIter) => + val index = keysIter.next() + assert(keysIter.hasNext() == false) + val (oldValues, bs) = valuesIter.next() + assert(valuesIter.hasNext() == false) + // Allocate the array to store the results into + val newValues: Array[U] = new Array[U](oldValues.size) + // Populate the new Values + for( (k,i) <- index ) { + if (bs(i)) { newValues(i) = f(k, oldValues(i)) } + } + Array((newValues.toSeq, bs)).iterator + } + new IndexedRDD[K,U](index, newValues) + } - def zipJoinRDD[W: ClassManifest](other: IndexedRDD[K,W]): RDD[(K,(V,W))] = { - assert(index == other.index) - index.rdd.zipPartitions(valuesRDD, other.valuesRDD){ - (thisIndexIter, thisIter, otherIter) => - val index = thisIndexIter.next() - assert(!thisIndexIter.hasNext) + + def zipJoin[W: ClassManifest](other: IndexedRDD[K,W]): IndexedRDD[K,(V,W)] = { + if(index != other.index) { + throw new SparkException("A zipJoin can only be applied to RDDs with the same index!") + } + val newValuesRDD: RDD[ (Seq[(V,W)], BitSet) ] = valuesRDD.zipPartitions(other.valuesRDD){ + (thisIter, otherIter) => val (thisValues, thisBS) = thisIter.next() assert(!thisIter.hasNext) val (otherValues, otherBS) = otherIter.next() assert(!otherIter.hasNext) val newBS = thisBS & otherBS - - index.iterator.flatMap{ case (k,i) => - if(newBS(i)) List((k, (thisValues(i), otherValues(i)))) - else List.empty - } + val newValues = thisValues.view.zip(otherValues) + Iterator((newValues, newBS)) } + new IndexedRDD(index, newValuesRDD) } - def zipJoin[W: ClassManifest](other: IndexedRDD[K,W]): IndexedRDD[K,(V,W)] = { - assert(index == other.index) - val newValuesRDD = valuesRDD.zipPartitions(other.valuesRDD){ + + def leftZipJoin[W: ClassManifest](other: IndexedRDD[K,W]): IndexedRDD[K,(V,Option[W])] = { + if(index != other.index) { + throw new SparkException("A zipJoin can only be applied to RDDs with the same index!") + } + val newValuesRDD: RDD[ (Seq[(V,Option[W])], BitSet) ] = valuesRDD.zipPartitions(other.valuesRDD){ (thisIter, otherIter) => val (thisValues, thisBS) = thisIter.next() assert(!thisIter.hasNext) val (otherValues, otherBS) = otherIter.next() assert(!otherIter.hasNext) - val newBS = thisBS & otherBS - val newValues = new Array[(V,W)](thisValues.size) - for( i <- newBS ) { - newValues(i) = (thisValues(i), otherValues(i)) - } - List((newValues, newBS)).iterator + val otherOption = otherValues.view.zipWithIndex + .map{ (x: (W, Int)) => if(otherBS(x._2)) Option(x._1) else None } + val newValues = thisValues.view.zip(otherOption) + Iterator((newValues, thisBS)) } new IndexedRDD(index, newValuesRDD) } + + def leftJoin[W: ClassManifest]( + other: RDD[(K,W)], merge: (W,W) => W = (a:W, b:W) => a): + IndexedRDD[K, (V, Option[W]) ] = { + val cleanMerge = index.rdd.context.clean(merge) + other match { + case other: IndexedRDD[_, _] if index == other.index => { + leftZipJoin(other) + } + case _ => { + // Get the partitioner from the index + val partitioner = index.rdd.partitioner match { + case Some(p) => p + case None => throw new SparkException("An index must have a partitioner.") + } + // Shuffle the other RDD using the partitioner for this index + val otherShuffled = + if (other.partitioner == Some(partitioner)) other + else other.partitionBy(partitioner) + val newValues = index.rdd.zipPartitions(valuesRDD, other) { + (thisIndexIter, thisIter, tuplesIter) => + val index = thisIndexIter.next() + assert(!thisIndexIter.hasNext) + val (thisValues, thisBS) = thisIter.next() + assert(!thisIter.hasNext) + val newW = new Array[W](thisValues.size) + // track which values are matched with values in other + val wBS = new BitSet(thisValues.size) + for( (k, w) <- tuplesIter if index.contains(k) ) { + val ind = index.get(k) + if(thisBS(ind)) { + if(wBS(ind)) { + newW(ind) = cleanMerge(newW(ind), w) + } else { + newW(ind) = w + wBS(ind) = true + } + } + } + + val otherOption = newW.view.zipWithIndex + .map{ (x: (W, Int)) => if(wBS(x._2)) Option(x._1) else None } + val newValues = thisValues.view.zip(otherOption) + + Iterator((newValues.toSeq, thisBS)) + } // end of newValues + new IndexedRDD(index, newValues) + } + } + } + + + + // + // def zipJoinToRDD[W: ClassManifest](other: IndexedRDD[K,W]): RDD[(K,(V,W))] = { + // if(index != other.index) { + // throw new SparkException("ZipJoinRDD can only be applied to RDDs with the same index!") + // } + // index.rdd.zipPartitions(valuesRDD, other.valuesRDD){ + // (thisIndexIter, thisIter, otherIter) => + // val index = thisIndexIter.next() + // assert(!thisIndexIter.hasNext) + // val (thisValues, thisBS) = thisIter.next() + // assert(!thisIter.hasNext) + // val (otherValues, otherBS) = otherIter.next() + // assert(!otherIter.hasNext) + // val newBS = thisBS & otherBS + // index.iterator.filter{ case (k,i) => newBS(i) }.map{ + // case (k,i) => (k, (thisValues(i), otherValues(i))) + // } + // } + // } + + +/* This is probably useful but we are not using it def zipJoinWithKeys[W: ClassManifest, Z: ClassManifest]( other: RDD[(K,W)])( f: (K, V, W) => Z, @@ -222,9 +336,9 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( } } } +*/ - - +/* def zipJoinLeftWithKeys[W: ClassManifest, Z: ClassManifest]( other: RDD[(K,W)])( f: (K, V, Option[W]) => Z, @@ -299,7 +413,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( } } - +*/ /** @@ -320,15 +434,13 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( assert(keysIter.hasNext() == false) val (oldValues, bs) = valuesIter.next() assert(valuesIter.hasNext() == false) - // Allocate the array to store the results into - val newValues: Array[V] = oldValues.clone().asInstanceOf[Array[V]] + // Allocate the array to store the results into val newBS = new BitSet(oldValues.size) // Populate the new Values for( (k,i) <- index ) { - if ( bs(i) && f( (k, oldValues(i)) ) ) { newBS(i) = true } - else { newValues(i) = null.asInstanceOf[V] } + newBS(i) = bs(i) && cleanF( (k, oldValues(i)) ) } - Array((newValues, newBS)).iterator + Array((oldValues, newBS)).iterator } new IndexedRDD[K,V](index, newValues) } @@ -371,7 +483,7 @@ object IndexedRDD { val groups = preAgg.mapPartitions( iter => { val indexMap = new BlockIndex[K]() - val values = new ArrayBuffer[V]() + val values = new ArrayBuffer[V] val bs = new BitSet for ((k,v) <- iter) { if(!indexMap.contains(k)) { @@ -384,7 +496,7 @@ object IndexedRDD { values(ind) = reduceFunc(values(ind), v) } } - List( (indexMap, (values.toArray, bs)) ).iterator + Iterator( (indexMap, (values.toSeq, bs)) ) }, true).cache // extract the index and the values val index = groups.mapPartitions(_.map{ case (kMap, vAr) => kMap }, true) @@ -468,7 +580,7 @@ object IndexedRDD { } // Use the index to build the new values table - val values = index.rdd.zipPartitions(partitioned)( (indexIter, tblIter) => { + val values: RDD[ (Seq[C], BitSet) ] = index.rdd.zipPartitions(partitioned)( (indexIter, tblIter) => { // There is only one map val index = indexIter.next() assert(!indexIter.hasNext()) @@ -487,7 +599,7 @@ object IndexedRDD { bs(ind) = true } } - List((values, bs)).iterator + Iterator((values, bs)) }) new IndexedRDD(index, values) } // end of apply @@ -521,7 +633,7 @@ object IndexedRDD { indexMap.put(k, ind) } } - List(indexMap).iterator + Iterator(indexMap) }, true).cache new RDDIndex(index) } diff --git a/core/src/main/scala/org/apache/spark/rdd/IndexedRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/IndexedRDDFunctions.scala index e497ef691f4ca..0310711d371e1 100644 --- a/core/src/main/scala/org/apache/spark/rdd/IndexedRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/IndexedRDDFunctions.scala @@ -37,48 +37,22 @@ class IndexedRDDFunctions[K: ClassManifest, V: ClassManifest](self: IndexedRDD[K def reindex(): IndexedRDD[K,V] = IndexedRDD(self) - /** - * Pass each value in the key-value pair RDD through a map function without changing the keys; - * this also retains the original RDD's partitioning. - */ - override def mapValues[U: ClassManifest](f: V => U): RDD[(K, U)] = { - val cleanF = self.index.rdd.context.clean(f) - val newValuesRDD = self.valuesRDD.mapPartitions(iter => iter.map{ - case (values, bs) => - val newValues = new Array[U](values.size) - for ( ind <- bs ) { - newValues(ind) = f(values(ind)) - } - (newValues, bs) - }, preservesPartitioning = true) - new IndexedRDD[K,U](self.index, newValuesRDD) - } - - - /** - * Pass each value in the key-value pair RDD through a map function without changing the keys; - * this also retains the original RDD's partitioning. - */ - override def mapValuesWithKeys[U: ClassManifest](f: (K, V) => U): RDD[(K, U)] = { - val cleanF = self.index.rdd.context.clean(f) - val newValues = self.index.rdd.zipPartitions(self.valuesRDD){ - (keysIter, valuesIter) => - val index = keysIter.next() - assert(keysIter.hasNext() == false) - val (oldValues, bs) = valuesIter.next() - assert(valuesIter.hasNext() == false) - // Allocate the array to store the results into - val newValues: Array[U] = new Array[U](oldValues.size) - // Populate the new Values - for( (k,i) <- index ) { - if (bs(i)) { newValues(i) = f(k, oldValues(i)) } - } - Array((newValues, bs)).iterator - } - new IndexedRDD[K,U](self.index, newValues) - } - - + // /** + // * Pass each value in the key-value pair RDD through a map function without changing the keys; + // * this also retains the original RDD's partitioning. + // */ + // override def mapValues[U: ClassManifest](f: V => U): RDD[(K, U)] = { + // val cleanF = self.index.rdd.context.clean(f) + // val newValuesRDD = self.valuesRDD.mapPartitions(iter => iter.map{ + // case (values, bs) => + // val newValues = new Array[U](values.size) + // for ( ind <- bs ) { + // newValues(ind) = f(values(ind)) + // } + // (newValues.toSeq, bs) + // }, preservesPartitioning = true) + // new IndexedRDD[K,U](self.index, newValuesRDD) + // } /** * Pass each value in the key-value pair RDD through a flatMap function without changing the @@ -97,7 +71,7 @@ class IndexedRDDFunctions[K: ClassManifest, V: ClassManifest](self: IndexedRDD[K newBS(ind) = true } } - (newValues, newBS) + (newValues.toSeq, newBS) }, preservesPartitioning = true) new IndexedRDD[K,U](self.index, newValuesRDD) } @@ -162,7 +136,7 @@ class IndexedRDDFunctions[K: ClassManifest, V: ClassManifest](self: IndexedRDD[K val b = if (otherBS(ind)) Seq(otherValues(ind)) else Seq.empty[W] newValues(ind) = (a, b) } - List((newValues, newBS)).iterator + Iterator((newValues.toSeq, newBS)) } new IndexedRDD(self.index, newValues) } @@ -225,7 +199,7 @@ class IndexedRDDFunctions[K: ClassManifest, V: ClassManifest](self: IndexedRDD[K newBS(ind) = true } } - List((newValues, newBS)).iterator + Iterator((newValues.toSeq, newBS)) }) new IndexedRDD(new RDDIndex(newIndex), newValues) } @@ -288,7 +262,7 @@ class IndexedRDDFunctions[K: ClassManifest, V: ClassManifest](self: IndexedRDD[K // case null => null // case (s, ab) => Seq((s, ab.toSeq)) // }.toSeq - List( (newIndex, (newValues.toArray, newBS)) ).iterator + Iterator( (newIndex, (newValues.toSeq, newBS)) ) }).cache() // Extract the index and values from the above RDD diff --git a/graph/src/main/scala/org/apache/spark/graph/EdgeTriplet.scala b/graph/src/main/scala/org/apache/spark/graph/EdgeTriplet.scala index c2ef63d1fd3d8..ef3aa199bdf41 100644 --- a/graph/src/main/scala/org/apache/spark/graph/EdgeTriplet.scala +++ b/graph/src/main/scala/org/apache/spark/graph/EdgeTriplet.scala @@ -28,10 +28,11 @@ class EdgeTriplet[VD, ED] extends Edge[ED] { /** * Set the edge properties of this triplet. */ - protected[spark] def set(other: Edge[ED]) { + protected[spark] def set(other: Edge[ED]): EdgeTriplet[VD,ED] = { srcId = other.srcId dstId = other.dstId attr = other.attr + this } /** diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index 87fb9dcd2e05f..ce1b9467c407d 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -5,6 +5,8 @@ import scala.collection.JavaConversions._ import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.ArrayBuilder +import scala.collection.mutable.BitSet + import org.apache.spark.SparkContext._ import org.apache.spark.Partitioner @@ -40,7 +42,6 @@ class EdgeTripletIterator[VD: ClassManifest, ED: ClassManifest]( et.dstId = edgePartition.dstIds(pos) // assert(vmap.containsKey(e.dst.id)) et.dstAttr = vertexArray(vidToIndex(et.dstId)) - //println("Iter called: " + pos) et.attr = edgePartition.data(pos) pos += 1 et @@ -63,32 +64,47 @@ class EdgeTripletIterator[VD: ClassManifest, ED: ClassManifest]( } } // end of Edge Triplet Iterator + + object EdgeTripletBuilder { def makeTriplets[VD: ClassManifest, ED: ClassManifest]( - vTableReplicationMap: IndexedRDD[Pid, VertexIdToIndexMap], + localVidMap: IndexedRDD[Pid, VertexIdToIndexMap], vTableReplicatedValues: IndexedRDD[Pid, Array[VD]], eTable: IndexedRDD[Pid, EdgePartition[ED]]): RDD[EdgeTriplet[VD, ED]] = { val iterFun = (iter: Iterator[(Pid, ((VertexIdToIndexMap, Array[VD]), EdgePartition[ED]))]) => { val (pid, ((vidToIndex, vertexArray), edgePartition)) = iter.next() - //assert(iter.hasNext == false) - // Return an iterator that looks up the hash map to find matching - // vertices for each edge. + assert(iter.hasNext == false) new EdgeTripletIterator(vidToIndex, vertexArray, edgePartition) } ClosureCleaner.clean(iterFun) - vTableReplicationMap.zipJoin(vTableReplicatedValues).zipJoinRDD(eTable) + localVidMap.zipJoin(vTableReplicatedValues).zipJoin(eTable) .mapPartitions( iterFun ) // end of map partition } - } +// { +// val iterFun = (iter: Iterator[(Pid, ((VertexIdToIndexMap, Array[VD]), EdgePartition[ED]))]) => { +// val (pid, ((vidToIndex, vertexArray), edgePartition)) = iter.next() +// assert(iter.hasNext == false) +// // Return an iterator that looks up the hash map to find matching +// // vertices for each edge. +// new EdgeTripletIterator(vidToIndex, vertexArray, edgePartition) +// } +// ClosureCleaner.clean(iterFun) +// localVidMap.zipJoin(vTableReplicatedValues).zipJoinRDD(eTable) +// .mapPartitions( iterFun ) // end of map partition +// } +// } + + /** * A Graph RDD that supports computation on graphs. */ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( @transient val vTable: IndexedRDD[Vid, VD], @transient val vid2pid: IndexedRDD[Vid, Array[Pid]], + @transient val localVidMap: IndexedRDD[Pid, VertexIdToIndexMap], @transient val eTable: IndexedRDD[Pid, EdgePartition[ED]]) extends Graph[VD, ED] { @@ -96,7 +112,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( /** - * (vTableReplicationMap: IndexedRDD[Pid, VertexIdToIndexMap]) is a version of the + * (localVidMap: IndexedRDD[Pid, VertexIdToIndexMap]) is a version of the * vertex data after it is replicated. Within each partition, it holds a map * from vertex ID to the index where that vertex's attribute is stored. This * index refers to an array in the same partition in vTableReplicatedValues. @@ -104,8 +120,8 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( * (vTableReplicatedValues: IndexedRDD[Pid, Array[VD]]) holds the vertex data * and is arranged as described above. */ - @transient val (vTableReplicationMap, vTableReplicatedValues) = - createVTableReplicated(vTable, vid2pid, eTable) + @transient val vTableReplicatedValues = + createVTableReplicated(vTable, vid2pid, localVidMap) /** Return a RDD of vertices. */ @@ -119,8 +135,8 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( /** Return a RDD that brings edges with its source and destination vertices together. */ - @transient override val triplets: RDD[EdgeTriplet[VD, ED]] = - EdgeTripletBuilder.makeTriplets(vTableReplicationMap, vTableReplicatedValues, eTable) + @transient override val triplets: RDD[EdgeTriplet[VD, ED]] = + EdgeTripletBuilder.makeTriplets(localVidMap, vTableReplicatedValues, eTable) // { @@ -164,26 +180,26 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( override def reverse: Graph[VD, ED] = { val etable = eTable.mapValues( _.reverse ).asInstanceOf[IndexedRDD[Pid, EdgePartition[ED]]] - new GraphImpl(vTable, vid2pid, etable) + new GraphImpl(vTable, vid2pid, localVidMap, etable) } override def mapVertices[VD2: ClassManifest](f: (Vid, VD) => VD2): Graph[VD2, ED] = { val newVTable = vTable.mapValuesWithKeys((vid, data) => f(vid, data)) .asInstanceOf[IndexedRDD[Vid, VD2]] - new GraphImpl(newVTable, vid2pid, eTable) + new GraphImpl(newVTable, vid2pid, localVidMap, eTable) } override def mapEdges[ED2: ClassManifest](f: Edge[ED] => ED2): Graph[VD, ED2] = { val newETable = eTable.mapValues(eBlock => eBlock.map(f)) .asInstanceOf[IndexedRDD[Pid, EdgePartition[ED2]]] - new GraphImpl(vTable, vid2pid, newETable) + new GraphImpl(vTable, vid2pid, localVidMap, newETable) } override def mapTriplets[ED2: ClassManifest](f: EdgeTriplet[VD, ED] => ED2): Graph[VD, ED2] = { - val newETable = eTable.zipJoin(vTableReplicationMap).zipJoin(vTableReplicatedValues).mapValues{ + val newETable = eTable.zipJoin(localVidMap).zipJoin(vTableReplicatedValues).mapValues{ case ((edgePartition, vidToIndex), vertexArray) => val et = new EdgeTriplet[VD, ED] edgePartition.map{e => @@ -193,7 +209,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( f(et) } }.asInstanceOf[IndexedRDD[Pid, EdgePartition[ED2]]] - new GraphImpl(vTable, vid2pid, newETable) + new GraphImpl(vTable, vid2pid, localVidMap, newETable) } // override def correctEdges(): Graph[VD, ED] = { @@ -239,8 +255,9 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( // behaves deterministically. // @todo reindex the vertex and edge tables val newVid2Pid = createVid2Pid(newETable, newVTable.index) + val newVidMap = createLocalVidMap(newETable) - new GraphImpl(newVTable, newVid2Pid, newETable) + new GraphImpl(newVTable, newVid2Pid, localVidMap, newETable) } @@ -276,9 +293,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( // Because all ETs with the same src and dst will live on the same // partition due to the EdgePartitioner, this guarantees that these // ET groups will be complete. - .groupBy { t: EdgeTriplet[VD, ED] => - //println("(" + t.src.id + ", " + t.dst.id + ", " + t.data + ")") - (t.srcId, t.dstId) } + .groupBy { t: EdgeTriplet[VD, ED] => (t.srcId, t.dstId) } //.groupBy { e => (e.src, e.dst) } // Apply the user supplied supplied edge group function to // each group of edges @@ -308,7 +323,8 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( val newETable = createETable(newEdges, eTable.index.partitioner.numPartitions) - new GraphImpl(vTable, vid2pid, newETable) + + new GraphImpl(vTable, vid2pid, localVidMap, newETable) } @@ -318,9 +334,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( val newEdges: RDD[Edge[ED2]] = edges.mapPartitions { partIter => partIter.toList - .groupBy { e: Edge[ED] => - println(e.srcId + " " + e.dstId) - (e.srcId, e.dstId) } + .groupBy { e: Edge[ED] => (e.srcId, e.dstId) } .mapValues { ts => f(ts.toIterator) } .toList .toIterator @@ -330,7 +344,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( val newETable = createETable(newEdges, eTable.index.partitioner.numPartitions) - new GraphImpl(vTable, vid2pid, newETable) + new GraphImpl(vTable, vid2pid, localVidMap, newETable) } @@ -348,26 +362,45 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( ClosureCleaner.clean(reduceFunc) // Map and preaggregate - val preAgg = vTableReplicationMap.zipJoin(vTableReplicatedValues).zipJoinRDD(eTable).flatMap{ + val preAgg = localVidMap.zipJoin(vTableReplicatedValues).zipJoin(eTable).flatMap{ case (pid, ((vidToIndex, vertexArray), edgePartition)) => - val aggMap = new VertexHashMap[A] + // We can reuse the vidToIndex map for aggregation here as well. + /** @todo Since this has the downside of not allowing "messages" to arbitrary + * vertices we should consider just using a fresh map. + */ + val msgArray = new Array[A](vertexArray.size) + val msgBS = new BitSet(vertexArray.size) + // Iterate over the partition val et = new EdgeTriplet[VD, ED] edgePartition.foreach{e => et.set(e) et.srcAttr = vertexArray(vidToIndex(e.srcId)) et.dstAttr = vertexArray(vidToIndex(e.dstId)) - mapFunc(et).foreach{case (vid, a) => - if(aggMap.containsKey(vid)) { - aggMap.put(vid, reduceFunc(aggMap.get(vid), a)) - } else { aggMap.put(vid, a) } + mapFunc(et).foreach{ case (vid, msg) => + // verify that the vid is valid + assert(vid == et.srcId || vid == et.dstId) + val ind = vidToIndex(vid) + // Populate the aggregator map + if(msgBS(ind)) { + msgArray(ind) = reduceFunc(msgArray(ind), msg) + } else { + msgArray(ind) = msg + msgBS(ind) = true + } } } // Return the aggregate map - aggMap.long2ObjectEntrySet().fastIterator().map{ - entry => (entry.getLongKey(), entry.getValue()) + vidToIndex.long2IntEntrySet().fastIterator() + // Remove the entries that did not receive a message + .filter{ entry => msgBS(entry.getValue()) } + // Construct the actual pairs + .map{ entry => + val vid = entry.getLongKey() + val ind = entry.getValue() + val msg = msgArray(ind) + (vid, msg) } }.partitionBy(vTable.index.rdd.partitioner.get) - // do the final reduction reusing the index map IndexedRDD(preAgg, vTable.index, reduceFunc) } @@ -377,8 +410,9 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( (updates: RDD[(Vid, U)])(updateF: (Vid, VD, Option[U]) => VD2) : Graph[VD2, ED] = { ClosureCleaner.clean(updateF) - val newVTable = vTable.zipJoinLeftWithKeys(updates)(updateF) - new GraphImpl(newVTable, vid2pid, eTable) + val newVTable = vTable.leftJoin(updates).mapValuesWithKeys( + (vid, vu) => updateF(vid, vu._1, vu._2) ) + new GraphImpl(newVTable, vid2pid, localVidMap, eTable) } @@ -417,8 +451,8 @@ object GraphImpl { val vtable = vertices.indexed(numVPart) val etable = createETable(edges, numEPart) val vid2pid = createVid2Pid(etable, vtable.index) - - new GraphImpl(vtable, vid2pid, etable) + val localVidMap = createLocalVidMap(etable) + new GraphImpl(vtable, vid2pid, localVidMap, etable) } @@ -476,40 +510,77 @@ object GraphImpl { } - protected def createVTableReplicated[VD: ClassManifest, ED: ClassManifest]( - vTable: IndexedRDD[Vid, VD], vid2pid: IndexedRDD[Vid, Array[Pid]], - eTable: IndexedRDD[Pid, EdgePartition[ED]]): - (IndexedRDD[Pid, VertexIdToIndexMap], IndexedRDD[Pid, Array[VD]]) = { + protected def createLocalVidMap[ED: ClassManifest]( + eTable: IndexedRDD[Pid, EdgePartition[ED]]): IndexedRDD[Pid, VertexIdToIndexMap] = { + eTable.mapValues{ epart => + val vidToIndex = new VertexIdToIndexMap() + var i = 0 + epart.foreach{ e => + if(!vidToIndex.contains(e.srcId)) { + vidToIndex.put(e.srcId, i) + i += 1 + } + if(!vidToIndex.contains(e.dstId)) { + vidToIndex.put(e.dstId, i) + i += 1 + } + } + vidToIndex + } + } + + + protected def createVTableReplicated[VD: ClassManifest]( + vTable: IndexedRDD[Vid, VD], + vid2pid: IndexedRDD[Vid, Array[Pid]], + replicationMap: IndexedRDD[Pid, VertexIdToIndexMap]): + IndexedRDD[Pid, Array[VD]] = { // Join vid2pid and vTable, generate a shuffle dependency on the joined // result, and get the shuffle id so we can use it on the slave. - val msgsByPartition = - vTable.zipJoinRDD(vid2pid) - .flatMap { case (vid, (vdata, pids)) => - pids.iterator.map { pid => MessageToPartition(pid, (vid, vdata)) } - } - .partitionBy(eTable.partitioner.get).cache() - // @todo assert edge table has partitioner + val msgsByPartition = vTable.zipJoin(vid2pid) + .flatMap { case (vid, (vdata, pids)) => + pids.iterator.map { pid => MessageToPartition(pid, (vid, vdata)) } + } + .partitionBy(replicationMap.partitioner.get).cache() + + val newValuesRDD = replicationMap.valuesRDD.zipPartitions(msgsByPartition){ + (mapIter, msgsIter) => + val (Seq(vidToIndex), bs) = mapIter.next() + assert(!mapIter.hasNext) + // Populate the vertex array using the vidToIndex map + val vertexArray = new Array[VD](vidToIndex.size) + for (msg <- msgsIter) { + val ind = vidToIndex(msg.data._1) + vertexArray(ind) = msg.data._2 + } + Iterator((Seq(vertexArray), bs)) + } - val vTableReplicationMap: IndexedRDD[Pid, VertexIdToIndexMap] = - msgsByPartition.mapPartitionsWithIndex( (pid, iter) => { - val vidToIndex = new VertexIdToIndexMap - var i = 0 - for (msg <- iter) { - vidToIndex.put(msg.data._1, i) - } - Array((pid, vidToIndex)).iterator - }, preservesPartitioning = true).indexed(eTable.index) - - val vTableReplicatedValues: IndexedRDD[Pid, Array[VD]] = - msgsByPartition.mapPartitionsWithIndex( (pid, iter) => { - val vertexArray = ArrayBuilder.make[VD] - for (msg <- iter) { - vertexArray += msg.data._2 - } - Array((pid, vertexArray.result)).iterator - }, preservesPartitioning = true).indexed(eTable.index) + new IndexedRDD(replicationMap.index, newValuesRDD) + + // @todo assert edge table has partitioner - (vTableReplicationMap, vTableReplicatedValues) + // val localVidMap: IndexedRDD[Pid, VertexIdToIndexMap] = + // msgsByPartition.mapPartitionsWithIndex( (pid, iter) => { + // val vidToIndex = new VertexIdToIndexMap + // var i = 0 + // for (msg <- iter) { + // vidToIndex.put(msg.data._1, i) + // i += 1 + // } + // Array((pid, vidToIndex)).iterator + // }, preservesPartitioning = true).indexed(eTable.index) + + // val vTableReplicatedValues: IndexedRDD[Pid, Array[VD]] = + // msgsByPartition.mapPartitionsWithIndex( (pid, iter) => { + // val vertexArray = ArrayBuilder.make[VD] + // for (msg <- iter) { + // vertexArray += msg.data._2 + // } + // Array((pid, vertexArray.result)).iterator + // }, preservesPartitioning = true).indexed(eTable.index) + + // (localVidMap, vTableReplicatedValues) } diff --git a/graph/src/main/scala/org/apache/spark/graph/package.scala b/graph/src/main/scala/org/apache/spark/graph/package.scala index 47d5acb9e76ba..4627c3566ca19 100644 --- a/graph/src/main/scala/org/apache/spark/graph/package.scala +++ b/graph/src/main/scala/org/apache/spark/graph/package.scala @@ -9,7 +9,7 @@ package object graph { type VertexSet = it.unimi.dsi.fastutil.longs.LongOpenHashSet type VertexArrayList = it.unimi.dsi.fastutil.longs.LongArrayList // @todo replace with rxin's fast hashmap - type VertexIdToIndexMap = scala.collection.mutable.HashMap[Vid, Int] + type VertexIdToIndexMap = it.unimi.dsi.fastutil.longs.Long2IntOpenHashMap /** * Return the default null-like value for a data type T. From 3f3d28c73f3771f6bbfd8096609629a53dc64987 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Thu, 17 Oct 2013 19:55:36 -0700 Subject: [PATCH 125/531] Switching from Seq to IndexedSeq --- .../org/apache/spark/rdd/IndexedRDD.scala | 34 +++++++++++-------- .../spark/rdd/IndexedRDDFunctions.scala | 17 +++++----- .../apache/spark/graph/impl/GraphImpl.scala | 4 +-- 3 files changed, 30 insertions(+), 25 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/IndexedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/IndexedRDD.scala index 2f1f907c6c746..5f95559f15122 100644 --- a/core/src/main/scala/org/apache/spark/rdd/IndexedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/IndexedRDD.scala @@ -24,7 +24,6 @@ import java.util.{HashMap => JHashMap, BitSet => JBitSet, HashSet => JHashSet} import scala.collection.JavaConversions._ import scala.collection.mutable.ArrayBuffer - import scala.collection.mutable.BitSet @@ -72,7 +71,7 @@ class RDDIndex[@specialized K: ClassManifest](private[spark] val rdd: RDD[BlockI */ class IndexedRDD[K: ClassManifest, V: ClassManifest]( @transient val index: RDDIndex[K], - @transient val valuesRDD: RDD[ (Seq[V], BitSet) ]) + @transient val valuesRDD: RDD[ (IndexedSeq[V], BitSet) ]) extends RDD[(K, V)](index.rdd.context, List(new OneToOneDependency(index.rdd), new OneToOneDependency(valuesRDD)) ) { @@ -119,13 +118,14 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( */ def mapValues[U: ClassManifest](f: V => U): IndexedRDD[K, U] = { val cleanF = index.rdd.context.clean(f) - val newValuesRDD = valuesRDD.mapPartitions(iter => iter.map{ + val newValuesRDD: RDD[ (IndexedSeq[U], BitSet) ] = + valuesRDD.mapPartitions(iter => iter.map{ case (values, bs) => val newValues = new Array[U](values.size) for ( ind <- bs ) { newValues(ind) = f(values(ind)) } - (newValues.toSeq, bs) + (newValues.toIndexedSeq, bs) }, preservesPartitioning = true) new IndexedRDD[K,U](index, newValuesRDD) } @@ -137,7 +137,8 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( */ def mapValuesWithKeys[U: ClassManifest](f: (K, V) => U): IndexedRDD[K, U] = { val cleanF = index.rdd.context.clean(f) - val newValues = index.rdd.zipPartitions(valuesRDD){ + val newValues: RDD[ (IndexedSeq[U], BitSet) ] = + index.rdd.zipPartitions(valuesRDD){ (keysIter, valuesIter) => val index = keysIter.next() assert(keysIter.hasNext() == false) @@ -149,7 +150,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( for( (k,i) <- index ) { if (bs(i)) { newValues(i) = f(k, oldValues(i)) } } - Array((newValues.toSeq, bs)).iterator + Array((newValues.toIndexedSeq, bs)).iterator } new IndexedRDD[K,U](index, newValues) } @@ -159,7 +160,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( if(index != other.index) { throw new SparkException("A zipJoin can only be applied to RDDs with the same index!") } - val newValuesRDD: RDD[ (Seq[(V,W)], BitSet) ] = valuesRDD.zipPartitions(other.valuesRDD){ + val newValuesRDD: RDD[ (IndexedSeq[(V,W)], BitSet) ] = valuesRDD.zipPartitions(other.valuesRDD){ (thisIter, otherIter) => val (thisValues, thisBS) = thisIter.next() assert(!thisIter.hasNext) @@ -167,7 +168,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( assert(!otherIter.hasNext) val newBS = thisBS & otherBS val newValues = thisValues.view.zip(otherValues) - Iterator((newValues, newBS)) + Iterator((newValues.toIndexedSeq, newBS)) } new IndexedRDD(index, newValuesRDD) } @@ -177,7 +178,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( if(index != other.index) { throw new SparkException("A zipJoin can only be applied to RDDs with the same index!") } - val newValuesRDD: RDD[ (Seq[(V,Option[W])], BitSet) ] = valuesRDD.zipPartitions(other.valuesRDD){ + val newValuesRDD: RDD[ (IndexedSeq[(V,Option[W])], BitSet) ] = valuesRDD.zipPartitions(other.valuesRDD){ (thisIter, otherIter) => val (thisValues, thisBS) = thisIter.next() assert(!thisIter.hasNext) @@ -186,7 +187,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( val otherOption = otherValues.view.zipWithIndex .map{ (x: (W, Int)) => if(otherBS(x._2)) Option(x._1) else None } val newValues = thisValues.view.zip(otherOption) - Iterator((newValues, thisBS)) + Iterator((newValues.toIndexedSeq, thisBS)) } new IndexedRDD(index, newValuesRDD) } @@ -197,6 +198,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( other: RDD[(K,W)], merge: (W,W) => W = (a:W, b:W) => a): IndexedRDD[K, (V, Option[W]) ] = { val cleanMerge = index.rdd.context.clean(merge) + other match { case other: IndexedRDD[_, _] if index == other.index => { leftZipJoin(other) @@ -211,7 +213,8 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( val otherShuffled = if (other.partitioner == Some(partitioner)) other else other.partitionBy(partitioner) - val newValues = index.rdd.zipPartitions(valuesRDD, other) { + val newValues: RDD[ (IndexedSeq[(V,Option[W])], BitSet) ] = + index.rdd.zipPartitions(valuesRDD, other) { (thisIndexIter, thisIter, tuplesIter) => val index = thisIndexIter.next() assert(!thisIndexIter.hasNext) @@ -236,7 +239,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( .map{ (x: (W, Int)) => if(wBS(x._2)) Option(x._1) else None } val newValues = thisValues.view.zip(otherOption) - Iterator((newValues.toSeq, thisBS)) + Iterator((newValues.toIndexedSeq, thisBS)) } // end of newValues new IndexedRDD(index, newValues) } @@ -496,11 +499,12 @@ object IndexedRDD { values(ind) = reduceFunc(values(ind), v) } } - Iterator( (indexMap, (values.toSeq, bs)) ) + Iterator( (indexMap, (values.toIndexedSeq, bs)) ) }, true).cache // extract the index and the values val index = groups.mapPartitions(_.map{ case (kMap, vAr) => kMap }, true) - val values = groups.mapPartitions(_.map{ case (kMap,vAr) => vAr }, true) + val values: RDD[(IndexedSeq[V], BitSet)] = + groups.mapPartitions(_.map{ case (kMap,vAr) => vAr }, true) new IndexedRDD[K,V](new RDDIndex(index), values) } @@ -580,7 +584,7 @@ object IndexedRDD { } // Use the index to build the new values table - val values: RDD[ (Seq[C], BitSet) ] = index.rdd.zipPartitions(partitioned)( (indexIter, tblIter) => { + val values: RDD[ (IndexedSeq[C], BitSet) ] = index.rdd.zipPartitions(partitioned)( (indexIter, tblIter) => { // There is only one map val index = indexIter.next() assert(!indexIter.hasNext()) diff --git a/core/src/main/scala/org/apache/spark/rdd/IndexedRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/IndexedRDDFunctions.scala index 0310711d371e1..fd7c16089d69e 100644 --- a/core/src/main/scala/org/apache/spark/rdd/IndexedRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/IndexedRDDFunctions.scala @@ -60,7 +60,7 @@ class IndexedRDDFunctions[K: ClassManifest, V: ClassManifest](self: IndexedRDD[K */ override def flatMapValues[U: ClassManifest](f: V => TraversableOnce[U]): RDD[(K,U)] = { val cleanF = self.index.rdd.context.clean(f) - val newValuesRDD = self.valuesRDD.mapPartitions(iter => iter.map{ + val newValuesRDD: RDD[(IndexedSeq[U], BitSet)] = self.valuesRDD.mapPartitions(iter => iter.map{ case (values, bs) => val newValues = new Array[U](values.size) val newBS = new BitSet(values.size) @@ -71,7 +71,7 @@ class IndexedRDDFunctions[K: ClassManifest, V: ClassManifest](self: IndexedRDD[K newBS(ind) = true } } - (newValues.toSeq, newBS) + (newValues.toIndexedSeq, newBS) }, preservesPartitioning = true) new IndexedRDD[K,U](self.index, newValuesRDD) } @@ -120,7 +120,7 @@ class IndexedRDDFunctions[K: ClassManifest, V: ClassManifest](self: IndexedRDD[K // then we simply merge the value RDDs. // However it is possible that both RDDs are missing a value for a given key in // which case the returned RDD should have a null value - val newValues = + val newValues: RDD[(IndexedSeq[(Seq[V], Seq[W])], BitSet)] = self.valuesRDD.zipPartitions(other.valuesRDD){ (thisIter, otherIter) => val (thisValues, thisBS) = thisIter.next() @@ -136,7 +136,7 @@ class IndexedRDDFunctions[K: ClassManifest, V: ClassManifest](self: IndexedRDD[K val b = if (otherBS(ind)) Seq(otherValues(ind)) else Seq.empty[W] newValues(ind) = (a, b) } - Iterator((newValues.toSeq, newBS)) + Iterator((newValues.toIndexedSeq, newBS)) } new IndexedRDD(self.index, newValues) } @@ -166,7 +166,7 @@ class IndexedRDDFunctions[K: ClassManifest, V: ClassManifest](self: IndexedRDD[K List(newIndex).iterator }).cache() // Use the new index along with the this and the other indices to merge the values - val newValues = + val newValues: RDD[(IndexedSeq[(Seq[V], Seq[W])], BitSet)] = newIndex.zipPartitions(self.tuples, other.tuples)( (newIndexIter, thisTuplesIter, otherTuplesIter) => { // Get the new index for this partition @@ -199,7 +199,7 @@ class IndexedRDDFunctions[K: ClassManifest, V: ClassManifest](self: IndexedRDD[K newBS(ind) = true } } - Iterator((newValues.toSeq, newBS)) + Iterator((newValues.toIndexedSeq, newBS)) }) new IndexedRDD(new RDDIndex(newIndex), newValues) } @@ -262,12 +262,13 @@ class IndexedRDDFunctions[K: ClassManifest, V: ClassManifest](self: IndexedRDD[K // case null => null // case (s, ab) => Seq((s, ab.toSeq)) // }.toSeq - Iterator( (newIndex, (newValues.toSeq, newBS)) ) + Iterator( (newIndex, (newValues.toIndexedSeq, newBS)) ) }).cache() // Extract the index and values from the above RDD val newIndex = groups.mapPartitions(_.map{ case (kMap,vAr) => kMap }, true) - val newValues = groups.mapPartitions(_.map{ case (kMap,vAr) => vAr }, true) + val newValues: RDD[(IndexedSeq[(Seq[V], Seq[W])], BitSet)] = + groups.mapPartitions(_.map{ case (kMap,vAr) => vAr }, true) new IndexedRDD[K, (Seq[V], Seq[W])](new RDDIndex(newIndex), newValues) } diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index ce1b9467c407d..413177b2dae63 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -545,7 +545,7 @@ object GraphImpl { val newValuesRDD = replicationMap.valuesRDD.zipPartitions(msgsByPartition){ (mapIter, msgsIter) => - val (Seq(vidToIndex), bs) = mapIter.next() + val (IndexedSeq(vidToIndex), bs) = mapIter.next() assert(!mapIter.hasNext) // Populate the vertex array using the vidToIndex map val vertexArray = new Array[VD](vidToIndex.size) @@ -553,7 +553,7 @@ object GraphImpl { val ind = vidToIndex(msg.data._1) vertexArray(ind) = msg.data._2 } - Iterator((Seq(vertexArray), bs)) + Iterator((IndexedSeq(vertexArray), bs)) } new IndexedRDD(replicationMap.index, newValuesRDD) From 3a40a5eb30eeeab20c48396d77172519b81476f2 Mon Sep 17 00:00:00 2001 From: Dan Crankshaw Date: Fri, 18 Oct 2013 15:11:21 -0700 Subject: [PATCH 126/531] Added some documentation. --- .../scala/org/apache/spark/graph/Graph.scala | 33 ++++++++++- .../org/apache/spark/graph/GraphLab.scala | 2 +- .../org/apache/spark/graph/GraphLoader.scala | 19 +++++-- .../org/apache/spark/graph/GraphOps.scala | 5 -- .../scala/org/apache/spark/graph/Pregel.scala | 29 ++++++++++ .../apache/spark/graph/impl/GraphImpl.scala | 55 ++----------------- .../spark/graph/util/GraphGenerators.scala | 8 +-- 7 files changed, 83 insertions(+), 68 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/Graph.scala b/graph/src/main/scala/org/apache/spark/graph/Graph.scala index 50a44e51e5d2d..342151173a636 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Graph.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Graph.scala @@ -184,13 +184,42 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { /** - * @todo document function + * groupEdgeTriplets is used to merge multiple edges that have the + * same source and destination vertex into a single edge. The user + * supplied function is applied to each directed pair of vertices (u, v) and + * has access to all EdgeTriplets + * + * {e: for all e in E where e.src = u and e.dst = v} + * + * This function is identical to [[org.apache.spark.graph.Graph.groupEdges]] + * except that this function + * provides the user-supplied function with an iterator over EdgeTriplets, + * which contain the vertex data, whereas groupEdges provides the user-supplied + * function with an iterator over Edges, which only contain the vertex IDs. + * + * @tparam ED2 the type of the resulting edge data after grouping + * + * @param f the user supplied function to merge multiple EdgeTriplets + * into a single ED2 object + * + * @return Graph[VD,ED2] The resulting graph with a single Edge for each + * source, dest vertex pair. + * */ def groupEdgeTriplets[ED2: ClassManifest](f: Iterator[EdgeTriplet[VD,ED]] => ED2 ): Graph[VD,ED2] /** - * @todo document function + * This function merges multiple edges between two vertices into a single + * Edge. See [[org.apache.spark.graph.Graph.groupEdgeTriplets]] for more detail. + * + * @tparam ED2 the type of the resulting edge data after grouping. + * + * @param f the user supplied function to merge multiple Edges + * into a single ED2 object. + * + * @return Graph[VD,ED2] The resulting graph with a single Edge for each + * source, dest vertex pair. */ def groupEdges[ED2: ClassManifest](f: Iterator[Edge[ED]] => ED2 ): Graph[VD,ED2] diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala b/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala index 8ba708ba3290b..2f2a624592de1 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala @@ -4,7 +4,7 @@ import scala.collection.JavaConversions._ import org.apache.spark.rdd.RDD /** - * This object implement the graphlab gather-apply-scatter api. + * This object implements the GraphLab gather-apply-scatter api. */ object GraphLab { diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala b/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala index 052f9acdeb6b3..76f69edf0e22f 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala @@ -9,7 +9,18 @@ import org.apache.spark.graph.impl.GraphImpl object GraphLoader { /** - * Load an edge list from file initializing the Graph RDD + * Load an edge list from file initializing the Graph + * + * @tparam ED the type of the edge data of the resulting Graph + * + * @param sc the SparkContext used to construct RDDs + * @param path the path to the text file containing the edge list + * @param edgeParser a function that takes an array of strings and + * returns an ED object + * @param minEdgePartitions the number of partitions for the + * the Edge RDD + * + * @todo remove minVertexPartitions arg */ def textFile[ED: ClassManifest]( sc: SparkContext, @@ -38,14 +49,10 @@ object GraphLoader { }.cache() val graph = fromEdges(edges) - // println("Loaded graph:" + - // "\n\t#edges: " + graph.numEdges + - // "\n\t#vertices: " + graph.numVertices) - graph } - def fromEdges[ED: ClassManifest](edges: RDD[Edge[ED]]): GraphImpl[Int, ED] = { + private def fromEdges[ED: ClassManifest](edges: RDD[Edge[ED]]): GraphImpl[Int, ED] = { val vertices = edges.flatMap { edge => List((edge.srcId, 1), (edge.dstId, 1)) } .reduceByKey(_ + _) .map{ case (vid, degree) => (vid, degree) } diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala b/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala index 92198a4995c8e..5e8f082fdad8a 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala @@ -94,11 +94,6 @@ class GraphOps[VD: ClassManifest, ED: ClassManifest](graph: Graph[VD, ED]) { } // end of aggregateNeighbors - - - - - def collectNeighborIds(edgeDirection: EdgeDirection) : RDD[(Vid, Array[Vid])] = { val nbrs = graph.aggregateNeighbors[Array[Vid]]( (vid, edge) => Some(Array(edge.otherVertexId(vid))), diff --git a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala index 065d196ff651d..7ad6fda2a4570 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala @@ -3,8 +3,37 @@ package org.apache.spark.graph import org.apache.spark.rdd.RDD +/** + * This object implements the Pregel bulk-synchronous + * message-passing API. + */ object Pregel { + + /** + * Execute the Pregel program. + * + * @tparam VD the vertex data type + * @tparam ED the edge data type + * @tparam A the Pregel message type + * + * @param vprog a user supplied function that acts as the vertex program for + * the Pregel computation. It takes the vertex ID of the vertex it is running on, + * the accompanying data for that vertex, and the incoming data and returns the + * new vertex value. + * @param sendMsg a user supplied function that takes the current vertex ID and an EdgeTriplet + * between the vertex and one of its neighbors and produces a message to send + * to that neighbor. + * @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.'' + * @param initialMsg the message each vertex will receive at the beginning of the + * first iteration. + * @param numIter the number of iterations to run this computation for. + * + * @return the resulting graph at the end of the computation + * + */ def iterate[VD: ClassManifest, ED: ClassManifest, A: ClassManifest](graph: Graph[VD, ED])( vprog: (Vid, VD, A) => VD, sendMsg: (Vid, EdgeTriplet[VD, ED]) => Option[A], diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index 413177b2dae63..a6604b978f996 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -261,71 +261,27 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( } - // Because of the edgepartitioner, we know that all edges with the same src and dst - // will be in the same partition - - // We will want to keep the same partitioning scheme. Use newGraph() rather than - // new GraphImpl() - // TODO(crankshaw) is there a better way to do this using RDD.groupBy() - // functions? - override def groupEdgeTriplets[ED2: ClassManifest]( f: Iterator[EdgeTriplet[VD,ED]] => ED2 ): Graph[VD,ED2] = { - //override def groupEdges[ED2: ClassManifest](f: Iterator[Edge[ED]] => ED2 ): - - // I think that - // myRDD.mapPartitions { part => - // val (vmap, edges) = part.next() - // gives me access to the vertex map and the set of - // edges within that partition - - // This is what happens during mapPartitions - // The iterator iterates over all partitions - // val result: RDD[U] = new RDD[T]().mapPartitions(f: Iterator[T] => Iterator[U]) - - // TODO(crankshaw) figure out how to actually get the new Edge RDD and what - // type that should have val newEdges: RDD[Edge[ED2]] = triplets.mapPartitions { partIter => - // toList lets us operate on all EdgeTriplets in a single partition at once partIter + // TODO(crankshaw) toList requires that the entire edge partition + // can fit in memory right now. .toList // groups all ETs in this partition that have the same src and dst // Because all ETs with the same src and dst will live on the same // partition due to the EdgePartitioner, this guarantees that these // ET groups will be complete. .groupBy { t: EdgeTriplet[VD, ED] => (t.srcId, t.dstId) } - //.groupBy { e => (e.src, e.dst) } - // Apply the user supplied supplied edge group function to - // each group of edges - // The result of this line is Map[(Long, Long, ED2] .mapValues { ts: List[EdgeTriplet[VD, ED]] => f(ts.toIterator) } - // convert the resulting map back to a list of tuples .toList - // TODO(crankshaw) needs an iterator over the tuples? - // Why can't I map over the list? - .toIterator - // map over those tuples that contain src and dst info plus the - // new edge data to make my new edges .map { case ((src, dst), data) => Edge(src, dst, data) } - - // How do I convert from a scala map to a list? - // I want to be able to apply a function like: - // f: (key, value): (K, V) => result: [R] - // so that I can transfrom a Map[K, V] to List[R] - - // Maybe look at collections.breakOut - // see http://stackoverflow.com/questions/1715681/scala-2-8-breakout - // and http://stackoverflow.com/questions/6998676/converting-a-scala-map-to-a-list - } - // @todo eliminate the need to call createETable + //TODO(crankshaw) eliminate the need to call createETable val newETable = createETable(newEdges, eTable.index.partitioner.numPartitions) - - new GraphImpl(vTable, vid2pid, localVidMap, newETable) - } @@ -340,7 +296,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( .toIterator .map { case ((src, dst), data) => Edge(src, dst, data) } } - // @todo eliminate the need to call createETable + // TODO(crankshaw) eliminate the need to call createETable val newETable = createETable(newEdges, eTable.index.partitioner.numPartitions) @@ -654,7 +610,8 @@ object GraphImpl { /** - * @todo(crankshaw) how does this effect load balancing? + * @todo This will only partition edges to the upper diagonal + * of the 2D processor space. */ protected def canonicalEdgePartitionFunction2D(srcOrig: Vid, dstOrig: Vid, numParts: Pid, ceilSqrtNumParts: Pid): Pid = { diff --git a/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala b/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala index d75a678b26949..061cce99b6e3d 100644 --- a/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala +++ b/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala @@ -16,9 +16,9 @@ import org.apache.spark.graph.Graph import org.apache.spark.graph.Edge import org.apache.spark.graph.impl.GraphImpl - -// TODO(crankshaw) I might want to pull at least RMAT out into a separate class. -// Might simplify the code to have classwide variables and such. +/** + * @todo(crankshaw) cleanup and modularize code + */ object GraphGenerators { val RMATa = 0.45 @@ -236,8 +236,6 @@ object GraphGenerators { } } - - } From 971f8240147b309f98edb5904657bf3bbd018dd8 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 18 Oct 2013 16:03:13 -0700 Subject: [PATCH 127/531] Revert unnecessary changes to core While benchmarking, we accidentally committed some unnecessary changes to core such as adding logging. These changes make it more difficult to merge from Spark upstream, so this commit reverts them. --- .../src/main/scala/org/apache/spark/SparkEnv.scala | 1 - .../org/apache/spark/rdd/PairRDDFunctions.scala | 6 ++---- .../apache/spark/serializer/KryoSerializer.scala | 14 +++++++++----- .../spark/serializer/SerializerManager.scala | 2 +- 4 files changed, 12 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 9b8384bcbb58c..29968c273c31d 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -156,7 +156,6 @@ object SparkEnv extends Logging { val serializer = serializerManager.setDefault( System.getProperty("spark.serializer", "org.apache.spark.serializer.JavaSerializer")) - logInfo("spark.serializer is " + System.getProperty("spark.serializer")) val closureSerializer = serializerManager.get( System.getProperty("spark.closure.serializer", "org.apache.spark.serializer.JavaSerializer")) diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 8a66297f6fcd9..81bf867188ac4 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -245,7 +245,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) if (getKeyClass().isArray && partitioner.isInstanceOf[HashPartitioner]) { throw new SparkException("Default partitioner cannot partition array keys.") } - new ShuffledRDD[K, V, (K, V)](self, partitioner) + new ShuffledRDD[K, V, (K, V)](self, partitioner) } /** @@ -265,9 +265,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) * pair (k, (v, None)) if no elements in `other` have key k. Uses the given Partitioner to * partition the output RDD. */ - - def leftOuterJoin[W: ClassManifest](other: RDD[(K, W)], partitioner: Partitioner): - RDD[(K, (V, Option[W]))] = { + def leftOuterJoin[W: ClassManifest](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, Option[W]))] = { this.cogroup(other, partitioner).flatMapValues { case (vs, ws) => if (ws.isEmpty) { vs.iterator.map(v => (v, None)) diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index 263ff59ba628d..55b25f145ae0d 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -63,10 +63,14 @@ class KryoSerializer extends org.apache.spark.serializer.Serializer with Logging kryo.register(classOf[HttpBroadcast[_]], new KryoJavaSerializer()) // Allow the user to register their own classes by setting spark.kryo.registrator - Option(System.getProperty("spark.kryo.registrator")).foreach { regCls => - logDebug("Running user registrator: " + regCls) - val reg = Class.forName(regCls, true, classLoader).newInstance().asInstanceOf[KryoRegistrator] - reg.registerClasses(kryo) + try { + Option(System.getProperty("spark.kryo.registrator")).foreach { regCls => + logDebug("Running user registrator: " + regCls) + val reg = Class.forName(regCls, true, classLoader).newInstance().asInstanceOf[KryoRegistrator] + reg.registerClasses(kryo) + } + } catch { + case _: Exception => println("Failed to register spark.kryo.registrator") } // Register Chill's classes; we do this after our ranges and the user's own classes to let @@ -118,7 +122,7 @@ class KryoDeserializationStream(kryo: Kryo, inStream: InputStream) extends Deser } } -private[spark] class KryoSerializerInstance(ks: KryoSerializer) extends SerializerInstance with Logging { +private[spark] class KryoSerializerInstance(ks: KryoSerializer) extends SerializerInstance { val kryo = ks.newKryo() // Make these lazy vals to avoid creating a buffer unless we use them diff --git a/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala b/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala index 5082730ae3fa3..2955986feced5 100644 --- a/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala +++ b/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala @@ -25,7 +25,7 @@ import java.util.concurrent.ConcurrentHashMap * instance of the serializer object has been created, the get method returns that instead of * creating a new one. */ -private[spark] class SerializerManager extends org.apache.spark.Logging { +private[spark] class SerializerManager { private val serializers = new ConcurrentHashMap[String, Serializer] private var _default: Serializer = _ From 36a902e52d062ac352f95b317c308894496eecc2 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 18 Oct 2013 16:19:40 -0700 Subject: [PATCH 128/531] Revert accidental removal of code in 3a40a5e --- graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index a6604b978f996..e7a708e895b28 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -275,6 +275,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( .groupBy { t: EdgeTriplet[VD, ED] => (t.srcId, t.dstId) } .mapValues { ts: List[EdgeTriplet[VD, ED]] => f(ts.toIterator) } .toList + .toIterator .map { case ((src, dst), data) => Edge(src, dst, data) } } From bb58aa53307271d5fb9141773209cc850c08d470 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Fri, 18 Oct 2013 16:14:39 -0700 Subject: [PATCH 129/531] Added some stub code to address the case where a vertex could occur multiple times in the vertex table or where a vertex in the edge list may not appear in the vertex table. Moving IndexedRDD into the graphx source tree and removing dependencies in /core. --- .../scala/org/apache/spark/SparkContext.scala | 2 +- .../apache/spark/api/java/JavaPairRDD.scala | 78 +-- .../spark/rdd/IndexedRDDFunctions.scala | 283 ----------- .../apache/spark/rdd/PairRDDFunctions.scala | 67 +-- .../main/scala/org/apache/spark/rdd/RDD.scala | 20 - .../org/apache/spark/IndexedRDDSuite.scala | 461 ------------------ .../apache/spark/rdd/IndexedRDDSuite.scala | 461 ------------------ .../scala/org/apache/spark/graph/Graph.scala | 29 +- .../org/apache/spark/graph/GraphLoader.scala | 2 +- .../org/apache/spark/graph}/IndexedRDD.scala | 184 ++++++- .../apache/spark/graph/impl/GraphImpl.scala | 57 ++- .../spark/graph/util/GraphGenerators.scala | 4 +- 12 files changed, 270 insertions(+), 1378 deletions(-) delete mode 100644 core/src/main/scala/org/apache/spark/rdd/IndexedRDDFunctions.scala delete mode 100644 core/src/test/scala/org/apache/spark/IndexedRDDSuite.scala delete mode 100644 core/src/test/scala/org/apache/spark/rdd/IndexedRDDSuite.scala rename {core/src/main/scala/org/apache/spark/rdd => graph/src/main/scala/org/apache/spark/graph}/IndexedRDD.scala (73%) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index b3a2cb39fcdd1..0aafc0a2fc7d0 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -959,7 +959,7 @@ object SparkContext { // TODO: Add AccumulatorParams for other types, e.g. lists and strings implicit def rddToPairRDDFunctions[K: ClassManifest, V: ClassManifest](rdd: RDD[(K, V)]) = - rdd.pairRDDFunctions + new PairRDDFunctions(rdd) implicit def rddToAsyncRDDActions[T: ClassManifest](rdd: RDD[T]) = new AsyncRDDActions(rdd) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index 2f94ae5fa8ee6..a6518abf456d3 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala @@ -264,11 +264,8 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif * the merging locally on each mapper before sending results to a reducer, similarly to a * "combiner" in MapReduce. */ - def join[W](other: JavaPairRDD[K, W], partitioner: Partitioner): JavaPairRDD[K, (V, W)] = { - implicit val wm: ClassManifest[W] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] + def join[W](other: JavaPairRDD[K, W], partitioner: Partitioner): JavaPairRDD[K, (V, W)] = fromRDD(rdd.join(other, partitioner)) - } /** * Perform a left outer join of `this` and `other`. For each element (k, v) in `this`, the @@ -278,8 +275,6 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif */ def leftOuterJoin[W](other: JavaPairRDD[K, W], partitioner: Partitioner) : JavaPairRDD[K, (V, Optional[W])] = { - implicit val wm: ClassManifest[W] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] val joinResult = rdd.leftOuterJoin(other, partitioner) fromRDD(joinResult.mapValues{case (v, w) => (v, JavaUtils.optionToOptional(w))}) } @@ -292,8 +287,6 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif */ def rightOuterJoin[W](other: JavaPairRDD[K, W], partitioner: Partitioner) : JavaPairRDD[K, (Optional[V], W)] = { - implicit val wm: ClassManifest[W] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] val joinResult = rdd.rightOuterJoin(other, partitioner) fromRDD(joinResult.mapValues{case (v, w) => (JavaUtils.optionToOptional(v), w)}) } @@ -332,22 +325,16 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and * (k, v2) is in `other`. Performs a hash join across the cluster. */ - def join[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (V, W)] = { - implicit val wm: ClassManifest[W] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] + def join[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (V, W)] = fromRDD(rdd.join(other)) - } /** * Return an RDD containing all pairs of elements with matching keys in `this` and `other`. Each * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and * (k, v2) is in `other`. Performs a hash join across the cluster. */ - def join[W](other: JavaPairRDD[K, W], numPartitions: Int): JavaPairRDD[K, (V, W)] = { - implicit val wm: ClassManifest[W] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] + def join[W](other: JavaPairRDD[K, W], numPartitions: Int): JavaPairRDD[K, (V, W)] = fromRDD(rdd.join(other, numPartitions)) - } /** * Perform a left outer join of `this` and `other`. For each element (k, v) in `this`, the @@ -356,8 +343,6 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif * using the existing partitioner/parallelism level. */ def leftOuterJoin[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (V, Optional[W])] = { - implicit val wm: ClassManifest[W] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] val joinResult = rdd.leftOuterJoin(other) fromRDD(joinResult.mapValues{case (v, w) => (v, JavaUtils.optionToOptional(w))}) } @@ -369,8 +354,6 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif * into `numPartitions` partitions. */ def leftOuterJoin[W](other: JavaPairRDD[K, W], numPartitions: Int): JavaPairRDD[K, (V, Optional[W])] = { - implicit val wm: ClassManifest[W] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] val joinResult = rdd.leftOuterJoin(other, numPartitions) fromRDD(joinResult.mapValues{case (v, w) => (v, JavaUtils.optionToOptional(w))}) } @@ -382,8 +365,6 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif * RDD using the existing partitioner/parallelism level. */ def rightOuterJoin[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (Optional[V], W)] = { - implicit val wm: ClassManifest[W] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] val joinResult = rdd.rightOuterJoin(other) fromRDD(joinResult.mapValues{case (v, w) => (JavaUtils.optionToOptional(v), w)}) } @@ -395,8 +376,6 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif * RDD into the given number of partitions. */ def rightOuterJoin[W](other: JavaPairRDD[K, W], numPartitions: Int): JavaPairRDD[K, (Optional[V], W)] = { - implicit val wm: ClassManifest[W] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] val joinResult = rdd.rightOuterJoin(other, numPartitions) fromRDD(joinResult.mapValues{case (v, w) => (JavaUtils.optionToOptional(v), w)}) } @@ -433,86 +412,55 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif * list of values for that key in `this` as well as `other`. */ def cogroup[W](other: JavaPairRDD[K, W], partitioner: Partitioner) - : JavaPairRDD[K, (JList[V], JList[W])] = { - implicit val wm: ClassManifest[W] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] + : JavaPairRDD[K, (JList[V], JList[W])] = fromRDD(cogroupResultToJava(rdd.cogroup(other, partitioner))) - } /** * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a * tuple with the list of values for that key in `this`, `other1` and `other2`. */ def cogroup[W1, W2](other1: JavaPairRDD[K, W1], other2: JavaPairRDD[K, W2], partitioner: Partitioner) - : JavaPairRDD[K, (JList[V], JList[W1], JList[W2])] = { - implicit val w1m: ClassManifest[W1] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W1]] - implicit val w2m: ClassManifest[W2] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W2]] + : JavaPairRDD[K, (JList[V], JList[W1], JList[W2])] = fromRDD(cogroupResult2ToJava(rdd.cogroup(other1, other2, partitioner))) - } /** * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the * list of values for that key in `this` as well as `other`. */ - def cogroup[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (JList[V], JList[W])] = { - implicit val wm: ClassManifest[W] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] + def cogroup[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (JList[V], JList[W])] = fromRDD(cogroupResultToJava(rdd.cogroup(other))) - } /** * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a * tuple with the list of values for that key in `this`, `other1` and `other2`. */ def cogroup[W1, W2](other1: JavaPairRDD[K, W1], other2: JavaPairRDD[K, W2]) - : JavaPairRDD[K, (JList[V], JList[W1], JList[W2])] = { - implicit val w1m: ClassManifest[W1] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W1]] - implicit val w2m: ClassManifest[W2] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W2]] + : JavaPairRDD[K, (JList[V], JList[W1], JList[W2])] = fromRDD(cogroupResult2ToJava(rdd.cogroup(other1, other2))) - } /** * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the * list of values for that key in `this` as well as `other`. */ - def cogroup[W](other: JavaPairRDD[K, W], numPartitions: Int): JavaPairRDD[K, (JList[V], JList[W])] = { - implicit val wm: ClassManifest[W] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] - fromRDD(cogroupResultToJava(rdd.cogroup(other, numPartitions))) - } + def cogroup[W](other: JavaPairRDD[K, W], numPartitions: Int): JavaPairRDD[K, (JList[V], JList[W])] + = fromRDD(cogroupResultToJava(rdd.cogroup(other, numPartitions))) + /** * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a * tuple with the list of values for that key in `this`, `other1` and `other2`. */ def cogroup[W1, W2](other1: JavaPairRDD[K, W1], other2: JavaPairRDD[K, W2], numPartitions: Int) - : JavaPairRDD[K, (JList[V], JList[W1], JList[W2])] = { - implicit val w1m: ClassManifest[W1] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W1]] - implicit val w2m: ClassManifest[W2] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W2]] + : JavaPairRDD[K, (JList[V], JList[W1], JList[W2])] = fromRDD(cogroupResult2ToJava(rdd.cogroup(other1, other2, numPartitions))) - } /** Alias for cogroup. */ - def groupWith[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (JList[V], JList[W])] = { - implicit val wm: ClassManifest[W] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] + def groupWith[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (JList[V], JList[W])] = fromRDD(cogroupResultToJava(rdd.groupWith(other))) - } /** Alias for cogroup. */ def groupWith[W1, W2](other1: JavaPairRDD[K, W1], other2: JavaPairRDD[K, W2]) - : JavaPairRDD[K, (JList[V], JList[W1], JList[W2])] = { - implicit val w1m: ClassManifest[W1] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W1]] - implicit val w2m: ClassManifest[W2] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W2]] + : JavaPairRDD[K, (JList[V], JList[W1], JList[W2])] = fromRDD(cogroupResult2ToJava(rdd.groupWith(other1, other2))) - } /** * Return the list of values in the RDD for key `key`. This operation is done efficiently if the diff --git a/core/src/main/scala/org/apache/spark/rdd/IndexedRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/IndexedRDDFunctions.scala deleted file mode 100644 index fd7c16089d69e..0000000000000 --- a/core/src/main/scala/org/apache/spark/rdd/IndexedRDDFunctions.scala +++ /dev/null @@ -1,283 +0,0 @@ -/* - * 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.rdd - -import java.util.{HashMap => JHashMap, BitSet => JBitSet, HashSet => JHashSet} - -import scala.collection.JavaConversions._ -import scala.collection.mutable.ArrayBuffer - -import scala.collection.mutable.BitSet - -import org.apache.spark._ - - - -class IndexedRDDFunctions[K: ClassManifest, V: ClassManifest](self: IndexedRDD[K,V]) - extends PairRDDFunctions[K,V](self) { - - /** - * Construct a new IndexedRDD that is indexed by only the keys in the RDD - */ - def reindex(): IndexedRDD[K,V] = IndexedRDD(self) - - - // /** - // * Pass each value in the key-value pair RDD through a map function without changing the keys; - // * this also retains the original RDD's partitioning. - // */ - // override def mapValues[U: ClassManifest](f: V => U): RDD[(K, U)] = { - // val cleanF = self.index.rdd.context.clean(f) - // val newValuesRDD = self.valuesRDD.mapPartitions(iter => iter.map{ - // case (values, bs) => - // val newValues = new Array[U](values.size) - // for ( ind <- bs ) { - // newValues(ind) = f(values(ind)) - // } - // (newValues.toSeq, bs) - // }, preservesPartitioning = true) - // new IndexedRDD[K,U](self.index, newValuesRDD) - // } - - /** - * Pass each value in the key-value pair RDD through a flatMap function without changing the - * keys; this also retains the original RDD's partitioning. - */ - override def flatMapValues[U: ClassManifest](f: V => TraversableOnce[U]): RDD[(K,U)] = { - val cleanF = self.index.rdd.context.clean(f) - val newValuesRDD: RDD[(IndexedSeq[U], BitSet)] = self.valuesRDD.mapPartitions(iter => iter.map{ - case (values, bs) => - val newValues = new Array[U](values.size) - val newBS = new BitSet(values.size) - for ( ind <- bs ) { - val res = f(values(ind)) - if(!res.isEmpty) { - newValues(ind) = res.toIterator.next() - newBS(ind) = true - } - } - (newValues.toIndexedSeq, newBS) - }, preservesPartitioning = true) - new IndexedRDD[K,U](self.index, newValuesRDD) - } - - - /** - * Generic function to combine the elements for each key using a custom set of aggregation - * functions. Turns an RDD[(K, V)] into a result of type RDD[(K, C)], for a "combined type" C - * Note that V and C can be different -- for example, one might group an RDD of type - * (Int, Int) into an RDD of type (Int, Seq[Int]). Users provide three functions: - * - * - `createCombiner`, which turns a V into a C (e.g., creates a one-element list) - * - `mergeValue`, to merge a V into a C (e.g., adds it to the end of a list) - * - `mergeCombiners`, to combine two C's into a single one. - */ - override def combineByKey[C: ClassManifest](createCombiner: V => C, - mergeValue: (C, V) => C, - mergeCombiners: (C, C) => C, - partitioner: Partitioner, - mapSideCombine: Boolean = true, - serializerClass: String = null): RDD[(K, C)] = { - mapValues(createCombiner) - } - - - // /** - // * Group the values for each key in the RDD into a single sequence. Hash-partitions the - // * resulting RDD with the existing partitioner/parallelism level. - // */ - // override def groupByKey(partitioner: Partitioner): RDD[(K, Seq[V])] = { - // val newValues = self.valuesRDD.mapPartitions(_.map{ar => ar.map{s => Seq(s)} }, true) - // new IndexedRDD[K, Seq[V]](self.index, newValues) - // } - - - /** - * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the - * list of values for that key in `this` as well as `other`. - */ - override def cogroup[W: ClassManifest](other: RDD[(K, W)], partitioner: Partitioner): - IndexedRDD[K, (Seq[V], Seq[W])] = { - //RDD[(K, (Seq[V], Seq[W]))] = { - other match { - case other: IndexedRDD[_, _] if self.index == other.index => { - // if both RDDs share exactly the same index and therefore the same super set of keys - // then we simply merge the value RDDs. - // However it is possible that both RDDs are missing a value for a given key in - // which case the returned RDD should have a null value - val newValues: RDD[(IndexedSeq[(Seq[V], Seq[W])], BitSet)] = - self.valuesRDD.zipPartitions(other.valuesRDD){ - (thisIter, otherIter) => - val (thisValues, thisBS) = thisIter.next() - assert(!thisIter.hasNext) - val (otherValues, otherBS) = otherIter.next() - assert(!otherIter.hasNext) - - val newValues = new Array[(Seq[V], Seq[W])](thisValues.size) - val newBS = thisBS | otherBS - - for( ind <- newBS ) { - val a = if (thisBS(ind)) Seq(thisValues(ind)) else Seq.empty[V] - val b = if (otherBS(ind)) Seq(otherValues(ind)) else Seq.empty[W] - newValues(ind) = (a, b) - } - Iterator((newValues.toIndexedSeq, newBS)) - } - new IndexedRDD(self.index, newValues) - } - case other: IndexedRDD[_, _] - if self.index.rdd.partitioner == other.index.rdd.partitioner => { - // If both RDDs are indexed using different indices but with the same partitioners - // then we we need to first merge the indicies and then use the merged index to - // merge the values. - val newIndex = - self.index.rdd.zipPartitions(other.index.rdd)( - (thisIter, otherIter) => { - val thisIndex = thisIter.next() - assert(!thisIter.hasNext) - val otherIndex = otherIter.next() - assert(!otherIter.hasNext) - val newIndex = new BlockIndex[K]() - // @todo Merge only the keys that correspond to non-null values - // Merge the keys - newIndex.putAll(thisIndex) - newIndex.putAll(otherIndex) - // We need to rekey the index - var ctr = 0 - for (e <- newIndex.entrySet) { - e.setValue(ctr) - ctr += 1 - } - List(newIndex).iterator - }).cache() - // Use the new index along with the this and the other indices to merge the values - val newValues: RDD[(IndexedSeq[(Seq[V], Seq[W])], BitSet)] = - newIndex.zipPartitions(self.tuples, other.tuples)( - (newIndexIter, thisTuplesIter, otherTuplesIter) => { - // Get the new index for this partition - val newIndex = newIndexIter.next() - assert(!newIndexIter.hasNext) - // Get the corresponding indicies and values for this and the other IndexedRDD - val (thisIndex, (thisValues, thisBS)) = thisTuplesIter.next() - assert(!thisTuplesIter.hasNext) - val (otherIndex, (otherValues, otherBS)) = otherTuplesIter.next() - assert(!otherTuplesIter.hasNext) - // Preallocate the new Values array - val newValues = new Array[(Seq[V], Seq[W])](newIndex.size) - val newBS = new BitSet(newIndex.size) - - // Lookup the sequences in both submaps - for ((k,ind) <- newIndex) { - // Get the left key - val a = if (thisIndex.contains(k)) { - val ind = thisIndex.get(k) - if(thisBS(ind)) Seq(thisValues(ind)) else Seq.empty[V] - } else Seq.empty[V] - // Get the right key - val b = if (otherIndex.contains(k)) { - val ind = otherIndex.get(k) - if (otherBS(ind)) Seq(otherValues(ind)) else Seq.empty[W] - } else Seq.empty[W] - // If at least one key was present then we generate a tuple. - if (!a.isEmpty || !b.isEmpty) { - newValues(ind) = (a, b) - newBS(ind) = true - } - } - Iterator((newValues.toIndexedSeq, newBS)) - }) - new IndexedRDD(new RDDIndex(newIndex), newValues) - } - case _ => { - // Get the partitioner from the index - val partitioner = self.index.rdd.partitioner match { - case Some(p) => p - case None => throw new SparkException("An index must have a partitioner.") - } - // Shuffle the other RDD using the partitioner for this index - val otherShuffled = - if (other.partitioner == Some(partitioner)) { - other - } else { - new ShuffledRDD[K, W, (K,W)](other, partitioner) - } - // Join the other RDD with this RDD building a new valueset and new index on the fly - val groups = - self.tuples.zipPartitions(otherShuffled)( - (thisTuplesIter, otherTuplesIter) => { - // Get the corresponding indicies and values for this IndexedRDD - val (thisIndex, (thisValues, thisBS)) = thisTuplesIter.next() - assert(!thisTuplesIter.hasNext()) - // Construct a new index - val newIndex = thisIndex.clone().asInstanceOf[BlockIndex[K]] - // Construct a new array Buffer to store the values - val newValues = ArrayBuffer.fill[ (Seq[V], Seq[W]) ](thisValues.size)(null) - val newBS = new BitSet(thisValues.size) - // populate the newValues with the values in this IndexedRDD - for ((k,i) <- thisIndex) { - if (thisBS(i)) { - newValues(i) = (Seq(thisValues(i)), ArrayBuffer.empty[W]) - newBS(i) = true - } - } - // Now iterate through the other tuples updating the map - for ((k,w) <- otherTuplesIter){ - if (newIndex.contains(k)) { - val ind = newIndex.get(k) - if(newBS(ind)) { - newValues(ind)._2.asInstanceOf[ArrayBuffer[W]].append(w) - } else { - // If the other key was in the index but not in the values - // of this indexed RDD then create a new values entry for it - newBS(ind) = true - newValues(ind) = (Seq.empty[V], ArrayBuffer(w)) - } - } else { - // update the index - val ind = newIndex.size - newIndex.put(k, ind) - newBS(ind) = true - // Update the values - newValues.append( (Seq.empty[V], ArrayBuffer(w) ) ) - } - } - // // Finalize the new values array - // val newValuesArray: Seq[Seq[(Seq[V],Seq[W])]] = - // newValues.view.map{ - // case null => null - // case (s, ab) => Seq((s, ab.toSeq)) - // }.toSeq - Iterator( (newIndex, (newValues.toIndexedSeq, newBS)) ) - }).cache() - - // Extract the index and values from the above RDD - val newIndex = groups.mapPartitions(_.map{ case (kMap,vAr) => kMap }, true) - val newValues: RDD[(IndexedSeq[(Seq[V], Seq[W])], BitSet)] = - groups.mapPartitions(_.map{ case (kMap,vAr) => vAr }, true) - - new IndexedRDD[K, (Seq[V], Seq[W])](new RDDIndex(newIndex), newValues) - } - } - } - - -} - -//(self: IndexedRDD[K, V]) extends PairRDDFunctions(self) { } - - diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 81bf867188ac4..93b78e123267c 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -68,7 +68,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) * In addition, users can control the partitioning of the output RDD, and whether to perform * map-side aggregation (if a mapper can produce multiple items with the same key). */ - def combineByKey[C: ClassManifest](createCombiner: V => C, + def combineByKey[C](createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => C, partitioner: Partitioner, @@ -108,7 +108,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) /** * Simplified version of combineByKey that hash-partitions the output RDD. */ - def combineByKey[C: ClassManifest](createCombiner: V => C, + def combineByKey[C](createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => C, numPartitions: Int): RDD[(K, C)] = { @@ -253,7 +253,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and * (k, v2) is in `other`. Uses the given Partitioner to partition the output RDD. */ - def join[W: ClassManifest](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, W))] = { + def join[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, W))] = { this.cogroup(other, partitioner).flatMapValues { case (vs, ws) => for (v <- vs.iterator; w <- ws.iterator) yield (v, w) } @@ -265,7 +265,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) * pair (k, (v, None)) if no elements in `other` have key k. Uses the given Partitioner to * partition the output RDD. */ - def leftOuterJoin[W: ClassManifest](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, Option[W]))] = { + def leftOuterJoin[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, Option[W]))] = { this.cogroup(other, partitioner).flatMapValues { case (vs, ws) => if (ws.isEmpty) { vs.iterator.map(v => (v, None)) @@ -281,7 +281,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) * pair (k, (None, w)) if no elements in `this` have key k. Uses the given Partitioner to * partition the output RDD. */ - def rightOuterJoin[W: ClassManifest](other: RDD[(K, W)], partitioner: Partitioner) + def rightOuterJoin[W](other: RDD[(K, W)], partitioner: Partitioner) : RDD[(K, (Option[V], W))] = { this.cogroup(other, partitioner).flatMapValues { case (vs, ws) => if (vs.isEmpty) { @@ -296,7 +296,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) * Simplified version of combineByKey that hash-partitions the resulting RDD using the * existing partitioner/parallelism level. */ - def combineByKey[C: ClassManifest](createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => C) + def combineByKey[C](createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => C) : RDD[(K, C)] = { combineByKey(createCombiner, mergeValue, mergeCombiners, defaultPartitioner(self)) } @@ -324,7 +324,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and * (k, v2) is in `other`. Performs a hash join across the cluster. */ - def join[W: ClassManifest](other: RDD[(K, W)]): RDD[(K, (V, W))] = { + def join[W](other: RDD[(K, W)]): RDD[(K, (V, W))] = { join(other, defaultPartitioner(self, other)) } @@ -333,7 +333,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and * (k, v2) is in `other`. Performs a hash join across the cluster. */ - def join[W: ClassManifest](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (V, W))] = { + def join[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (V, W))] = { join(other, new HashPartitioner(numPartitions)) } @@ -343,7 +343,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) * pair (k, (v, None)) if no elements in `other` have key k. Hash-partitions the output * using the existing partitioner/parallelism level. */ - def leftOuterJoin[W: ClassManifest](other: RDD[(K, W)]): RDD[(K, (V, Option[W]))] = { + def leftOuterJoin[W](other: RDD[(K, W)]): RDD[(K, (V, Option[W]))] = { leftOuterJoin(other, defaultPartitioner(self, other)) } @@ -353,7 +353,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) * pair (k, (v, None)) if no elements in `other` have key k. Hash-partitions the output * into `numPartitions` partitions. */ - def leftOuterJoin[W: ClassManifest](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (V, Option[W]))] = { + def leftOuterJoin[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (V, Option[W]))] = { leftOuterJoin(other, new HashPartitioner(numPartitions)) } @@ -363,7 +363,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) * pair (k, (None, w)) if no elements in `this` have key k. Hash-partitions the resulting * RDD using the existing partitioner/parallelism level. */ - def rightOuterJoin[W: ClassManifest](other: RDD[(K, W)]): RDD[(K, (Option[V], W))] = { + def rightOuterJoin[W](other: RDD[(K, W)]): RDD[(K, (Option[V], W))] = { rightOuterJoin(other, defaultPartitioner(self, other)) } @@ -373,7 +373,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) * pair (k, (None, w)) if no elements in `this` have key k. Hash-partitions the resulting * RDD into the given number of partitions. */ - def rightOuterJoin[W: ClassManifest](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (Option[V], W))] = { + def rightOuterJoin[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (Option[V], W))] = { rightOuterJoin(other, new HashPartitioner(numPartitions)) } @@ -392,25 +392,16 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) * Pass each value in the key-value pair RDD through a map function without changing the keys; * this also retains the original RDD's partitioning. */ - def mapValues[U: ClassManifest](f: V => U): RDD[(K, U)] = { + def mapValues[U](f: V => U): RDD[(K, U)] = { val cleanF = self.context.clean(f) new MappedValuesRDD(self, cleanF) } - - /** - * Pass each value in the key-value pair RDD through a map function without changing the keys; - * this also retains the original RDD's partitioning. - */ - def mapValuesWithKeys[U: ClassManifest](f: (K, V) => U): RDD[(K, U)] = { - self.map{ case (k,v) => (k, f(k,v)) } - } - /** * Pass each value in the key-value pair RDD through a flatMap function without changing the * keys; this also retains the original RDD's partitioning. */ - def flatMapValues[U: ClassManifest](f: V => TraversableOnce[U]): RDD[(K, U)] = { + def flatMapValues[U](f: V => TraversableOnce[U]): RDD[(K, U)] = { val cleanF = self.context.clean(f) new FlatMappedValuesRDD(self, cleanF) } @@ -419,7 +410,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the * list of values for that key in `this` as well as `other`. */ - def cogroup[W: ClassManifest](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (Seq[V], Seq[W]))] = { + def cogroup[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (Seq[V], Seq[W]))] = { if (partitioner.isInstanceOf[HashPartitioner] && getKeyClass().isArray) { throw new SparkException("Default partitioner cannot partition array keys.") } @@ -434,7 +425,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a * tuple with the list of values for that key in `this`, `other1` and `other2`. */ - def cogroup[W1: ClassManifest, W2: ClassManifest](other1: RDD[(K, W1)], other2: RDD[(K, W2)], partitioner: Partitioner) + def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)], partitioner: Partitioner) : RDD[(K, (Seq[V], Seq[W1], Seq[W2]))] = { if (partitioner.isInstanceOf[HashPartitioner] && getKeyClass().isArray) { throw new SparkException("Default partitioner cannot partition array keys.") @@ -450,7 +441,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the * list of values for that key in `this` as well as `other`. */ - def cogroup[W: ClassManifest](other: RDD[(K, W)]): RDD[(K, (Seq[V], Seq[W]))] = { + def cogroup[W](other: RDD[(K, W)]): RDD[(K, (Seq[V], Seq[W]))] = { cogroup(other, defaultPartitioner(self, other)) } @@ -458,7 +449,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a * tuple with the list of values for that key in `this`, `other1` and `other2`. */ - def cogroup[W1: ClassManifest, W2: ClassManifest](other1: RDD[(K, W1)], other2: RDD[(K, W2)]) + def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)]) : RDD[(K, (Seq[V], Seq[W1], Seq[W2]))] = { cogroup(other1, other2, defaultPartitioner(self, other1, other2)) } @@ -467,7 +458,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the * list of values for that key in `this` as well as `other`. */ - def cogroup[W: ClassManifest](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (Seq[V], Seq[W]))] = { + def cogroup[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (Seq[V], Seq[W]))] = { cogroup(other, new HashPartitioner(numPartitions)) } @@ -475,18 +466,18 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a * tuple with the list of values for that key in `this`, `other1` and `other2`. */ - def cogroup[W1: ClassManifest, W2: ClassManifest](other1: RDD[(K, W1)], other2: RDD[(K, W2)], numPartitions: Int) + def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)], numPartitions: Int) : RDD[(K, (Seq[V], Seq[W1], Seq[W2]))] = { cogroup(other1, other2, new HashPartitioner(numPartitions)) } /** Alias for cogroup. */ - def groupWith[W: ClassManifest](other: RDD[(K, W)]): RDD[(K, (Seq[V], Seq[W]))] = { + def groupWith[W](other: RDD[(K, W)]): RDD[(K, (Seq[V], Seq[W]))] = { cogroup(other, defaultPartitioner(self, other)) } /** Alias for cogroup. */ - def groupWith[W1: ClassManifest, W2: ClassManifest](other1: RDD[(K, W1)], other2: RDD[(K, W2)]) + def groupWith[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)]) : RDD[(K, (Seq[V], Seq[W1], Seq[W2]))] = { cogroup(other1, other2, defaultPartitioner(self, other1, other2)) } @@ -707,20 +698,6 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) */ def values: RDD[V] = self.map(_._2) - - - def indexed(): IndexedRDD[K,V] = IndexedRDD(self) - - def indexed(numPartitions: Int): IndexedRDD[K,V] = - IndexedRDD(self.partitionBy(new HashPartitioner(numPartitions))) - - def indexed(partitioner: Partitioner): IndexedRDD[K,V] = - IndexedRDD(self.partitionBy(partitioner)) - - def indexed(existingIndex: RDDIndex[K]): IndexedRDD[K,V] = - IndexedRDD(self, existingIndex) - - private[spark] def getKeyClass() = implicitly[ClassManifest[K]].erasure private[spark] def getValueClass() = implicitly[ClassManifest[V]].erasure diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index d14b4c60c7323..0355618e435bd 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -805,26 +805,6 @@ abstract class RDD[T: ClassManifest]( return buf.toArray } - - - /** - * For RDD[(K,V)] this function returns a pair-functions object for this RDD - */ - def pairRDDFunctions[K, V]( - implicit t: T <:< (K, V), k: ClassManifest[K], v: ClassManifest[V]): - PairRDDFunctions[K, V] = { - new PairRDDFunctions(this.asInstanceOf[RDD[(K,V)]]) - } - - - /** - * Construct an index over the unique elements in this RDD. The - * index can then be used to organize a RDD[(T,V)]. - */ - def makeIndex(partitioner: Option[Partitioner] = None): RDDIndex[T] = - IndexedRDD.makeIndex(this, partitioner) - - /** * Return the first element in this RDD. */ diff --git a/core/src/test/scala/org/apache/spark/IndexedRDDSuite.scala b/core/src/test/scala/org/apache/spark/IndexedRDDSuite.scala deleted file mode 100644 index dadb183bdca09..0000000000000 --- a/core/src/test/scala/org/apache/spark/IndexedRDDSuite.scala +++ /dev/null @@ -1,461 +0,0 @@ -/* - * 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 - - -import org.scalatest.FunSuite -import org.scalatest.prop.Checkers -import org.scalacheck.Arbitrary._ -import org.scalacheck.Gen -import org.scalacheck.Prop._ - -import com.google.common.io.Files - -import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.HashSet - -import org.apache.spark.rdd.RDD -import org.apache.spark.rdd.ShuffledRDD -import org.apache.spark.rdd.IndexedRDD - -import org.apache.spark.SparkContext._ -import org.apache.spark._ - - - -class IndexedRDDSuite extends FunSuite with SharedSparkContext { - - def lineage(rdd: RDD[_]): collection.mutable.HashSet[RDD[_]] = { - val set = new collection.mutable.HashSet[RDD[_]] - def visit(rdd: RDD[_]) { - for (dep <- rdd.dependencies) { - set += dep.rdd - visit(dep.rdd) - } - } - visit(rdd) - set - } - - test("groupByKey") { - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1))).indexed() - val groups = pairs.groupByKey().collect() - assert(groups.size === 2) - val valuesFor1 = groups.find(_._1 == 1).get._2 - assert(valuesFor1.toList.sorted === List(1, 2, 3)) - val valuesFor2 = groups.find(_._1 == 2).get._2 - assert(valuesFor2.toList.sorted === List(1)) - } - - test("groupByKey with duplicates") { - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))).indexed() - val groups = pairs.groupByKey().collect() - assert(groups.size === 2) - val valuesFor1 = groups.find(_._1 == 1).get._2 - assert(valuesFor1.toList.sorted === List(1, 1, 2, 3)) - val valuesFor2 = groups.find(_._1 == 2).get._2 - assert(valuesFor2.toList.sorted === List(1)) - } - - test("groupByKey with negative key hash codes") { - val pairs = sc.parallelize(Array((-1, 1), (-1, 2), (-1, 3), (2, 1))).indexed() - val groups = pairs.groupByKey().collect() - assert(groups.size === 2) - val valuesForMinus1 = groups.find(_._1 == -1).get._2 - assert(valuesForMinus1.toList.sorted === List(1, 2, 3)) - val valuesFor2 = groups.find(_._1 == 2).get._2 - assert(valuesFor2.toList.sorted === List(1)) - } - - test("groupByKey with many output partitions") { - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1))).indexed(10) - val groups = pairs.groupByKey().collect() - assert(groups.size === 2) - val valuesFor1 = groups.find(_._1 == 1).get._2 - assert(valuesFor1.toList.sorted === List(1, 2, 3)) - val valuesFor2 = groups.find(_._1 == 2).get._2 - assert(valuesFor2.toList.sorted === List(1)) - } - - test("reduceByKey") { - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))).indexed() - val sums = pairs.reduceByKey(_+_).collect() - assert(sums.toSet === Set((1, 7), (2, 1))) - } - - test("reduceByKey with collectAsMap") { - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))).indexed() - val sums = pairs.reduceByKey(_+_).collectAsMap() - assert(sums.size === 2) - assert(sums(1) === 7) - assert(sums(2) === 1) - } - - test("reduceByKey with many output partitons") { - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))).indexed(10) - val sums = pairs.reduceByKey(_+_).collect() - assert(sums.toSet === Set((1, 7), (2, 1))) - } - - test("reduceByKey with partitioner") { - val p = new Partitioner() { - def numPartitions = 2 - def getPartition(key: Any) = key.asInstanceOf[Int] - } - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 1), (0, 1))).indexed(p) - val sums = pairs.reduceByKey(_+_) - assert(sums.collect().toSet === Set((1, 4), (0, 1))) - assert(sums.partitioner === Some(p)) - // count the dependencies to make sure there is only 1 ShuffledRDD - val deps = lineage(sums) - - assert(deps.filter(_.isInstanceOf[ShuffledRDD[_,_,_]]).size === 1) // ShuffledRDD, ParallelCollection - } - - - - test("joinIndexVsPair") { - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed() - val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) - val joined = rdd1.join(rdd2).collect() - assert(joined.size === 4) - assert(joined.toSet === Set( - (1, (1, 'x')), - (1, (2, 'x')), - (2, (1, 'y')), - (2, (1, 'z')) - )) - } - - test("joinIndexVsIndex") { - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed() - val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).indexed() - val joined = rdd1.join(rdd2).collect() - assert(joined.size === 4) - assert(joined.toSet === Set( - (1, (1, 'x')), - (1, (2, 'x')), - (2, (1, 'y')), - (2, (1, 'z')) - )) - } - - test("joinSharedIndex") { - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1), (4,-4), (4, 4) )).indexed() - val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).indexed(rdd1.index) - val joined = rdd1.join(rdd2).collect() - assert(joined.size === 6) - assert(joined.toSet === Set( - (1, (1, 'x')), - (1, (2, 'x')), - (2, (1, 'y')), - (2, (1, 'z')), - (4, (-4, 'w')), - (4, (4, 'w')) - )) - } - - - test("join all-to-all") { - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (1, 3))).indexed() - val rdd2 = sc.parallelize(Array((1, 'x'), (1, 'y'))).indexed(rdd1.index) - val joined = rdd1.join(rdd2).collect() - assert(joined.size === 6) - assert(joined.toSet === Set( - (1, (1, 'x')), - (1, (1, 'y')), - (1, (2, 'x')), - (1, (2, 'y')), - (1, (3, 'x')), - (1, (3, 'y')) - )) - } - - test("leftOuterJoinIndex") { - val index = sc.parallelize( 1 to 6 ).makeIndex() - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed(index) - val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) - val joined = rdd1.leftOuterJoin(rdd2).collect() - assert(joined.size === 5) - assert(joined.toSet === Set( - (1, (1, Some('x'))), - (1, (2, Some('x'))), - (2, (1, Some('y'))), - (2, (1, Some('z'))), - (3, (1, None)) - )) - } - - test("leftOuterJoinIndextoIndex") { - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed() - val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).indexed() - val joined = rdd1.leftOuterJoin(rdd2).collect() - assert(joined.size === 5) - assert(joined.toSet === Set( - (1, (1, Some('x'))), - (1, (2, Some('x'))), - (2, (1, Some('y'))), - (2, (1, Some('z'))), - (3, (1, None)) - )) - } - - test("leftOuterJoinIndextoSharedIndex") { - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1), (4, -4))).indexed() - val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).indexed(rdd1.index) - val joined = rdd1.leftOuterJoin(rdd2).collect() - assert(joined.size === 6) - assert(joined.toSet === Set( - (1, (1, Some('x'))), - (1, (2, Some('x'))), - (2, (1, Some('y'))), - (2, (1, Some('z'))), - (4, (-4, Some('w'))), - (3, (1, None)) - )) - } - -test("leftOuterJoinIndextoIndexExternal") { - val index = sc.parallelize( 1 to 6 ).makeIndex() - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed(index) - val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).indexed(index) - val joined = rdd1.leftOuterJoin(rdd2).collect() - assert(joined.size === 5) - assert(joined.toSet === Set( - (1, (1, Some('x'))), - (1, (2, Some('x'))), - (2, (1, Some('y'))), - (2, (1, Some('z'))), - (3, (1, None)) - )) - } - - - test("rightOuterJoin") { - val index = sc.parallelize( 1 to 6 ).makeIndex() - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed(index) - val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) - val joined = rdd1.rightOuterJoin(rdd2).collect() - assert(joined.size === 5) - assert(joined.toSet === Set( - (1, (Some(1), 'x')), - (1, (Some(2), 'x')), - (2, (Some(1), 'y')), - (2, (Some(1), 'z')), - (4, (None, 'w')) - )) - } - - test("rightOuterJoinIndex2Index") { - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed() - val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).indexed() - val joined = rdd1.rightOuterJoin(rdd2).collect() - assert(joined.size === 5) - assert(joined.toSet === Set( - (1, (Some(1), 'x')), - (1, (Some(2), 'x')), - (2, (Some(1), 'y')), - (2, (Some(1), 'z')), - (4, (None, 'w')) - )) - } - - - test("rightOuterJoinIndex2Indexshared") { - val index = sc.parallelize( 1 to 6 ).makeIndex() - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed(index) - val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).indexed(index) - val joined = rdd1.rightOuterJoin(rdd2).collect() - assert(joined.size === 5) - assert(joined.toSet === Set( - (1, (Some(1), 'x')), - (1, (Some(2), 'x')), - (2, (Some(1), 'y')), - (2, (Some(1), 'z')), - (4, (None, 'w')) - )) - } - - - test("join with no matches index") { - val index = IndexedRDD.makeIndex( sc.parallelize( 1 to 6 ) ) - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed(index) - val rdd2 = sc.parallelize(Array((4, 'x'), (5, 'y'), (5, 'z'), (6, 'w'))) - val joined = rdd1.join(rdd2).collect() - assert(joined.size === 0) - } - - test("join with no matches shared index") { - val index = IndexedRDD.makeIndex( sc.parallelize( 1 to 6 ) ) - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed(index) - val rdd2 = sc.parallelize(Array((4, 'x'), (5, 'y'), (5, 'z'), (6, 'w'))).indexed(index) - val joined = rdd1.join(rdd2).collect() - assert(joined.size === 0) - } - - - test("join with many output partitions") { - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed(10) - val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) - val joined = rdd1.join(rdd2).collect() - assert(joined.size === 4) - assert(joined.toSet === Set( - (1, (1, 'x')), - (1, (2, 'x')), - (2, (1, 'y')), - (2, (1, 'z')) - )) - } - - test("join with many output partitions and two indices") { - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed(10) - val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).indexed(20) - val joined = rdd1.join(rdd2).collect() - assert(joined.size === 4) - assert(joined.toSet === Set( - (1, (1, 'x')), - (1, (2, 'x')), - (2, (1, 'y')), - (2, (1, 'z')) - )) - } - - - test("groupWith") { - val index = IndexedRDD.makeIndex( sc.parallelize( 1 to 6 ) ) - - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed(index) - val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).indexed(index) - val joined = rdd1.groupWith(rdd2).collect() - assert(joined.size === 4) - assert(joined.toSet === Set( - (1, (ArrayBuffer(1, 2), ArrayBuffer('x'))), - (2, (ArrayBuffer(1), ArrayBuffer('y', 'z'))), - (3, (ArrayBuffer(1), ArrayBuffer())), - (4, (ArrayBuffer(), ArrayBuffer('w'))) - )) - } - - test("zero-partition RDD") { - val emptyDir = Files.createTempDir() - val file = sc.textFile(emptyDir.getAbsolutePath) - assert(file.partitions.size == 0) - assert(file.collect().toList === Nil) - // Test that a shuffle on the file works, because this used to be a bug - assert(file.map(line => (line, 1)).reduceByKey(_ + _).collect().toList === Nil) - } - - test("keys and values") { - val rdd = sc.parallelize(Array((1, "a"), (2, "b"))).indexed() - assert(rdd.keys.collect().toList === List(1, 2)) - assert(rdd.values.collect().toList === List("a", "b")) - } - - test("default partitioner uses partition size") { - // specify 2000 partitions - val a = sc.makeRDD(Array(1, 2, 3, 4), 2000) - // do a map, which loses the partitioner - val b = a.map(a => (a, (a * 2).toString)) - // then a group by, and see we didn't revert to 2 partitions - val c = b.groupByKey() - assert(c.partitions.size === 2000) - } - - // test("default partitioner uses largest partitioner indexed to indexed") { - // val a = sc.makeRDD(Array((1, "a"), (2, "b")), 2).indexed() - // val b = sc.makeRDD(Array((1, "a"), (2, "b")), 2000).indexed() - // val c = a.join(b) - // assert(c.partitions.size === 2000) - // } - - - - test("subtract") { - val a = sc.parallelize(Array(1, 2, 3), 2) - val b = sc.parallelize(Array(2, 3, 4), 4) - val c = a.subtract(b) - assert(c.collect().toSet === Set(1)) - assert(c.partitions.size === a.partitions.size) - } - - test("subtract with narrow dependency") { - // use a deterministic partitioner - val p = new Partitioner() { - def numPartitions = 5 - def getPartition(key: Any) = key.asInstanceOf[Int] - } - // partitionBy so we have a narrow dependency - val a = sc.parallelize(Array((1, "a"), (2, "b"), (3, "c"))).indexed(p) - // more partitions/no partitioner so a shuffle dependency - val b = sc.parallelize(Array((2, "b"), (3, "cc"), (4, "d")), 4) - val c = a.subtract(b) - assert(c.collect().toSet === Set((1, "a"), (3, "c"))) - // Ideally we could keep the original partitioner... - assert(c.partitioner === None) - } - - test("subtractByKey") { - - val a = sc.parallelize(Array((1, "a"), (1, "a"), (2, "b"), (3, "c")), 2).indexed() - val b = sc.parallelize(Array((2, 20), (3, 30), (4, 40)), 4) - val c = a.subtractByKey(b) - assert(c.collect().toSet === Set((1, "a"), (1, "a"))) - assert(c.partitions.size === a.partitions.size) - } - - // test("subtractByKey with narrow dependency") { - // // use a deterministic partitioner - // val p = new Partitioner() { - // def numPartitions = 5 - // def getPartition(key: Any) = key.asInstanceOf[Int] - // } - - // val index = sc.parallelize( 1 to 6 ).makeIndex(Some(p)) - // // partitionBy so we have a narrow dependency - // val a = sc.parallelize(Array((1, "a"), (1, "a"), (2, "b"), (3, "c"))).indexed(index) - // // more partitions/no partitioner so a shuffle dependency - // val b = sc.parallelize(Array((2, "b"), (3, "cc"), (4, "d")), 4).indexed(index) - // val c = a.subtractByKey(b) - // assert(c.collect().toSet === Set((1, "a"), (1, "a"))) - // assert(c.partitioner.get === p) - // } - - test("foldByKey") { - val index = IndexedRDD.makeIndex( sc.parallelize( 1 to 6 ) ) - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))).indexed(index) - val sums = pairs.foldByKey(0)(_+_).collect() - assert(sums.toSet === Set((1, 7), (2, 1))) - } - - test("foldByKey with mutable result type") { - val index = IndexedRDD.makeIndex( sc.parallelize( 1 to 6 ) ) - - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))).indexed(index) - val bufs = pairs.mapValues(v => ArrayBuffer(v)).cache() - // Fold the values using in-place mutation - val sums = bufs.foldByKey(new ArrayBuffer[Int])(_ ++= _).collect() - assert(sums.toSet === Set((1, ArrayBuffer(1, 2, 3, 1)), (2, ArrayBuffer(1)))) - // Check that the mutable objects in the original RDD were not changed - assert(bufs.collect().toSet === Set( - (1, ArrayBuffer(1)), - (1, ArrayBuffer(2)), - (1, ArrayBuffer(3)), - (1, ArrayBuffer(1)), - (2, ArrayBuffer(1)))) - } -} diff --git a/core/src/test/scala/org/apache/spark/rdd/IndexedRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/IndexedRDDSuite.scala deleted file mode 100644 index 3a2ce4e4da4c6..0000000000000 --- a/core/src/test/scala/org/apache/spark/rdd/IndexedRDDSuite.scala +++ /dev/null @@ -1,461 +0,0 @@ -/* - * 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.rdd - - -import org.scalatest.FunSuite -import org.scalatest.prop.Checkers -import org.scalacheck.Arbitrary._ -import org.scalacheck.Gen -import org.scalacheck.Prop._ - -import com.google.common.io.Files - -import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.HashSet - -import org.apache.spark.rdd.RDD -import org.apache.spark.rdd.ShuffledRDD -import org.apache.spark.rdd.IndexedRDD - -import org.apache.spark.SparkContext._ -import org.apache.spark._ - - - -class IndexedRDDSuite extends FunSuite with SharedSparkContext { - - def lineage(rdd: RDD[_]): collection.mutable.HashSet[RDD[_]] = { - val set = new collection.mutable.HashSet[RDD[_]] - def visit(rdd: RDD[_]) { - for (dep <- rdd.dependencies) { - set += dep.rdd - visit(dep.rdd) - } - } - visit(rdd) - set - } - - test("groupByKey") { - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1))).indexed() - val groups = pairs.groupByKey().collect() - assert(groups.size === 2) - val valuesFor1 = groups.find(_._1 == 1).get._2 - assert(valuesFor1.toList.sorted === List(1, 2, 3)) - val valuesFor2 = groups.find(_._1 == 2).get._2 - assert(valuesFor2.toList.sorted === List(1)) - } - - test("groupByKey with duplicates") { - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))).indexed() - val groups = pairs.groupByKey().collect() - assert(groups.size === 2) - val valuesFor1 = groups.find(_._1 == 1).get._2 - assert(valuesFor1.toList.sorted === List(1, 1, 2, 3)) - val valuesFor2 = groups.find(_._1 == 2).get._2 - assert(valuesFor2.toList.sorted === List(1)) - } - - test("groupByKey with negative key hash codes") { - val pairs = sc.parallelize(Array((-1, 1), (-1, 2), (-1, 3), (2, 1))).indexed() - val groups = pairs.groupByKey().collect() - assert(groups.size === 2) - val valuesForMinus1 = groups.find(_._1 == -1).get._2 - assert(valuesForMinus1.toList.sorted === List(1, 2, 3)) - val valuesFor2 = groups.find(_._1 == 2).get._2 - assert(valuesFor2.toList.sorted === List(1)) - } - - test("groupByKey with many output partitions") { - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1))).indexed(10) - val groups = pairs.groupByKey().collect() - assert(groups.size === 2) - val valuesFor1 = groups.find(_._1 == 1).get._2 - assert(valuesFor1.toList.sorted === List(1, 2, 3)) - val valuesFor2 = groups.find(_._1 == 2).get._2 - assert(valuesFor2.toList.sorted === List(1)) - } - - test("reduceByKey") { - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))).indexed() - val sums = pairs.reduceByKey(_+_).collect() - assert(sums.toSet === Set((1, 7), (2, 1))) - } - - test("reduceByKey with collectAsMap") { - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))).indexed() - val sums = pairs.reduceByKey(_+_).collectAsMap() - assert(sums.size === 2) - assert(sums(1) === 7) - assert(sums(2) === 1) - } - - test("reduceByKey with many output partitons") { - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))).indexed(10) - val sums = pairs.reduceByKey(_+_).collect() - assert(sums.toSet === Set((1, 7), (2, 1))) - } - - test("reduceByKey with partitioner") { - val p = new Partitioner() { - def numPartitions = 2 - def getPartition(key: Any) = key.asInstanceOf[Int] - } - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 1), (0, 1))).indexed(p) - val sums = pairs.reduceByKey(_+_) - assert(sums.collect().toSet === Set((1, 4), (0, 1))) - assert(sums.partitioner === Some(p)) - // count the dependencies to make sure there is only 1 ShuffledRDD - val deps = lineage(sums) - - assert(deps.filter(_.isInstanceOf[ShuffledRDD[_,_,_]]).size === 1) // ShuffledRDD, ParallelCollection - } - - - - test("joinIndexVsPair") { - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed() - val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) - val joined = rdd1.join(rdd2).collect() - assert(joined.size === 4) - assert(joined.toSet === Set( - (1, (1, 'x')), - (1, (2, 'x')), - (2, (1, 'y')), - (2, (1, 'z')) - )) - } - - test("joinIndexVsIndex") { - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed() - val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).indexed() - val joined = rdd1.join(rdd2).collect() - assert(joined.size === 4) - assert(joined.toSet === Set( - (1, (1, 'x')), - (1, (2, 'x')), - (2, (1, 'y')), - (2, (1, 'z')) - )) - } - - test("joinSharedIndex") { - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1), (4,-4), (4, 4) )).indexed() - val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).indexed(rdd1.index) - val joined = rdd1.join(rdd2).collect() - assert(joined.size === 6) - assert(joined.toSet === Set( - (1, (1, 'x')), - (1, (2, 'x')), - (2, (1, 'y')), - (2, (1, 'z')), - (4, (-4, 'w')), - (4, (4, 'w')) - )) - } - - - test("join all-to-all") { - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (1, 3))).indexed() - val rdd2 = sc.parallelize(Array((1, 'x'), (1, 'y'))).indexed(rdd1.index) - val joined = rdd1.join(rdd2).collect() - assert(joined.size === 6) - assert(joined.toSet === Set( - (1, (1, 'x')), - (1, (1, 'y')), - (1, (2, 'x')), - (1, (2, 'y')), - (1, (3, 'x')), - (1, (3, 'y')) - )) - } - - test("leftOuterJoinIndex") { - val index = sc.parallelize( 1 to 6 ).makeIndex() - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed(index) - val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) - val joined = rdd1.leftOuterJoin(rdd2).collect() - assert(joined.size === 5) - assert(joined.toSet === Set( - (1, (1, Some('x'))), - (1, (2, Some('x'))), - (2, (1, Some('y'))), - (2, (1, Some('z'))), - (3, (1, None)) - )) - } - - test("leftOuterJoinIndextoIndex") { - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed() - val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).indexed() - val joined = rdd1.leftOuterJoin(rdd2).collect() - assert(joined.size === 5) - assert(joined.toSet === Set( - (1, (1, Some('x'))), - (1, (2, Some('x'))), - (2, (1, Some('y'))), - (2, (1, Some('z'))), - (3, (1, None)) - )) - } - - test("leftOuterJoinIndextoSharedIndex") { - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1), (4, -4))).indexed() - val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).indexed(rdd1.index) - val joined = rdd1.leftOuterJoin(rdd2).collect() - assert(joined.size === 6) - assert(joined.toSet === Set( - (1, (1, Some('x'))), - (1, (2, Some('x'))), - (2, (1, Some('y'))), - (2, (1, Some('z'))), - (4, (-4, Some('w'))), - (3, (1, None)) - )) - } - -test("leftOuterJoinIndextoIndexExternal") { - val index = sc.parallelize( 1 to 6 ).makeIndex() - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed(index) - val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).indexed(index) - val joined = rdd1.leftOuterJoin(rdd2).collect() - assert(joined.size === 5) - assert(joined.toSet === Set( - (1, (1, Some('x'))), - (1, (2, Some('x'))), - (2, (1, Some('y'))), - (2, (1, Some('z'))), - (3, (1, None)) - )) - } - - - test("rightOuterJoin") { - val index = sc.parallelize( 1 to 6 ).makeIndex() - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed(index) - val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) - val joined = rdd1.rightOuterJoin(rdd2).collect() - assert(joined.size === 5) - assert(joined.toSet === Set( - (1, (Some(1), 'x')), - (1, (Some(2), 'x')), - (2, (Some(1), 'y')), - (2, (Some(1), 'z')), - (4, (None, 'w')) - )) - } - - test("rightOuterJoinIndex2Index") { - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed() - val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).indexed() - val joined = rdd1.rightOuterJoin(rdd2).collect() - assert(joined.size === 5) - assert(joined.toSet === Set( - (1, (Some(1), 'x')), - (1, (Some(2), 'x')), - (2, (Some(1), 'y')), - (2, (Some(1), 'z')), - (4, (None, 'w')) - )) - } - - - test("rightOuterJoinIndex2Indexshared") { - val index = sc.parallelize( 1 to 6 ).makeIndex() - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed(index) - val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).indexed(index) - val joined = rdd1.rightOuterJoin(rdd2).collect() - assert(joined.size === 5) - assert(joined.toSet === Set( - (1, (Some(1), 'x')), - (1, (Some(2), 'x')), - (2, (Some(1), 'y')), - (2, (Some(1), 'z')), - (4, (None, 'w')) - )) - } - - - test("join with no matches index") { - val index = IndexedRDD.makeIndex( sc.parallelize( 1 to 6 ) ) - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed(index) - val rdd2 = sc.parallelize(Array((4, 'x'), (5, 'y'), (5, 'z'), (6, 'w'))) - val joined = rdd1.join(rdd2).collect() - assert(joined.size === 0) - } - - test("join with no matches shared index") { - val index = IndexedRDD.makeIndex( sc.parallelize( 1 to 6 ) ) - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed(index) - val rdd2 = sc.parallelize(Array((4, 'x'), (5, 'y'), (5, 'z'), (6, 'w'))).indexed(index) - val joined = rdd1.join(rdd2).collect() - assert(joined.size === 0) - } - - - test("join with many output partitions") { - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed(10) - val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) - val joined = rdd1.join(rdd2).collect() - assert(joined.size === 4) - assert(joined.toSet === Set( - (1, (1, 'x')), - (1, (2, 'x')), - (2, (1, 'y')), - (2, (1, 'z')) - )) - } - - test("join with many output partitions and two indices") { - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed(10) - val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).indexed(20) - val joined = rdd1.join(rdd2).collect() - assert(joined.size === 4) - assert(joined.toSet === Set( - (1, (1, 'x')), - (1, (2, 'x')), - (2, (1, 'y')), - (2, (1, 'z')) - )) - } - - - test("groupWith") { - val index = IndexedRDD.makeIndex( sc.parallelize( 1 to 6 ) ) - - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed(index) - val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).indexed(index) - val joined = rdd1.groupWith(rdd2).collect() - assert(joined.size === 4) - assert(joined.toSet === Set( - (1, (ArrayBuffer(1, 2), ArrayBuffer('x'))), - (2, (ArrayBuffer(1), ArrayBuffer('y', 'z'))), - (3, (ArrayBuffer(1), ArrayBuffer())), - (4, (ArrayBuffer(), ArrayBuffer('w'))) - )) - } - - test("zero-partition RDD") { - val emptyDir = Files.createTempDir() - val file = sc.textFile(emptyDir.getAbsolutePath) - assert(file.partitions.size == 0) - assert(file.collect().toList === Nil) - // Test that a shuffle on the file works, because this used to be a bug - assert(file.map(line => (line, 1)).reduceByKey(_ + _).collect().toList === Nil) - } - - test("keys and values") { - val rdd = sc.parallelize(Array((1, "a"), (2, "b"))).indexed() - assert(rdd.keys.collect().toList === List(1, 2)) - assert(rdd.values.collect().toList === List("a", "b")) - } - - test("default partitioner uses partition size") { - // specify 2000 partitions - val a = sc.makeRDD(Array(1, 2, 3, 4), 2000) - // do a map, which loses the partitioner - val b = a.map(a => (a, (a * 2).toString)) - // then a group by, and see we didn't revert to 2 partitions - val c = b.groupByKey() - assert(c.partitions.size === 2000) - } - - // test("default partitioner uses largest partitioner indexed to indexed") { - // val a = sc.makeRDD(Array((1, "a"), (2, "b")), 2).indexed() - // val b = sc.makeRDD(Array((1, "a"), (2, "b")), 2000).indexed() - // val c = a.join(b) - // assert(c.partitions.size === 2000) - // } - - - - test("subtract") { - val a = sc.parallelize(Array(1, 2, 3), 2) - val b = sc.parallelize(Array(2, 3, 4), 4) - val c = a.subtract(b) - assert(c.collect().toSet === Set(1)) - assert(c.partitions.size === a.partitions.size) - } - - test("subtract with narrow dependency") { - // use a deterministic partitioner - val p = new Partitioner() { - def numPartitions = 5 - def getPartition(key: Any) = key.asInstanceOf[Int] - } - // partitionBy so we have a narrow dependency - val a = sc.parallelize(Array((1, "a"), (2, "b"), (3, "c"))).indexed(p) - // more partitions/no partitioner so a shuffle dependency - val b = sc.parallelize(Array((2, "b"), (3, "cc"), (4, "d")), 4) - val c = a.subtract(b) - assert(c.collect().toSet === Set((1, "a"), (3, "c"))) - // Ideally we could keep the original partitioner... - assert(c.partitioner === None) - } - - test("subtractByKey") { - - val a = sc.parallelize(Array((1, "a"), (1, "a"), (2, "b"), (3, "c")), 2).indexed() - val b = sc.parallelize(Array((2, 20), (3, 30), (4, 40)), 4) - val c = a.subtractByKey(b) - assert(c.collect().toSet === Set((1, "a"), (1, "a"))) - assert(c.partitions.size === a.partitions.size) - } - - // test("subtractByKey with narrow dependency") { - // // use a deterministic partitioner - // val p = new Partitioner() { - // def numPartitions = 5 - // def getPartition(key: Any) = key.asInstanceOf[Int] - // } - - // val index = sc.parallelize( 1 to 6 ).makeIndex(Some(p)) - // // partitionBy so we have a narrow dependency - // val a = sc.parallelize(Array((1, "a"), (1, "a"), (2, "b"), (3, "c"))).indexed(index) - // // more partitions/no partitioner so a shuffle dependency - // val b = sc.parallelize(Array((2, "b"), (3, "cc"), (4, "d")), 4).indexed(index) - // val c = a.subtractByKey(b) - // assert(c.collect().toSet === Set((1, "a"), (1, "a"))) - // assert(c.partitioner.get === p) - // } - - test("foldByKey") { - val index = IndexedRDD.makeIndex( sc.parallelize( 1 to 6 ) ) - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))).indexed(index) - val sums = pairs.foldByKey(0)(_+_).collect() - assert(sums.toSet === Set((1, 7), (2, 1))) - } - - test("foldByKey with mutable result type") { - val index = IndexedRDD.makeIndex( sc.parallelize( 1 to 6 ) ) - - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))).indexed(index) - val bufs = pairs.mapValues(v => ArrayBuffer(v)).cache() - // Fold the values using in-place mutation - val sums = bufs.foldByKey(new ArrayBuffer[Int])(_ ++= _).collect() - assert(sums.toSet === Set((1, ArrayBuffer(1, 2, 3, 1)), (2, ArrayBuffer(1)))) - // Check that the mutable objects in the original RDD were not changed - assert(bufs.collect().toSet === Set( - (1, ArrayBuffer(1)), - (1, ArrayBuffer(2)), - (1, ArrayBuffer(3)), - (1, ArrayBuffer(1)), - (2, ArrayBuffer(1)))) - } -} diff --git a/graph/src/main/scala/org/apache/spark/graph/Graph.scala b/graph/src/main/scala/org/apache/spark/graph/Graph.scala index 342151173a636..68f0394dd4bfa 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Graph.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Graph.scala @@ -306,6 +306,9 @@ object Graph { import org.apache.spark.graph.impl._ import org.apache.spark.SparkContext._ + def apply(rawEdges: RDD[(Vid, Vid)]): Graph[Int, Int] = { Graph(rawEdges, true) } + + /** * Construct a graph from a list of Edges. * @@ -316,7 +319,7 @@ object Graph { * * */ - def apply(rawEdges: RDD[(Vid, Vid)], uniqueEdges: Boolean = true): Graph[Int, Int] = { + def apply(rawEdges: RDD[(Vid, Vid)], uniqueEdges: Boolean): Graph[Int, Int] = { // Reduce to unique edges. val edges: RDD[Edge[Int]] = if (uniqueEdges) { @@ -330,13 +333,29 @@ object Graph { edges.flatMap{ case Edge(s, t, cnt) => Array((s, 1), (t, 1)) }.reduceByKey(_ + _) // Return graph - GraphImpl(vertices, edges) + GraphImpl(vertices, edges, 0) } + def apply[VD: ClassManifest, ED: ClassManifest]( - vertices: RDD[(Vid,VD)], edges: RDD[Edge[ED]]): Graph[VD, ED] = { - GraphImpl(vertices, edges) + vertices: RDD[(Vid,VD)], + edges: RDD[Edge[ED]]): Graph[VD, ED] = { + val defaultAttr: VD = null.asInstanceOf[VD] + Graph(vertices, edges, defaultAttr, (a:VD,b:VD) => a) + } + + + + /** + * Construct a new graph from a set of edges and vertices + */ + def apply[VD: ClassManifest, ED: ClassManifest]( + vertices: RDD[(Vid,VD)], + edges: RDD[Edge[ED]], + defaultVertexAttr: VD, + mergeFunc: (VD, VD) => VD): Graph[VD, ED] = { + GraphImpl(vertices, edges, defaultVertexAttr, mergeFunc) } implicit def graphToGraphOps[VD: ClassManifest, ED: ClassManifest](g: Graph[VD, ED]) = g.ops -} +} // end of Graph object diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala b/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala index 76f69edf0e22f..2295084024f9b 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala @@ -56,6 +56,6 @@ object GraphLoader { val vertices = edges.flatMap { edge => List((edge.srcId, 1), (edge.dstId, 1)) } .reduceByKey(_ + _) .map{ case (vid, degree) => (vid, degree) } - GraphImpl(vertices, edges) + GraphImpl(vertices, edges, 0) } } diff --git a/core/src/main/scala/org/apache/spark/rdd/IndexedRDD.scala b/graph/src/main/scala/org/apache/spark/graph/IndexedRDD.scala similarity index 73% rename from core/src/main/scala/org/apache/spark/rdd/IndexedRDD.scala rename to graph/src/main/scala/org/apache/spark/graph/IndexedRDD.scala index 5f95559f15122..900a46bb42362 100644 --- a/core/src/main/scala/org/apache/spark/rdd/IndexedRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/IndexedRDD.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.rdd +package org.apache.spark.graph import java.nio.ByteBuffer @@ -76,6 +76,12 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( List(new OneToOneDependency(index.rdd), new OneToOneDependency(valuesRDD)) ) { + /** + * Construct a new IndexedRDD that is indexed by only the keys in the RDD + */ + def reindex(): IndexedRDD[K,V] = IndexedRDD(this) + + /** * An internal representation which joins the block indices with the values */ @@ -247,6 +253,173 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( } +/** + * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the + * list of values for that key in `this` as well as `other`. + */ + def cogroup[W: ClassManifest](other: RDD[(K, W)], partitioner: Partitioner): + IndexedRDD[K, (Seq[V], Seq[W])] = { + //RDD[(K, (Seq[V], Seq[W]))] = { + other match { + case other: IndexedRDD[_, _] if index == other.index => { + // if both RDDs share exactly the same index and therefore the same super set of keys + // then we simply merge the value RDDs. + // However it is possible that both RDDs are missing a value for a given key in + // which case the returned RDD should have a null value + val newValues: RDD[(IndexedSeq[(Seq[V], Seq[W])], BitSet)] = + valuesRDD.zipPartitions(other.valuesRDD){ + (thisIter, otherIter) => + val (thisValues, thisBS) = thisIter.next() + assert(!thisIter.hasNext) + val (otherValues, otherBS) = otherIter.next() + assert(!otherIter.hasNext) + + val newValues = new Array[(Seq[V], Seq[W])](thisValues.size) + val newBS = thisBS | otherBS + + for( ind <- newBS ) { + val a = if (thisBS(ind)) Seq(thisValues(ind)) else Seq.empty[V] + val b = if (otherBS(ind)) Seq(otherValues(ind)) else Seq.empty[W] + newValues(ind) = (a, b) + } + Iterator((newValues.toIndexedSeq, newBS)) + } + new IndexedRDD(index, newValues) + } + case other: IndexedRDD[_, _] + if index.rdd.partitioner == other.index.rdd.partitioner => { + // If both RDDs are indexed using different indices but with the same partitioners + // then we we need to first merge the indicies and then use the merged index to + // merge the values. + val newIndex = + index.rdd.zipPartitions(other.index.rdd)( + (thisIter, otherIter) => { + val thisIndex = thisIter.next() + assert(!thisIter.hasNext) + val otherIndex = otherIter.next() + assert(!otherIter.hasNext) + val newIndex = new BlockIndex[K]() + // @todo Merge only the keys that correspond to non-null values + // Merge the keys + newIndex.putAll(thisIndex) + newIndex.putAll(otherIndex) + // We need to rekey the index + var ctr = 0 + for (e <- newIndex.entrySet) { + e.setValue(ctr) + ctr += 1 + } + List(newIndex).iterator + }).cache() + // Use the new index along with the this and the other indices to merge the values + val newValues: RDD[(IndexedSeq[(Seq[V], Seq[W])], BitSet)] = + newIndex.zipPartitions(tuples, other.tuples)( + (newIndexIter, thisTuplesIter, otherTuplesIter) => { + // Get the new index for this partition + val newIndex = newIndexIter.next() + assert(!newIndexIter.hasNext) + // Get the corresponding indicies and values for this and the other IndexedRDD + val (thisIndex, (thisValues, thisBS)) = thisTuplesIter.next() + assert(!thisTuplesIter.hasNext) + val (otherIndex, (otherValues, otherBS)) = otherTuplesIter.next() + assert(!otherTuplesIter.hasNext) + // Preallocate the new Values array + val newValues = new Array[(Seq[V], Seq[W])](newIndex.size) + val newBS = new BitSet(newIndex.size) + + // Lookup the sequences in both submaps + for ((k,ind) <- newIndex) { + // Get the left key + val a = if (thisIndex.contains(k)) { + val ind = thisIndex.get(k) + if(thisBS(ind)) Seq(thisValues(ind)) else Seq.empty[V] + } else Seq.empty[V] + // Get the right key + val b = if (otherIndex.contains(k)) { + val ind = otherIndex.get(k) + if (otherBS(ind)) Seq(otherValues(ind)) else Seq.empty[W] + } else Seq.empty[W] + // If at least one key was present then we generate a tuple. + if (!a.isEmpty || !b.isEmpty) { + newValues(ind) = (a, b) + newBS(ind) = true + } + } + Iterator((newValues.toIndexedSeq, newBS)) + }) + new IndexedRDD(new RDDIndex(newIndex), newValues) + } + case _ => { + // Get the partitioner from the index + val partitioner = index.rdd.partitioner match { + case Some(p) => p + case None => throw new SparkException("An index must have a partitioner.") + } + // Shuffle the other RDD using the partitioner for this index + val otherShuffled = + if (other.partitioner == Some(partitioner)) { + other + } else { + new ShuffledRDD[K, W, (K,W)](other, partitioner) + } + // Join the other RDD with this RDD building a new valueset and new index on the fly + val groups = tuples.zipPartitions(otherShuffled)( + (thisTuplesIter, otherTuplesIter) => { + // Get the corresponding indicies and values for this IndexedRDD + val (thisIndex, (thisValues, thisBS)) = thisTuplesIter.next() + assert(!thisTuplesIter.hasNext()) + // Construct a new index + val newIndex = thisIndex.clone().asInstanceOf[BlockIndex[K]] + // Construct a new array Buffer to store the values + val newValues = ArrayBuffer.fill[ (Seq[V], Seq[W]) ](thisValues.size)(null) + val newBS = new BitSet(thisValues.size) + // populate the newValues with the values in this IndexedRDD + for ((k,i) <- thisIndex) { + if (thisBS(i)) { + newValues(i) = (Seq(thisValues(i)), ArrayBuffer.empty[W]) + newBS(i) = true + } + } + // Now iterate through the other tuples updating the map + for ((k,w) <- otherTuplesIter){ + if (newIndex.contains(k)) { + val ind = newIndex.get(k) + if(newBS(ind)) { + newValues(ind)._2.asInstanceOf[ArrayBuffer[W]].append(w) + } else { + // If the other key was in the index but not in the values + // of this indexed RDD then create a new values entry for it + newBS(ind) = true + newValues(ind) = (Seq.empty[V], ArrayBuffer(w)) + } + } else { + // update the index + val ind = newIndex.size + newIndex.put(k, ind) + newBS(ind) = true + // Update the values + newValues.append( (Seq.empty[V], ArrayBuffer(w) ) ) + } + } + // // Finalize the new values array + // val newValuesArray: Seq[Seq[(Seq[V],Seq[W])]] = + // newValues.view.map{ + // case null => null + // case (s, ab) => Seq((s, ab.toSeq)) + // }.toSeq + Iterator( (newIndex, (newValues.toIndexedSeq, newBS)) ) + }).cache() + + // Extract the index and values from the above RDD + val newIndex = groups.mapPartitions(_.map{ case (kMap,vAr) => kMap }, true) + val newValues: RDD[(IndexedSeq[(Seq[V], Seq[W])], BitSet)] = + groups.mapPartitions(_.map{ case (kMap,vAr) => vAr }, true) + + new IndexedRDD[K, (Seq[V], Seq[W])](new RDDIndex(newIndex), newValues) + } + } + } + // // def zipJoinToRDD[W: ClassManifest](other: IndexedRDD[K,W]): RDD[(K,(V,W))] = { @@ -419,15 +592,6 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( */ - /** - * The IndexedRDD has its own optimized version of the pairRDDFunctions. - */ - override def pairRDDFunctions[K1, V1]( - implicit t: (K, V) <:< (K1,V1), k: ClassManifest[K1], v: ClassManifest[V1]): - PairRDDFunctions[K1, V1] = { - new IndexedRDDFunctions[K1,V1](this.asInstanceOf[IndexedRDD[K1,V1]]) - } - override def filter(f: Tuple2[K,V] => Boolean): RDD[(K,V)] = { val cleanF = index.rdd.context.clean(f) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index e7a708e895b28..c6875f0c9c470 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -15,8 +15,6 @@ import org.apache.spark.util.ClosureCleaner import org.apache.spark.rdd import org.apache.spark.rdd.RDD -import org.apache.spark.rdd.IndexedRDD -import org.apache.spark.rdd.RDDIndex import org.apache.spark.graph._ @@ -239,7 +237,8 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( // .map(v => (v.id, v.data._1)).indexed() // Reuse the partitioner (but not the index) from this graph - val newVTable = vertices.filter(v => vpred(v._1, v._2)).indexed(vTable.index.partitioner) + val newVTable = + IndexedRDD(vertices.filter(v => vpred(v._1, v._2)).partitionBy(vTable.index.partitioner)) // Restrict the set of edges to those that satisfy the vertex and the edge predicate. @@ -247,9 +246,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( triplets.filter( t => vpred( t.srcId, t.srcAttr ) && vpred( t.dstId, t.dstAttr ) && epred(t) ) - .map( t => Edge(t.srcId, t.dstId, t.attr) ), - eTable.index.partitioner.numPartitions - ) + .map( t => Edge(t.srcId, t.dstId, t.attr) )) // Construct the Vid2Pid map. Here we assume that the filter operation // behaves deterministically. @@ -277,11 +274,11 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( .toList .toIterator .map { case ((src, dst), data) => Edge(src, dst, data) } + .toIterator } //TODO(crankshaw) eliminate the need to call createETable - val newETable = createETable(newEdges, - eTable.index.partitioner.numPartitions) + val newETable = createETable(newEdges) new GraphImpl(vTable, vid2pid, localVidMap, newETable) } @@ -298,8 +295,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( .map { case ((src, dst), data) => Edge(src, dst, data) } } // TODO(crankshaw) eliminate the need to call createETable - val newETable = createETable(newEdges, - eTable.index.partitioner.numPartitions) + val newETable = createETable(newEdges) new GraphImpl(vTable, vid2pid, localVidMap, newETable) } @@ -393,20 +389,33 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( object GraphImpl { def apply[VD: ClassManifest, ED: ClassManifest]( - vertices: RDD[(Vid, VD)], edges: RDD[Edge[ED]]): + vertices: RDD[(Vid, VD)], edges: RDD[Edge[ED]], + defaultVertexAttr: VD): GraphImpl[VD,ED] = { - - apply(vertices, edges, - vertices.context.defaultParallelism, edges.context.defaultParallelism) + apply(vertices, edges, defaultVertexAttr, (a:VD, b:VD) => a) } def apply[VD: ClassManifest, ED: ClassManifest]( - vertices: RDD[(Vid, VD)], edges: RDD[Edge[ED]], - numVPart: Int, numEPart: Int): GraphImpl[VD,ED] = { - - val vtable = vertices.indexed(numVPart) - val etable = createETable(edges, numEPart) + vertices: RDD[(Vid, VD)], + edges: RDD[Edge[ED]], + defaultVertexAttr: VD, + mergeFunc: (VD, VD) => VD): GraphImpl[VD,ED] = { + + val vtable = IndexedRDD(vertices, mergeFunc) + /** + * @todo Verify that there are no edges that contain vertices + * that are not in vTable. This should probably be resolved: + * + * edges.flatMap{ e => Array((e.srcId, null), (e.dstId, null)) } + * .cogroup(vertices).map{ + * case (vid, _, attr) => + * if (attr.isEmpty) (vid, defaultValue) + * else (vid, attr) + * } + * + */ + val etable = createETable(edges) val vid2pid = createVid2Pid(etable, vtable.index) val localVidMap = createLocalVidMap(etable) new GraphImpl(vtable, vid2pid, localVidMap, etable) @@ -422,12 +431,12 @@ object GraphImpl { * key-value pair: the key is the partition id, and the value is an EdgePartition object * containing all the edges in a partition. */ - protected def createETable[ED: ClassManifest]( - edges: RDD[Edge[ED]], numPartitions: Int) + protected def createETable[ED: ClassManifest](edges: RDD[Edge[ED]]) : IndexedRDD[Pid, EdgePartition[ED]] = { + // Get the number of partitions + val numPartitions = edges.partitions.size val ceilSqrt: Pid = math.ceil(math.sqrt(numPartitions)).toInt - edges - .map { e => + IndexedRDD(edges.map { e => // Random partitioning based on the source vertex id. // val part: Pid = edgePartitionFunction1D(e.srcId, e.dstId, numPartitions) // val part: Pid = edgePartitionFunction2D(e.srcId, e.dstId, numPartitions, ceilSqrt) @@ -446,7 +455,7 @@ object GraphImpl { } val edgePartition = builder.toEdgePartition Iterator((pid, edgePartition)) - }, preservesPartitioning = true).indexed() + }, preservesPartitioning = true)) } diff --git a/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala b/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala index 061cce99b6e3d..895c65c14c903 100644 --- a/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala +++ b/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala @@ -79,7 +79,7 @@ object GraphGenerators { v => generateRandomEdges(v._1.toInt, v._2, numVertices) } - GraphImpl(vertices, edges) + GraphImpl(vertices, edges, 0) //println("Vertices:") //for (v <- vertices) { // println(v.id) @@ -160,7 +160,7 @@ object GraphGenerators { val vertices = edges.flatMap { edge => List((edge.srcId, 1)) } .reduceByKey(_ + _) .map{ case (vid, degree) => (vid, degree) } - GraphImpl(vertices, edges) + GraphImpl(vertices, edges, 0) } /** From 5d01ebca3c7dcf9695c9abb5a7690b547ed83cfe Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Fri, 18 Oct 2013 18:45:10 -0700 Subject: [PATCH 130/531] Specializing IndexedRDD as VertexSetRDD. 1) This allows the index map to be optimized for Vids 2) This makes the code more readable 2) The Graph API can now return VertexSetRDDs from operations that produce results for vertices --- .../scala/org/apache/spark/graph/Graph.scala | 4 +- .../org/apache/spark/graph/GraphOps.scala | 14 +- .../{IndexedRDD.scala => VertexSetRDD.scala} | 167 ++++++------ .../apache/spark/graph/impl/GraphImpl.scala | 238 +++++++++--------- 4 files changed, 213 insertions(+), 210 deletions(-) rename graph/src/main/scala/org/apache/spark/graph/{IndexedRDD.scala => VertexSetRDD.scala} (85%) diff --git a/graph/src/main/scala/org/apache/spark/graph/Graph.scala b/graph/src/main/scala/org/apache/spark/graph/Graph.scala index 68f0394dd4bfa..8c7ee1fcefc86 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Graph.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Graph.scala @@ -28,7 +28,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * @see Vertex for the vertex type. * */ - val vertices: RDD[(Vid,VD)] + val vertices: VertexSetRDD[VD] /** * Get the Edges and their data as an RDD. The entries in the RDD contain @@ -257,7 +257,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { def mapReduceTriplets[A: ClassManifest]( mapFunc: EdgeTriplet[VD, ED] => Array[(Vid, A)], reduceFunc: (A, A) => A) - : RDD[(Vid, A)] + : VertexSetRDD[A] /** diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala b/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala index 5e8f082fdad8a..cecd3ff2913fc 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala @@ -13,11 +13,11 @@ class GraphOps[VD: ClassManifest, ED: ClassManifest](graph: Graph[VD, ED]) { lazy val numVertices: Long = graph.vertices.count() - lazy val inDegrees: RDD[(Vid, Int)] = degreesRDD(EdgeDirection.In) + lazy val inDegrees: VertexSetRDD[Int] = degreesRDD(EdgeDirection.In) - lazy val outDegrees: RDD[(Vid, Int)] = degreesRDD(EdgeDirection.Out) + lazy val outDegrees: VertexSetRDD[Int] = degreesRDD(EdgeDirection.Out) - lazy val degrees: RDD[(Vid, Int)] = degreesRDD(EdgeDirection.Both) + lazy val degrees: VertexSetRDD[Int] = degreesRDD(EdgeDirection.Both) /** @@ -62,7 +62,7 @@ class GraphOps[VD: ClassManifest, ED: ClassManifest](graph: Graph[VD, ED]) { mapFunc: (Vid, EdgeTriplet[VD, ED]) => Option[A], reduceFunc: (A, A) => A, dir: EdgeDirection) - : RDD[(Vid, A)] = { + : VertexSetRDD[A] = { ClosureCleaner.clean(mapFunc) ClosureCleaner.clean(reduceFunc) @@ -94,20 +94,20 @@ class GraphOps[VD: ClassManifest, ED: ClassManifest](graph: Graph[VD, ED]) { } // end of aggregateNeighbors - def collectNeighborIds(edgeDirection: EdgeDirection) : RDD[(Vid, Array[Vid])] = { + def collectNeighborIds(edgeDirection: EdgeDirection) : VertexSetRDD[Array[Vid]] = { val nbrs = graph.aggregateNeighbors[Array[Vid]]( (vid, edge) => Some(Array(edge.otherVertexId(vid))), (a, b) => a ++ b, edgeDirection) - graph.vertices.leftOuterJoin(nbrs).mapValues{ + graph.vertices.leftZipJoin(nbrs).mapValues{ case (_, Some(nbrs)) => nbrs case (_, None) => Array.empty[Vid] } } - private def degreesRDD(edgeDirection: EdgeDirection): RDD[(Vid, Int)] = { + private def degreesRDD(edgeDirection: EdgeDirection): VertexSetRDD[Int] = { graph.aggregateNeighbors((vid, edge) => Some(1), _+_, edgeDirection) } diff --git a/graph/src/main/scala/org/apache/spark/graph/IndexedRDD.scala b/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala similarity index 85% rename from graph/src/main/scala/org/apache/spark/graph/IndexedRDD.scala rename to graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala index 900a46bb42362..b3f1fa768c86d 100644 --- a/graph/src/main/scala/org/apache/spark/graph/IndexedRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala @@ -39,47 +39,40 @@ import org.apache.spark.storage.StorageLevel -/** - * The BlockIndex is the internal map structure used inside the index - * of the IndexedRDD. - */ -class BlockIndex[@specialized K: ClassManifest] extends JHashMap[K,Int] - /** - * The RDDIndex is an opaque type used to represent the organization + * The VertexSetIndex is an opaque type used to represent the organization * of values in an RDD */ -class RDDIndex[@specialized K: ClassManifest](private[spark] val rdd: RDD[BlockIndex[K]]) { - def persist(newLevel: StorageLevel): RDDIndex[K] = { +class VertexSetIndex(private[spark] val rdd: RDD[VertexIdToIndexMap]) { + def persist(newLevel: StorageLevel): VertexSetIndex = { rdd.persist(newLevel) return this } - def partitioner: Partitioner = rdd.partitioner.get } /** - * An IndexedRDD[K,V] extends the RDD[(K,V)] by pre-indexing the keys and + * An VertexSetRDD[V] extends the RDD[(Vid,V)] by pre-indexing the keys and * organizing the values to enable faster join operations. * - * In addition to providing the basic RDD[(K,V)] functionality the IndexedRDD - * exposes an index member which can be used to "key" other IndexedRDDs + * In addition to providing the basic RDD[(Vid,V)] functionality the VertexSetRDD + * exposes an index member which can be used to "key" other VertexSetRDDs * */ -class IndexedRDD[K: ClassManifest, V: ClassManifest]( - @transient val index: RDDIndex[K], +class VertexSetRDD[V: ClassManifest]( + @transient val index: VertexSetIndex, @transient val valuesRDD: RDD[ (IndexedSeq[V], BitSet) ]) - extends RDD[(K, V)](index.rdd.context, + extends RDD[(Vid, V)](index.rdd.context, List(new OneToOneDependency(index.rdd), new OneToOneDependency(valuesRDD)) ) { /** - * Construct a new IndexedRDD that is indexed by only the keys in the RDD + * Construct a new VertexSetRDD that is indexed by only the keys in the RDD */ - def reindex(): IndexedRDD[K,V] = IndexedRDD(this) + def reindex(): VertexSetRDD[V] = VertexSetRDD(this) /** @@ -109,20 +102,26 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( /** - * Caching an IndexedRDD causes the index and values to be cached separately. + * Caching an VertexSetRDD causes the index and values to be cached separately. */ - override def persist(newLevel: StorageLevel): RDD[(K,V)] = { + override def persist(newLevel: StorageLevel): VertexSetRDD[V] = { index.persist(newLevel) valuesRDD.persist(newLevel) return this } + override def persist(): VertexSetRDD[V] = persist(StorageLevel.MEMORY_ONLY) + + /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */ + override def cache(): VertexSetRDD[V] = persist() + + /** * Pass each value in the key-value pair RDD through a map function without changing the keys; * this also retains the original RDD's partitioning. */ - def mapValues[U: ClassManifest](f: V => U): IndexedRDD[K, U] = { + def mapValues[U: ClassManifest](f: V => U): VertexSetRDD[U] = { val cleanF = index.rdd.context.clean(f) val newValuesRDD: RDD[ (IndexedSeq[U], BitSet) ] = valuesRDD.mapPartitions(iter => iter.map{ @@ -133,7 +132,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( } (newValues.toIndexedSeq, bs) }, preservesPartitioning = true) - new IndexedRDD[K,U](index, newValuesRDD) + new VertexSetRDD[U](index, newValuesRDD) } @@ -141,7 +140,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( * Pass each value in the key-value pair RDD through a map function without changing the keys; * this also retains the original RDD's partitioning. */ - def mapValuesWithKeys[U: ClassManifest](f: (K, V) => U): IndexedRDD[K, U] = { + def mapValuesWithKeys[U: ClassManifest](f: (Vid, V) => U): VertexSetRDD[U] = { val cleanF = index.rdd.context.clean(f) val newValues: RDD[ (IndexedSeq[U], BitSet) ] = index.rdd.zipPartitions(valuesRDD){ @@ -158,11 +157,11 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( } Array((newValues.toIndexedSeq, bs)).iterator } - new IndexedRDD[K,U](index, newValues) + new VertexSetRDD[U](index, newValues) } - def zipJoin[W: ClassManifest](other: IndexedRDD[K,W]): IndexedRDD[K,(V,W)] = { + def zipJoin[W: ClassManifest](other: VertexSetRDD[W]): VertexSetRDD[(V,W)] = { if(index != other.index) { throw new SparkException("A zipJoin can only be applied to RDDs with the same index!") } @@ -176,11 +175,11 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( val newValues = thisValues.view.zip(otherValues) Iterator((newValues.toIndexedSeq, newBS)) } - new IndexedRDD(index, newValuesRDD) + new VertexSetRDD(index, newValuesRDD) } - def leftZipJoin[W: ClassManifest](other: IndexedRDD[K,W]): IndexedRDD[K,(V,Option[W])] = { + def leftZipJoin[W: ClassManifest](other: VertexSetRDD[W]): VertexSetRDD[(V,Option[W])] = { if(index != other.index) { throw new SparkException("A zipJoin can only be applied to RDDs with the same index!") } @@ -195,18 +194,18 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( val newValues = thisValues.view.zip(otherOption) Iterator((newValues.toIndexedSeq, thisBS)) } - new IndexedRDD(index, newValuesRDD) + new VertexSetRDD(index, newValuesRDD) } def leftJoin[W: ClassManifest]( - other: RDD[(K,W)], merge: (W,W) => W = (a:W, b:W) => a): - IndexedRDD[K, (V, Option[W]) ] = { + other: RDD[(Vid,W)], merge: (W,W) => W = (a:W, b:W) => a): + VertexSetRDD[(V, Option[W]) ] = { val cleanMerge = index.rdd.context.clean(merge) other match { - case other: IndexedRDD[_, _] if index == other.index => { + case other: VertexSetRDD[_] if index == other.index => { leftZipJoin(other) } case _ => { @@ -247,21 +246,21 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( Iterator((newValues.toIndexedSeq, thisBS)) } // end of newValues - new IndexedRDD(index, newValues) + new VertexSetRDD(index, newValues) } } } -/** + /** * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the * list of values for that key in `this` as well as `other`. */ - def cogroup[W: ClassManifest](other: RDD[(K, W)], partitioner: Partitioner): - IndexedRDD[K, (Seq[V], Seq[W])] = { + def cogroup[W: ClassManifest](other: RDD[(Vid, W)], partitioner: Partitioner): + VertexSetRDD[(Seq[V], Seq[W])] = { //RDD[(K, (Seq[V], Seq[W]))] = { other match { - case other: IndexedRDD[_, _] if index == other.index => { + case other: VertexSetRDD[_] if index == other.index => { // if both RDDs share exactly the same index and therefore the same super set of keys // then we simply merge the value RDDs. // However it is possible that both RDDs are missing a value for a given key in @@ -284,9 +283,9 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( } Iterator((newValues.toIndexedSeq, newBS)) } - new IndexedRDD(index, newValues) + new VertexSetRDD(index, newValues) } - case other: IndexedRDD[_, _] + case other: VertexSetRDD[_] if index.rdd.partitioner == other.index.rdd.partitioner => { // If both RDDs are indexed using different indices but with the same partitioners // then we we need to first merge the indicies and then use the merged index to @@ -298,7 +297,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( assert(!thisIter.hasNext) val otherIndex = otherIter.next() assert(!otherIter.hasNext) - val newIndex = new BlockIndex[K]() + val newIndex = new VertexIdToIndexMap() // @todo Merge only the keys that correspond to non-null values // Merge the keys newIndex.putAll(thisIndex) @@ -318,7 +317,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( // Get the new index for this partition val newIndex = newIndexIter.next() assert(!newIndexIter.hasNext) - // Get the corresponding indicies and values for this and the other IndexedRDD + // Get the corresponding indicies and values for this and the other VertexSetRDD val (thisIndex, (thisValues, thisBS)) = thisTuplesIter.next() assert(!thisTuplesIter.hasNext) val (otherIndex, (otherValues, otherBS)) = otherTuplesIter.next() @@ -347,7 +346,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( } Iterator((newValues.toIndexedSeq, newBS)) }) - new IndexedRDD(new RDDIndex(newIndex), newValues) + new VertexSetRDD(new VertexSetIndex(newIndex), newValues) } case _ => { // Get the partitioner from the index @@ -360,20 +359,20 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( if (other.partitioner == Some(partitioner)) { other } else { - new ShuffledRDD[K, W, (K,W)](other, partitioner) + other.partitionBy(partitioner) } // Join the other RDD with this RDD building a new valueset and new index on the fly val groups = tuples.zipPartitions(otherShuffled)( (thisTuplesIter, otherTuplesIter) => { - // Get the corresponding indicies and values for this IndexedRDD + // Get the corresponding indicies and values for this VertexSetRDD val (thisIndex, (thisValues, thisBS)) = thisTuplesIter.next() assert(!thisTuplesIter.hasNext()) // Construct a new index - val newIndex = thisIndex.clone().asInstanceOf[BlockIndex[K]] + val newIndex = thisIndex.clone().asInstanceOf[VertexIdToIndexMap] // Construct a new array Buffer to store the values val newValues = ArrayBuffer.fill[ (Seq[V], Seq[W]) ](thisValues.size)(null) val newBS = new BitSet(thisValues.size) - // populate the newValues with the values in this IndexedRDD + // populate the newValues with the values in this VertexSetRDD for ((k,i) <- thisIndex) { if (thisBS(i)) { newValues(i) = (Seq(thisValues(i)), ArrayBuffer.empty[W]) @@ -415,14 +414,14 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( val newValues: RDD[(IndexedSeq[(Seq[V], Seq[W])], BitSet)] = groups.mapPartitions(_.map{ case (kMap,vAr) => vAr }, true) - new IndexedRDD[K, (Seq[V], Seq[W])](new RDDIndex(newIndex), newValues) + new VertexSetRDD[(Seq[V], Seq[W])](new VertexSetIndex(newIndex), newValues) } } } // - // def zipJoinToRDD[W: ClassManifest](other: IndexedRDD[K,W]): RDD[(K,(V,W))] = { + // def zipJoinToRDD[W: ClassManifest](other: VertexSetRDD[K,W]): RDD[(K,(V,W))] = { // if(index != other.index) { // throw new SparkException("ZipJoinRDD can only be applied to RDDs with the same index!") // } @@ -447,11 +446,11 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( other: RDD[(K,W)])( f: (K, V, W) => Z, merge: (Z,Z) => Z = (a:Z, b:Z) => a): - IndexedRDD[K,Z] = { + VertexSetRDD[K,Z] = { val cleanF = index.rdd.context.clean(f) val cleanMerge = index.rdd.context.clean(merge) other match { - case other: IndexedRDD[_, _] if index == other.index => { + case other: VertexSetRDD[_, _] if index == other.index => { val newValues = index.rdd.zipPartitions(valuesRDD, other.valuesRDD){ (thisIndexIter, thisIter, otherIter) => val index = thisIndexIter.next() @@ -469,7 +468,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( } List((newValues, newBS)).iterator } - new IndexedRDD(index, newValues) + new VertexSetRDD(index, newValues) } case _ => { @@ -508,7 +507,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( } List((newValues, tempBS)).iterator } // end of newValues - new IndexedRDD(index, newValues) + new VertexSetRDD(index, newValues) } } } @@ -519,11 +518,11 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( other: RDD[(K,W)])( f: (K, V, Option[W]) => Z, merge: (Z,Z) => Z = (a:Z, b:Z) => a): - IndexedRDD[K,Z] = { + VertexSetRDD[K,Z] = { val cleanF = index.rdd.context.clean(f) val cleanMerge = index.rdd.context.clean(merge) other match { - case other: IndexedRDD[_, _] if index == other.index => { + case other: VertexSetRDD[_, _] if index == other.index => { val newValues = index.rdd.zipPartitions(valuesRDD, other.valuesRDD){ (thisIndexIter, thisIter, otherIter) => val index = thisIndexIter.next() @@ -541,7 +540,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( } List((newValues, thisBS)).iterator } - new IndexedRDD(index, newValues) + new VertexSetRDD(index, newValues) } case _ => { @@ -584,7 +583,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( } List((newValues, thisBS)).iterator } // end of newValues - new IndexedRDD(index, newValues) + new VertexSetRDD(index, newValues) } } } @@ -593,7 +592,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( - override def filter(f: Tuple2[K,V] => Boolean): RDD[(K,V)] = { + override def filter(f: Tuple2[Vid,V] => Boolean): VertexSetRDD[V] = { val cleanF = index.rdd.context.clean(f) val newValues = index.rdd.zipPartitions(valuesRDD){ (keysIter, valuesIter) => @@ -609,14 +608,14 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( } Array((oldValues, newBS)).iterator } - new IndexedRDD[K,V](index, newValues) + new VertexSetRDD[V](index, newValues) } /** * Provide the RDD[(K,V)] equivalent output. */ - override def compute(part: Partition, context: TaskContext): Iterator[(K, V)] = { + override def compute(part: Partition, context: TaskContext): Iterator[(Vid, V)] = { tuples.compute(part, context).flatMap { case (indexMap, (values, bs) ) => // Walk the index to construct the key, value pairs indexMap.iterator @@ -629,27 +628,27 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( } } -} // End of IndexedRDD +} // End of VertexSetRDD -object IndexedRDD { +object VertexSetRDD { - def apply[K: ClassManifest, V: ClassManifest](rdd: RDD[(K,V)]): IndexedRDD[K,V] = + def apply[V: ClassManifest](rdd: RDD[(Vid,V)]): VertexSetRDD[V] = apply(rdd, (a:V, b:V) => a ) - def apply[K: ClassManifest, V: ClassManifest]( - rdd: RDD[(K,V)], reduceFunc: (V, V) => V): IndexedRDD[K,V] = { + def apply[V: ClassManifest]( + rdd: RDD[(Vid,V)], reduceFunc: (V, V) => V): VertexSetRDD[V] = { // Preaggregate and shuffle if necessary // Preaggregation. - val aggregator = new Aggregator[K, V, V](v => v, reduceFunc, reduceFunc) + val aggregator = new Aggregator[Vid, V, V](v => v, reduceFunc, reduceFunc) val partitioner = new HashPartitioner(rdd.partitions.size) val preAgg = rdd.mapPartitions(aggregator.combineValuesByKey).partitionBy(partitioner) val groups = preAgg.mapPartitions( iter => { - val indexMap = new BlockIndex[K]() + val indexMap = new VertexIdToIndexMap() val values = new ArrayBuffer[V] val bs = new BitSet for ((k,v) <- iter) { @@ -669,19 +668,19 @@ object IndexedRDD { val index = groups.mapPartitions(_.map{ case (kMap, vAr) => kMap }, true) val values: RDD[(IndexedSeq[V], BitSet)] = groups.mapPartitions(_.map{ case (kMap,vAr) => vAr }, true) - new IndexedRDD[K,V](new RDDIndex(index), values) + new VertexSetRDD[V](new VertexSetIndex(index), values) } - def apply[K: ClassManifest, V: ClassManifest]( - rdd: RDD[(K,V)], index: RDDIndex[K]): IndexedRDD[K,V] = + def apply[V: ClassManifest]( + rdd: RDD[(Vid,V)], index: VertexSetIndex): VertexSetRDD[V] = apply(rdd, index, (a:V,b:V) => a) - def apply[K: ClassManifest, V: ClassManifest]( - rdd: RDD[(K,V)], index: RDDIndex[K], - reduceFunc: (V, V) => V): IndexedRDD[K,V] = + def apply[V: ClassManifest]( + rdd: RDD[(Vid,V)], index: VertexSetIndex, + reduceFunc: (V, V) => V): VertexSetRDD[V] = apply(rdd,index, (v:V) => v, reduceFunc, reduceFunc) // { // // Get the index Partitioner @@ -721,16 +720,16 @@ object IndexedRDD { // } // List((values, bs)).iterator // }) - // new IndexedRDD[K,V](index, values) + // new VertexSetRDD[K,V](index, values) // } // end of apply - def apply[K: ClassManifest, V: ClassManifest, C: ClassManifest]( - rdd: RDD[(K,V)], - index: RDDIndex[K], + def apply[V: ClassManifest, C: ClassManifest]( + rdd: RDD[(Vid,V)], + index: VertexSetIndex, createCombiner: V => C, mergeValue: (C, V) => C, - mergeCombiners: (C, C) => C): IndexedRDD[K,C] = { + mergeCombiners: (C, C) => C): VertexSetRDD[C] = { // Get the index Partitioner val partitioner = index.rdd.partitioner match { case Some(p) => p @@ -740,7 +739,7 @@ object IndexedRDD { val partitioned = if (rdd.partitioner != Some(partitioner)) { // Preaggregation. - val aggregator = new Aggregator[K, V, C](createCombiner, mergeValue, + val aggregator = new Aggregator[Vid, V, C](createCombiner, mergeValue, mergeCombiners) rdd.mapPartitions(aggregator.combineValuesByKey).partitionBy(partitioner) } else { @@ -769,15 +768,15 @@ object IndexedRDD { } Iterator((values, bs)) }) - new IndexedRDD(index, values) + new VertexSetRDD(index, values) } // end of apply /** * Construct and index of the unique values in a given RDD. */ - def makeIndex[K: ClassManifest](keys: RDD[K], - partitioner: Option[Partitioner] = None): RDDIndex[K] = { + def makeIndex(keys: RDD[Vid], + partitioner: Option[Partitioner] = None): VertexSetIndex = { // @todo: I don't need the boolean its only there to be the second type since I want to shuffle a single RDD // Ugly hack :-(. In order to partition the keys they must have values. val tbl = keys.mapPartitions(_.map(k => (k, false)), true) @@ -786,7 +785,7 @@ object IndexedRDD { case None => { if (tbl.partitioner.isEmpty) { // @todo: I don't need the boolean its only there to be the second type of the shuffle. - new ShuffledRDD[K, Boolean, (K, Boolean)](tbl, Partitioner.defaultPartitioner(tbl)) + new ShuffledRDD[Vid, Boolean, (Vid, Boolean)](tbl, Partitioner.defaultPartitioner(tbl)) } else { tbl } } case Some(partitioner) => @@ -794,7 +793,7 @@ object IndexedRDD { } val index = shuffledTbl.mapPartitions( iter => { - val indexMap = new BlockIndex[K]() + val indexMap = new VertexIdToIndexMap() for ( (k,_) <- iter ){ if(!indexMap.contains(k)){ val ind = indexMap.size @@ -803,10 +802,10 @@ object IndexedRDD { } Iterator(indexMap) }, true).cache - new RDDIndex(index) + new VertexSetIndex(index) } -} // end of object IndexedRDD +} // end of object VertexSetRDD diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index c6875f0c9c470..bdf79bf9f0fba 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -66,17 +66,16 @@ class EdgeTripletIterator[VD: ClassManifest, ED: ClassManifest]( object EdgeTripletBuilder { def makeTriplets[VD: ClassManifest, ED: ClassManifest]( - localVidMap: IndexedRDD[Pid, VertexIdToIndexMap], - vTableReplicatedValues: IndexedRDD[Pid, Array[VD]], - eTable: IndexedRDD[Pid, EdgePartition[ED]]): RDD[EdgeTriplet[VD, ED]] = { - val iterFun = (iter: Iterator[(Pid, ((VertexIdToIndexMap, Array[VD]), EdgePartition[ED]))]) => { - val (pid, ((vidToIndex, vertexArray), edgePartition)) = iter.next() - assert(iter.hasNext == false) + localVidMap: RDD[(Pid, VertexIdToIndexMap)], + vTableReplicatedValues: RDD[(Pid, Array[VD]) ], + eTable: RDD[(Pid, EdgePartition[ED])]): RDD[EdgeTriplet[VD, ED]] = { + localVidMap.zipPartitions(vTableReplicatedValues, eTable) { + (vidMapIter, replicatedValuesIter, eTableIter) => + val (_, vidToIndex) = vidMapIter.next() + val (_, vertexArray) = replicatedValuesIter.next() + val (_, edgePartition) = eTableIter.next() new EdgeTripletIterator(vidToIndex, vertexArray, edgePartition) } - ClosureCleaner.clean(iterFun) - localVidMap.zipJoin(vTableReplicatedValues).zipJoin(eTable) - .mapPartitions( iterFun ) // end of map partition } } @@ -100,30 +99,30 @@ object EdgeTripletBuilder { * A Graph RDD that supports computation on graphs. */ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( - @transient val vTable: IndexedRDD[Vid, VD], - @transient val vid2pid: IndexedRDD[Vid, Array[Pid]], - @transient val localVidMap: IndexedRDD[Pid, VertexIdToIndexMap], - @transient val eTable: IndexedRDD[Pid, EdgePartition[ED]]) + @transient val vTable: VertexSetRDD[VD], + @transient val vid2pid: VertexSetRDD[Array[Pid]], + @transient val localVidMap: RDD[(Pid, VertexIdToIndexMap)], + @transient val eTable: RDD[(Pid, EdgePartition[ED])] ) extends Graph[VD, ED] { // def this() = this(null,null,null) /** - * (localVidMap: IndexedRDD[Pid, VertexIdToIndexMap]) is a version of the + * (localVidMap: VertexSetRDD[Pid, VertexIdToIndexMap]) is a version of the * vertex data after it is replicated. Within each partition, it holds a map * from vertex ID to the index where that vertex's attribute is stored. This * index refers to an array in the same partition in vTableReplicatedValues. * - * (vTableReplicatedValues: IndexedRDD[Pid, Array[VD]]) holds the vertex data + * (vTableReplicatedValues: VertexSetRDD[Pid, Array[VD]]) holds the vertex data * and is arranged as described above. */ - @transient val vTableReplicatedValues = + @transient val vTableReplicatedValues: RDD[(Pid, Array[VD])] = createVTableReplicated(vTable, vid2pid, localVidMap) /** Return a RDD of vertices. */ - @transient override val vertices: RDD[(Vid, VD)] = vTable + @transient override val vertices = vTable /** Return a RDD of edges. */ @@ -177,36 +176,40 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( override def reverse: Graph[VD, ED] = { - val etable = eTable.mapValues( _.reverse ).asInstanceOf[IndexedRDD[Pid, EdgePartition[ED]]] - new GraphImpl(vTable, vid2pid, localVidMap, etable) + val newEtable = eTable.mapPartitions( _.map{ case (pid, epart) => (pid, epart.reverse) }, + preservesPartitioning = true) + new GraphImpl(vTable, vid2pid, localVidMap, newEtable) } override def mapVertices[VD2: ClassManifest](f: (Vid, VD) => VD2): Graph[VD2, ED] = { val newVTable = vTable.mapValuesWithKeys((vid, data) => f(vid, data)) - .asInstanceOf[IndexedRDD[Vid, VD2]] new GraphImpl(newVTable, vid2pid, localVidMap, eTable) } override def mapEdges[ED2: ClassManifest](f: Edge[ED] => ED2): Graph[VD, ED2] = { - val newETable = eTable.mapValues(eBlock => eBlock.map(f)) - .asInstanceOf[IndexedRDD[Pid, EdgePartition[ED2]]] + val newETable = eTable.mapPartitions(_.map{ case (pid, epart) => (pid, epart.map(f)) }, + preservesPartitioning = true) new GraphImpl(vTable, vid2pid, localVidMap, newETable) } override def mapTriplets[ED2: ClassManifest](f: EdgeTriplet[VD, ED] => ED2): Graph[VD, ED2] = { - val newETable = eTable.zipJoin(localVidMap).zipJoin(vTableReplicatedValues).mapValues{ - case ((edgePartition, vidToIndex), vertexArray) => - val et = new EdgeTriplet[VD, ED] - edgePartition.map{e => - et.set(e) - et.srcAttr = vertexArray(vidToIndex(e.srcId)) - et.dstAttr = vertexArray(vidToIndex(e.dstId)) - f(et) - } - }.asInstanceOf[IndexedRDD[Pid, EdgePartition[ED2]]] + val newETable = eTable.zipPartitions(localVidMap, vTableReplicatedValues){ + (edgePartitionIter, vidToIndexIter, vertexArrayIter) => + val (pid, edgePartition) = edgePartitionIter.next() + val (_, vidToIndex) = vidToIndexIter.next() + val (_, vertexArray) = vertexArrayIter.next() + val et = new EdgeTriplet[VD, ED] + val newEdgePartition = edgePartition.map{e => + et.set(e) + et.srcAttr = vertexArray(vidToIndex(e.srcId)) + et.dstAttr = vertexArray(vidToIndex(e.dstId)) + f(et) + } + Iterator((pid, newEdgePartition)) + } new GraphImpl(vTable, vid2pid, localVidMap, newETable) } @@ -238,7 +241,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( // Reuse the partitioner (but not the index) from this graph val newVTable = - IndexedRDD(vertices.filter(v => vpred(v._1, v._2)).partitionBy(vTable.index.partitioner)) + VertexSetRDD(vertices.filter(v => vpred(v._1, v._2)).partitionBy(vTable.index.partitioner)) // Restrict the set of edges to those that satisfy the vertex and the edge predicate. @@ -309,53 +312,56 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( override def mapReduceTriplets[A: ClassManifest]( mapFunc: EdgeTriplet[VD, ED] => Array[(Vid, A)], reduceFunc: (A, A) => A) - : RDD[(Vid, A)] = { + : VertexSetRDD[A] = { ClosureCleaner.clean(mapFunc) ClosureCleaner.clean(reduceFunc) // Map and preaggregate - val preAgg = localVidMap.zipJoin(vTableReplicatedValues).zipJoin(eTable).flatMap{ - case (pid, ((vidToIndex, vertexArray), edgePartition)) => - // We can reuse the vidToIndex map for aggregation here as well. - /** @todo Since this has the downside of not allowing "messages" to arbitrary - * vertices we should consider just using a fresh map. - */ - val msgArray = new Array[A](vertexArray.size) - val msgBS = new BitSet(vertexArray.size) - // Iterate over the partition - val et = new EdgeTriplet[VD, ED] - edgePartition.foreach{e => - et.set(e) - et.srcAttr = vertexArray(vidToIndex(e.srcId)) - et.dstAttr = vertexArray(vidToIndex(e.dstId)) - mapFunc(et).foreach{ case (vid, msg) => - // verify that the vid is valid - assert(vid == et.srcId || vid == et.dstId) - val ind = vidToIndex(vid) - // Populate the aggregator map - if(msgBS(ind)) { - msgArray(ind) = reduceFunc(msgArray(ind), msg) - } else { - msgArray(ind) = msg - msgBS(ind) = true - } + val preAgg = eTable.zipPartitions(localVidMap, vTableReplicatedValues){ + (edgePartitionIter, vidToIndexIter, vertexArrayIter) => + val (pid, edgePartition) = edgePartitionIter.next() + val (_, vidToIndex) = vidToIndexIter.next() + val (_, vertexArray) = vertexArrayIter.next() + // We can reuse the vidToIndex map for aggregation here as well. + /** @todo Since this has the downside of not allowing "messages" to arbitrary + * vertices we should consider just using a fresh map. + */ + val msgArray = new Array[A](vertexArray.size) + val msgBS = new BitSet(vertexArray.size) + // Iterate over the partition + val et = new EdgeTriplet[VD, ED] + edgePartition.foreach{e => + et.set(e) + et.srcAttr = vertexArray(vidToIndex(e.srcId)) + et.dstAttr = vertexArray(vidToIndex(e.dstId)) + mapFunc(et).foreach{ case (vid, msg) => + // verify that the vid is valid + assert(vid == et.srcId || vid == et.dstId) + val ind = vidToIndex(vid) + // Populate the aggregator map + if(msgBS(ind)) { + msgArray(ind) = reduceFunc(msgArray(ind), msg) + } else { + msgArray(ind) = msg + msgBS(ind) = true } } - // Return the aggregate map - vidToIndex.long2IntEntrySet().fastIterator() - // Remove the entries that did not receive a message - .filter{ entry => msgBS(entry.getValue()) } - // Construct the actual pairs - .map{ entry => - val vid = entry.getLongKey() - val ind = entry.getValue() - val msg = msgArray(ind) - (vid, msg) - } + } + // Return the aggregate map + vidToIndex.long2IntEntrySet().fastIterator() + // Remove the entries that did not receive a message + .filter{ entry => msgBS(entry.getValue()) } + // Construct the actual pairs + .map{ entry => + val vid = entry.getLongKey() + val ind = entry.getValue() + val msg = msgArray(ind) + (vid, msg) + } }.partitionBy(vTable.index.rdd.partitioner.get) // do the final reduction reusing the index map - IndexedRDD(preAgg, vTable.index, reduceFunc) + VertexSetRDD(preAgg, vTable.index, reduceFunc) } @@ -402,7 +408,7 @@ object GraphImpl { defaultVertexAttr: VD, mergeFunc: (VD, VD) => VD): GraphImpl[VD,ED] = { - val vtable = IndexedRDD(vertices, mergeFunc) + val vtable = VertexSetRDD(vertices, mergeFunc) /** * @todo Verify that there are no edges that contain vertices * that are not in vTable. This should probably be resolved: @@ -432,54 +438,54 @@ object GraphImpl { * containing all the edges in a partition. */ protected def createETable[ED: ClassManifest](edges: RDD[Edge[ED]]) - : IndexedRDD[Pid, EdgePartition[ED]] = { - // Get the number of partitions - val numPartitions = edges.partitions.size - val ceilSqrt: Pid = math.ceil(math.sqrt(numPartitions)).toInt - IndexedRDD(edges.map { e => - // Random partitioning based on the source vertex id. - // val part: Pid = edgePartitionFunction1D(e.srcId, e.dstId, numPartitions) - // val part: Pid = edgePartitionFunction2D(e.srcId, e.dstId, numPartitions, ceilSqrt) - val part: Pid = randomVertexCut(e.srcId, e.dstId, numPartitions) - //val part: Pid = canonicalEdgePartitionFunction2D(e.srcId, e.dstId, numPartitions, ceilSqrt) - - // Should we be using 3-tuple or an optimized class - MessageToPartition(part, (e.srcId, e.dstId, e.attr)) + : RDD[(Pid, EdgePartition[ED])] = { + // Get the number of partitions + val numPartitions = edges.partitions.size + val ceilSqrt: Pid = math.ceil(math.sqrt(numPartitions)).toInt + edges.map { e => + // Random partitioning based on the source vertex id. + // val part: Pid = edgePartitionFunction1D(e.srcId, e.dstId, numPartitions) + // val part: Pid = edgePartitionFunction2D(e.srcId, e.dstId, numPartitions, ceilSqrt) + val part: Pid = randomVertexCut(e.srcId, e.dstId, numPartitions) + //val part: Pid = canonicalEdgePartitionFunction2D(e.srcId, e.dstId, numPartitions, ceilSqrt) + + // Should we be using 3-tuple or an optimized class + MessageToPartition(part, (e.srcId, e.dstId, e.attr)) + } + .partitionBy(new HashPartitioner(numPartitions)) + .mapPartitionsWithIndex( (pid, iter) => { + val builder = new EdgePartitionBuilder[ED] + iter.foreach { message => + val data = message.data + builder.add(data._1, data._2, data._3) } - .partitionBy(new HashPartitioner(numPartitions)) - .mapPartitionsWithIndex({ (pid, iter) => - val builder = new EdgePartitionBuilder[ED] - iter.foreach { message => - val data = message.data - builder.add(data._1, data._2, data._3) - } - val edgePartition = builder.toEdgePartition - Iterator((pid, edgePartition)) - }, preservesPartitioning = true)) + val edgePartition = builder.toEdgePartition + Iterator((pid, edgePartition)) + }, preservesPartitioning = true).cache() } protected def createVid2Pid[ED: ClassManifest]( - eTable: IndexedRDD[Pid, EdgePartition[ED]], - vTableIndex: RDDIndex[Vid]): IndexedRDD[Vid, Array[Pid]] = { + eTable: RDD[(Pid, EdgePartition[ED])], + vTableIndex: VertexSetIndex): VertexSetRDD[Array[Pid]] = { val preAgg = eTable.mapPartitions { iter => val (pid, edgePartition) = iter.next() val vSet = new VertexSet edgePartition.foreach(e => {vSet.add(e.srcId); vSet.add(e.dstId)}) vSet.iterator.map { vid => (vid.toLong, pid) } } - IndexedRDD[Vid, Pid, ArrayBuffer[Pid]](preAgg, vTableIndex, + VertexSetRDD[Pid, ArrayBuffer[Pid]](preAgg, vTableIndex, (p: Pid) => ArrayBuffer(p), (ab: ArrayBuffer[Pid], p:Pid) => {ab.append(p); ab}, (a: ArrayBuffer[Pid], b: ArrayBuffer[Pid]) => a ++ b) - .mapValues(a => a.toArray).asInstanceOf[IndexedRDD[Vid, Array[Pid]]] + .mapValues(a => a.toArray).cache() } - protected def createLocalVidMap[ED: ClassManifest]( - eTable: IndexedRDD[Pid, EdgePartition[ED]]): IndexedRDD[Pid, VertexIdToIndexMap] = { - eTable.mapValues{ epart => - val vidToIndex = new VertexIdToIndexMap() + protected def createLocalVidMap[ED: ClassManifest](eTable: RDD[(Pid, EdgePartition[ED])]): + RDD[(Pid, VertexIdToIndexMap)] = { + eTable.mapPartitions( _.map{ case (pid, epart) => + val vidToIndex = new VertexIdToIndexMap var i = 0 epart.foreach{ e => if(!vidToIndex.contains(e.srcId)) { @@ -491,16 +497,16 @@ object GraphImpl { i += 1 } } - vidToIndex - } + (pid, vidToIndex) + }, preservesPartitioning = true).cache() } protected def createVTableReplicated[VD: ClassManifest]( - vTable: IndexedRDD[Vid, VD], - vid2pid: IndexedRDD[Vid, Array[Pid]], - replicationMap: IndexedRDD[Pid, VertexIdToIndexMap]): - IndexedRDD[Pid, Array[VD]] = { + vTable: VertexSetRDD[VD], + vid2pid: VertexSetRDD[Array[Pid]], + replicationMap: RDD[(Pid, VertexIdToIndexMap)]): + RDD[(Pid, Array[VD])] = { // Join vid2pid and vTable, generate a shuffle dependency on the joined // result, and get the shuffle id so we can use it on the slave. val msgsByPartition = vTable.zipJoin(vid2pid) @@ -509,9 +515,9 @@ object GraphImpl { } .partitionBy(replicationMap.partitioner.get).cache() - val newValuesRDD = replicationMap.valuesRDD.zipPartitions(msgsByPartition){ + replicationMap.zipPartitions(msgsByPartition){ (mapIter, msgsIter) => - val (IndexedSeq(vidToIndex), bs) = mapIter.next() + val (pid, vidToIndex) = mapIter.next() assert(!mapIter.hasNext) // Populate the vertex array using the vidToIndex map val vertexArray = new Array[VD](vidToIndex.size) @@ -519,14 +525,12 @@ object GraphImpl { val ind = vidToIndex(msg.data._1) vertexArray(ind) = msg.data._2 } - Iterator((IndexedSeq(vertexArray), bs)) - } - - new IndexedRDD(replicationMap.index, newValuesRDD) + Iterator((pid, vertexArray)) + }.cache() // @todo assert edge table has partitioner - // val localVidMap: IndexedRDD[Pid, VertexIdToIndexMap] = + // val localVidMap: VertexSetRDD[Pid, VertexIdToIndexMap] = // msgsByPartition.mapPartitionsWithIndex( (pid, iter) => { // val vidToIndex = new VertexIdToIndexMap // var i = 0 @@ -537,7 +541,7 @@ object GraphImpl { // Array((pid, vidToIndex)).iterator // }, preservesPartitioning = true).indexed(eTable.index) - // val vTableReplicatedValues: IndexedRDD[Pid, Array[VD]] = + // val vTableReplicatedValues: VertexSetRDD[Pid, Array[VD]] = // msgsByPartition.mapPartitionsWithIndex( (pid, iter) => { // val vertexArray = ArrayBuilder.make[VD] // for (msg <- iter) { From d429f015c0a90d10a301f0901ac7e36c543d3220 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 18 Oct 2013 15:02:41 -0700 Subject: [PATCH 131/531] Update GraphSuite aggregateNeighbors test --- .../org/apache/spark/graph/GraphSuite.scala | 17 ++++++++--------- 1 file changed, 8 insertions(+), 9 deletions(-) diff --git a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala index 145be3c126a38..ff0306eaa3034 100644 --- a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala @@ -8,8 +8,6 @@ import org.apache.spark.graph.LocalSparkContext._ class GraphSuite extends FunSuite with LocalSparkContext { -// val sc = new SparkContext("local[4]", "test") - System.setProperty("spark.serializer", "org.apache.spark.serializer.KryoSerializer") System.setProperty("spark.kryo.registrator", "org.apache.spark.graph.GraphKryoRegistrator") @@ -24,25 +22,26 @@ class GraphSuite extends FunSuite with LocalSparkContext { test("aggregateNeighbors") { withSpark(new SparkContext("local", "test")) { sc => - val star = Graph(sc.parallelize(List((0, 1), (0, 2), (0, 3)))) + val n = 3 + val star = Graph(sc.parallelize((1 to n).map(x => (0: Vid, x: Vid)))) val indegrees = star.aggregateNeighbors( (vid, edge) => Some(1), (a: Int, b: Int) => a + b, - EdgeDirection.In)// .map((vid, attr) => (vid, attr._2.getOrElse(0))) - assert(indegrees.collect().toSet === Set((1, 1), (2, 1), (3, 1))) // (0, 0), + 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) //.map((vid, attr) => (vid, attr._2.getOrElse(0))) - assert(outdegrees.collect().toSet === Set((0, 3))) //, (1, 0), (2, 0), (3, 0))) + EdgeDirection.Out) + assert(outdegrees.collect().toSet === Set((0, n))) val noVertexValues = star.aggregateNeighbors[Int]( (vid: Vid, edge: EdgeTriplet[Int, Int]) => None, (a: Int, b: Int) => throw new Exception("reduceFunc called unexpectedly"), - EdgeDirection.In)//.map((vid, attr) => (vid, attr)) - assert(noVertexValues.collect().toSet === Set.empty[(Vid, Int)] ) // ((0, None), (1, None), (2, None), (3, None))) + EdgeDirection.In) + assert(noVertexValues.collect().toSet === Set.empty[(Vid, Int)]) } } From d15db10831691348f22f415aeba00cd5f771b663 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 18 Oct 2013 19:43:33 -0700 Subject: [PATCH 132/531] Add a unit test for Graph.mapEdges --- .../scala/org/apache/spark/graph/GraphSuite.scala | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala index ff0306eaa3034..b501bac2a25ef 100644 --- a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala @@ -20,6 +20,19 @@ class GraphSuite extends FunSuite with LocalSparkContext { } } + test("mapEdges") { + withSpark(new SparkContext("local", "test")) { sc => + val n = 3 + val star = Graph(sc.parallelize((1 to n).map(x => (0: Vid, x: Vid)))) + val starWithEdgeAttrs = star.mapEdges(e => e.dstId) + + // map(_.copy()) is a workaround for https://github.com/amplab/graphx/issues/25 + val edges = starWithEdgeAttrs.edges.map(_.copy()).collect() + assert(edges.size === n) + assert(edges.toSet === (1 to n).map(x => Edge(0, x, x)).toSet) + } + } + test("aggregateNeighbors") { withSpark(new SparkContext("local", "test")) { sc => val n = 3 From 2d3603930e112b9c92e6863f6ec08eb55695779e Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 18 Oct 2013 19:44:58 -0700 Subject: [PATCH 133/531] Add a unit test for GraphOps.joinVertices --- .../org/apache/spark/graph/GraphSuite.scala | 29 ++++++++----------- 1 file changed, 12 insertions(+), 17 deletions(-) diff --git a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala index b501bac2a25ef..2d74ce92e25ef 100644 --- a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala @@ -58,24 +58,19 @@ class GraphSuite extends FunSuite with LocalSparkContext { } } - /* test("joinVertices") { - sc = new SparkContext("local", "test") - val vertices = sc.parallelize(Seq(Vertex(1, "one"), Vertex(2, "two"), Vertex(3, "three")), 2) - val edges = sc.parallelize((Seq(Edge(1, 2, "onetwo")))) - val g: Graph[String, String] = new GraphImpl(vertices, edges) - - val tbl = sc.parallelize(Seq((1, 10), (2, 20))) - val g1 = g.joinVertices(tbl, (v: Vertex[String], u: Int) => v.data + u) - - val v = g1.vertices.collect().sortBy(_.id) - assert(v(0).data === "one10") - assert(v(1).data === "two20") - assert(v(2).data === "three") - - val e = g1.edges.collect() - assert(e(0).data === "onetwo") + test("joinVertices") { + withSpark(new SparkContext("local", "test")) { sc => + val vertices = sc.parallelize(Seq[(Vid, String)]((1, "one"), (2, "two"), (3, "three")), 2) + val edges = sc.parallelize((Seq(Edge(1, 2, "onetwo")))) + val g: Graph[String, String] = Graph(vertices, edges) + + val tbl = sc.parallelize(Seq[(Vid, Int)]((1, 10), (2, 20))) + val g1 = g.joinVertices(tbl) { (vid: Vid, attr: String, u: Int) => attr + u } + + val v = g1.vertices.collect().toSet + assert(v === Set((1, "one10"), (2, "two20"), (3, "three"))) + } } - */ // test("graph partitioner") { // sc = new SparkContext("local", "test") From dbc8c9868aac33faa45ab54935df3613e31d4ff9 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Fri, 18 Oct 2013 23:44:06 -0700 Subject: [PATCH 134/531] Fixing bug in VertexSetRDD that breaks Graph tests. --- graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala b/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala index b3f1fa768c86d..e92c4e1fb107d 100644 --- a/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala @@ -219,7 +219,7 @@ class VertexSetRDD[V: ClassManifest]( if (other.partitioner == Some(partitioner)) other else other.partitionBy(partitioner) val newValues: RDD[ (IndexedSeq[(V,Option[W])], BitSet) ] = - index.rdd.zipPartitions(valuesRDD, other) { + index.rdd.zipPartitions(valuesRDD, otherShuffled) { (thisIndexIter, thisIter, tuplesIter) => val index = thisIndexIter.next() assert(!thisIndexIter.hasNext) From ebdbedc3e9d541116b4109ae692b12bf60e4ff06 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Sat, 19 Oct 2013 01:26:08 -0700 Subject: [PATCH 135/531] Documenting VertexSetRDD and added some testing code for VertexSetRDD --- .../org/apache/spark/graph/VertexSetRDD.scala | 588 ++++++++---------- .../org/apache/spark/graph/GraphSuite.scala | 54 +- 2 files changed, 278 insertions(+), 364 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala b/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala index e92c4e1fb107d..cb75da6c211cb 100644 --- a/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala @@ -41,28 +41,62 @@ import org.apache.spark.storage.StorageLevel /** - * The VertexSetIndex is an opaque type used to represent the organization - * of values in an RDD + * The `VertexSetIndex` maintains the per-partition mapping from vertex id + * to the corresponding location in the per-partition values array. + * This class is meant to be an opaque type. + * */ class VertexSetIndex(private[spark] val rdd: RDD[VertexIdToIndexMap]) { + /** + * The persist function behaves like the standard RDD persist + */ def persist(newLevel: StorageLevel): VertexSetIndex = { rdd.persist(newLevel) return this } + + /** + * Returns the partitioner object of the underlying RDD. This is used + * by the VertexSetRDD to partition the values RDD. + */ def partitioner: Partitioner = rdd.partitioner.get -} +} // end of VertexSetIndex /** - * An VertexSetRDD[V] extends the RDD[(Vid,V)] by pre-indexing the keys and - * organizing the values to enable faster join operations. + * An VertexSetRDD[V] extends the RDD[(Vid,V)] by ensuring that there is only + * one entry for each vertex and by pre-indexing the entries for fast, efficient + * joins. * * In addition to providing the basic RDD[(Vid,V)] functionality the VertexSetRDD * exposes an index member which can be used to "key" other VertexSetRDDs * + * @tparam V the vertex attribute associated with each vertex in the set. + * + * @param index the index which contains the vertex id information and is used + * to organize the values in the RDD. + * @param valuesRDD the values RDD contains the actual vertex attributes organized + * as an array within each partition. + * + * To construct a `VertexSetRDD` use the singleton object: + * + * @example Construct a `VertexSetRDD` from a plain RDD + * {{{ + * // Construct an intial vertex set + * val someData: RDD[(Vid, SomeType)] = loadData(someFile) + * val vset = VertexSetRDD(someData) + * // If there were redundant values in someData we would use a reduceFunc + * val vset2 = VertexSetRDD(someData, reduceFunc) + * // Finally we can use the VertexSetRDD to index another dataset + * val otherData: RDD[(Vid, OtherType)] = loadData(otherFile) + * val vset3 = VertexSetRDD(otherData, vset.index) + * // Now we can construct very fast joins between the two sets + * val vset4: VertexSetRDD[(SomeType, OtherType)] = vset.leftJoin(vset3) + * }}} + * */ -class VertexSetRDD[V: ClassManifest]( +class VertexSetRDD[@specialized V: ClassManifest]( @transient val index: VertexSetIndex, @transient val valuesRDD: RDD[ (IndexedSeq[V], BitSet) ]) extends RDD[(Vid, V)](index.rdd.context, @@ -70,20 +104,23 @@ class VertexSetRDD[V: ClassManifest]( /** - * Construct a new VertexSetRDD that is indexed by only the keys in the RDD + * Construct a new VertexSetRDD that is indexed by only the keys in the RDD. + * The resulting VertexSet will be based on a different index and can + * no longer be quickly joined with this RDD. */ def reindex(): VertexSetRDD[V] = VertexSetRDD(this) /** * An internal representation which joins the block indices with the values + * This is used by the compute function to emulate RDD[(Vid, V)] */ protected[spark] val tuples = new ZippedRDD(index.rdd.context, index.rdd, valuesRDD) /** - * The partitioner is defined by the index + * The partitioner is defined by the index. */ override val partitioner = index.rdd.partitioner @@ -95,7 +132,8 @@ class VertexSetRDD[V: ClassManifest]( /** - * The preferred locations are computed based on the preferred locations of the tuples. + * The preferred locations are computed based on the preferred + * locations of the tuples. */ override def getPreferredLocations(s: Partition): Seq[String] = tuples.getPreferredLocations(s) @@ -110,75 +148,170 @@ class VertexSetRDD[V: ClassManifest]( return this } + + /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */ override def persist(): VertexSetRDD[V] = persist(StorageLevel.MEMORY_ONLY) + /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */ override def cache(): VertexSetRDD[V] = persist() - /** - * Pass each value in the key-value pair RDD through a map function without changing the keys; - * this also retains the original RDD's partitioning. + * Provide the RDD[(K,V)] equivalent output. */ - def mapValues[U: ClassManifest](f: V => U): VertexSetRDD[U] = { - val cleanF = index.rdd.context.clean(f) - val newValuesRDD: RDD[ (IndexedSeq[U], BitSet) ] = - valuesRDD.mapPartitions(iter => iter.map{ - case (values, bs) => - val newValues = new Array[U](values.size) - for ( ind <- bs ) { - newValues(ind) = f(values(ind)) - } - (newValues.toIndexedSeq, bs) - }, preservesPartitioning = true) - new VertexSetRDD[U](index, newValuesRDD) - } + override def compute(part: Partition, context: TaskContext): Iterator[(Vid, V)] = { + tuples.compute(part, context).flatMap { case (indexMap, (values, bs) ) => + // Walk the index to construct the key, value pairs + indexMap.iterator + // Extract rows with key value pairs and indicators + .map{ case (k, ind) => (bs(ind), k, ind) } + // Remove tuples that aren't actually present in the array + .filter( _._1 ) + // Extract the pair (removing the indicator from the tuple) + .map( x => (x._2, values(x._3) ) ) + } + } // end of compute /** - * Pass each value in the key-value pair RDD through a map function without changing the keys; - * this also retains the original RDD's partitioning. + * @todo finish documenting */ - def mapValuesWithKeys[U: ClassManifest](f: (Vid, V) => U): VertexSetRDD[U] = { + override def filter(f: Tuple2[Vid,V] => Boolean): VertexSetRDD[V] = { val cleanF = index.rdd.context.clean(f) - val newValues: RDD[ (IndexedSeq[U], BitSet) ] = - index.rdd.zipPartitions(valuesRDD){ + val newValues = index.rdd.zipPartitions(valuesRDD){ (keysIter, valuesIter) => val index = keysIter.next() assert(keysIter.hasNext() == false) val (oldValues, bs) = valuesIter.next() assert(valuesIter.hasNext() == false) - // Allocate the array to store the results into - val newValues: Array[U] = new Array[U](oldValues.size) + // Allocate the array to store the results into + val newBS = new BitSet(oldValues.size) // Populate the new Values for( (k,i) <- index ) { - if (bs(i)) { newValues(i) = f(k, oldValues(i)) } + newBS(i) = bs(i) && cleanF( (k, oldValues(i)) ) } - Array((newValues.toIndexedSeq, bs)).iterator + Array((oldValues, newBS)).iterator } + new VertexSetRDD[V](index, newValues) + } // end of filter + + + /** + * Pass each vertex attribute through a map function and retain + * the original RDD's partitioning and index. + * + * @tparam U the type returned by the map function + * + * @param f the function applied to each value in the RDD + * @return a new VertexSet with values obtaind by applying `f` to each of the + * entries in the original VertexSet. The resulting VertexSetRDD retains the + * same index. + */ + def mapValues[U: ClassManifest](f: V => U): VertexSetRDD[U] = { + val cleanF = index.rdd.context.clean(f) + val newValuesRDD: RDD[ (IndexedSeq[U], BitSet) ] = + valuesRDD.mapPartitions(iter => iter.map{ + case (values, bs) => + /** + * @todo Consider using a view rather than creating a new array. + * This is already being done for join operations. It could reduce + * memory overhead but require additional recomputation. + */ + val newValues = new Array[U](values.size) + for ( ind <- bs ) { + newValues(ind) = f(values(ind)) + } + (newValues.toIndexedSeq, bs) + }, preservesPartitioning = true) + new VertexSetRDD[U](index, newValuesRDD) + } // end of mapValues + + + /** + * Pass each vertex attribute along with the vertex id through a + * map function and retain the original RDD's partitioning and index. + * + * @tparam U the type returned by the map function + * + * @param f the function applied to each vertex id and vertex + * attribute in the RDD + * @return a new VertexSet with values obtaind by applying `f` to each of the + * entries in the original VertexSet. The resulting VertexSetRDD retains the + * same index. + */ + def mapValuesWithKeys[U: ClassManifest](f: (Vid, V) => U): VertexSetRDD[U] = { + val cleanF = index.rdd.context.clean(f) + val newValues: RDD[ (IndexedSeq[U], BitSet) ] = + index.rdd.zipPartitions(valuesRDD){ + (keysIter, valuesIter) => + val index = keysIter.next() + assert(keysIter.hasNext() == false) + val (oldValues, bs) = valuesIter.next() + assert(valuesIter.hasNext() == false) + /** + * @todo Consider using a view rather than creating a new array. + * This is already being done for join operations. It could reduce + * memory overhead but require additional recomputation. + */ + // Allocate the array to store the results into + val newValues: Array[U] = new Array[U](oldValues.size) + // Populate the new Values + for( (k,i) <- index ) { + if (bs(i)) { newValues(i) = f(k, oldValues(i)) } + } + Array((newValues.toIndexedSeq, bs)).iterator + } new VertexSetRDD[U](index, newValues) - } + } // end of mapValuesWithKeys + /** + * Inner join this VertexSet with another VertexSet which has the same Index. + * This function will fail if both VertexSets do not share the same index. + * The resulting vertex set will only contain vertices that are in both this + * and the other vertex set. + * + * @tparam W the attribute type of the other VertexSet + * + * @param other the other VertexSet with which to join. + * @return a VertexSetRDD containing only the vertices in both this and the + * other VertexSet and with tuple attributes. + * + */ def zipJoin[W: ClassManifest](other: VertexSetRDD[W]): VertexSetRDD[(V,W)] = { if(index != other.index) { throw new SparkException("A zipJoin can only be applied to RDDs with the same index!") } - val newValuesRDD: RDD[ (IndexedSeq[(V,W)], BitSet) ] = valuesRDD.zipPartitions(other.valuesRDD){ - (thisIter, otherIter) => - val (thisValues, thisBS) = thisIter.next() - assert(!thisIter.hasNext) - val (otherValues, otherBS) = otherIter.next() - assert(!otherIter.hasNext) - val newBS = thisBS & otherBS - val newValues = thisValues.view.zip(otherValues) - Iterator((newValues.toIndexedSeq, newBS)) - } + val newValuesRDD: RDD[ (IndexedSeq[(V,W)], BitSet) ] = + valuesRDD.zipPartitions(other.valuesRDD){ + (thisIter, otherIter) => + val (thisValues, thisBS) = thisIter.next() + assert(!thisIter.hasNext) + val (otherValues, otherBS) = otherIter.next() + assert(!otherIter.hasNext) + val newBS = thisBS & otherBS + val newValues = thisValues.view.zip(otherValues) + Iterator((newValues.toIndexedSeq, newBS)) + } new VertexSetRDD(index, newValuesRDD) } + /** + * Left join this VertexSet with another VertexSet which has the same Index. + * This function will fail if both VertexSets do not share the same index. + * The resulting vertex set contains an entry for each vertex in this set. + * If the other VertexSet is missing any vertex in this VertexSet then a + * `None` attribute is generated + * + * @tparam W the attribute type of the other VertexSet + * + * @param other the other VertexSet with which to join. + * @return a VertexSetRDD containing all the vertices in this VertexSet + * with `None` attributes used for Vertices missing in the other VertexSet. + * + */ def leftZipJoin[W: ClassManifest](other: VertexSetRDD[W]): VertexSetRDD[(V,Option[W])] = { if(index != other.index) { throw new SparkException("A zipJoin can only be applied to RDDs with the same index!") @@ -195,41 +328,63 @@ class VertexSetRDD[V: ClassManifest]( Iterator((newValues.toIndexedSeq, thisBS)) } new VertexSetRDD(index, newValuesRDD) - } - + } // end of leftZipJoin + /** + * Left join this VertexSet with an RDD containing vertex attribute pairs. + * If the other RDD is backed by a VertexSet with the same index than the + * efficient leftZipJoin implementation is used. + * The resulting vertex set contains an entry for each vertex in this set. + * If the other VertexSet is missing any vertex in this VertexSet then a + * `None` attribute is generated + * + * @tparam W the attribute type of the other VertexSet + * + * @param other the other VertexSet with which to join. + * @param merge the function used combine duplicate vertex attributes + * @return a VertexSetRDD containing all the vertices in this VertexSet + * with `None` attributes used for Vertices missing in the other VertexSet. + * + */ def leftJoin[W: ClassManifest]( other: RDD[(Vid,W)], merge: (W,W) => W = (a:W, b:W) => a): VertexSetRDD[(V, Option[W]) ] = { val cleanMerge = index.rdd.context.clean(merge) - + // Test if the other vertex is a VertexSetRDD to choose the optimal + // join strategy other match { + // If the other set is a VertexSetRDD and shares the same index as + // this vertex set then we use the much more efficient leftZipJoin case other: VertexSetRDD[_] if index == other.index => { leftZipJoin(other) } case _ => { - // Get the partitioner from the index - val partitioner = index.rdd.partitioner match { - case Some(p) => p - case None => throw new SparkException("An index must have a partitioner.") - } - // Shuffle the other RDD using the partitioner for this index + // Otherwise we treat the other RDD as a collectiong of + // vertex-attribute pairs. + // If necessary shuffle the other RDD using the partitioner + // for this VertexSet val otherShuffled = - if (other.partitioner == Some(partitioner)) other - else other.partitionBy(partitioner) + if (other.partitioner == partitioner) other + else other.partitionBy(partitioner.get) + // Compute the new values RDD val newValues: RDD[ (IndexedSeq[(V,Option[W])], BitSet) ] = index.rdd.zipPartitions(valuesRDD, otherShuffled) { (thisIndexIter, thisIter, tuplesIter) => + // Get the Index and values for this RDD val index = thisIndexIter.next() assert(!thisIndexIter.hasNext) val (thisValues, thisBS) = thisIter.next() assert(!thisIter.hasNext) + // Create a new array to store the values in the resulting VertexSet val newW = new Array[W](thisValues.size) // track which values are matched with values in other val wBS = new BitSet(thisValues.size) + // Loop over all the tuples that have vertices in this VertexSet. for( (k, w) <- tuplesIter if index.contains(k) ) { val ind = index.get(k) + // Not all the vertex ids in the index are in this VertexSet. + // If there is a vertex in this set then record the other value if(thisBS(ind)) { if(wBS(ind)) { newW(ind) = cleanMerge(newW(ind), w) @@ -238,31 +393,33 @@ class VertexSetRDD[V: ClassManifest]( wBS(ind) = true } } - } - + } // end of for loop over tuples + // Some vertices in this vertex set may not have a corresponding + // tuple in the join and so a None value should be returned. val otherOption = newW.view.zipWithIndex .map{ (x: (W, Int)) => if(wBS(x._2)) Option(x._1) else None } + // the final values is the zip of the values in this RDD along with + // the values in the other val newValues = thisValues.view.zip(otherOption) - Iterator((newValues.toIndexedSeq, thisBS)) } // end of newValues new VertexSetRDD(index, newValues) } } - } + } // end of leftJoin /** - * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the - * list of values for that key in `this` as well as `other`. + * For each key k in `this` or `other`, return a resulting RDD that contains a + * tuple with the list of values for that key in `this` as well as `other`. */ def cogroup[W: ClassManifest](other: RDD[(Vid, W)], partitioner: Partitioner): VertexSetRDD[(Seq[V], Seq[W])] = { //RDD[(K, (Seq[V], Seq[W]))] = { other match { case other: VertexSetRDD[_] if index == other.index => { - // if both RDDs share exactly the same index and therefore the same super set of keys - // then we simply merge the value RDDs. + // if both RDDs share exactly the same index and therefore the same + // super set of keys then we simply merge the value RDDs. // However it is possible that both RDDs are missing a value for a given key in // which case the returned RDD should have a null value val newValues: RDD[(IndexedSeq[(Seq[V], Seq[W])], BitSet)] = @@ -272,10 +429,12 @@ class VertexSetRDD[V: ClassManifest]( assert(!thisIter.hasNext) val (otherValues, otherBS) = otherIter.next() assert(!otherIter.hasNext) - + /** + * @todo consider implementing this with a view as in leftJoin to + * reduce array allocations + */ val newValues = new Array[(Seq[V], Seq[W])](thisValues.size) val newBS = thisBS | otherBS - for( ind <- newBS ) { val a = if (thisBS(ind)) Seq(thisValues(ind)) else Seq.empty[V] val b = if (otherBS(ind)) Seq(otherValues(ind)) else Seq.empty[W] @@ -400,12 +559,6 @@ class VertexSetRDD[V: ClassManifest]( newValues.append( (Seq.empty[V], ArrayBuffer(w) ) ) } } - // // Finalize the new values array - // val newValuesArray: Seq[Seq[(Seq[V],Seq[W])]] = - // newValues.view.map{ - // case null => null - // case (s, ab) => Seq((s, ab.toSeq)) - // }.toSeq Iterator( (newIndex, (newValues.toIndexedSeq, newBS)) ) }).cache() @@ -417,228 +570,37 @@ class VertexSetRDD[V: ClassManifest]( new VertexSetRDD[(Seq[V], Seq[W])](new VertexSetIndex(newIndex), newValues) } } - } - - - // - // def zipJoinToRDD[W: ClassManifest](other: VertexSetRDD[K,W]): RDD[(K,(V,W))] = { - // if(index != other.index) { - // throw new SparkException("ZipJoinRDD can only be applied to RDDs with the same index!") - // } - // index.rdd.zipPartitions(valuesRDD, other.valuesRDD){ - // (thisIndexIter, thisIter, otherIter) => - // val index = thisIndexIter.next() - // assert(!thisIndexIter.hasNext) - // val (thisValues, thisBS) = thisIter.next() - // assert(!thisIter.hasNext) - // val (otherValues, otherBS) = otherIter.next() - // assert(!otherIter.hasNext) - // val newBS = thisBS & otherBS - // index.iterator.filter{ case (k,i) => newBS(i) }.map{ - // case (k,i) => (k, (thisValues(i), otherValues(i))) - // } - // } - // } - - -/* This is probably useful but we are not using it - def zipJoinWithKeys[W: ClassManifest, Z: ClassManifest]( - other: RDD[(K,W)])( - f: (K, V, W) => Z, - merge: (Z,Z) => Z = (a:Z, b:Z) => a): - VertexSetRDD[K,Z] = { - val cleanF = index.rdd.context.clean(f) - val cleanMerge = index.rdd.context.clean(merge) - other match { - case other: VertexSetRDD[_, _] if index == other.index => { - val newValues = index.rdd.zipPartitions(valuesRDD, other.valuesRDD){ - (thisIndexIter, thisIter, otherIter) => - val index = thisIndexIter.next() - assert(!thisIndexIter.hasNext) - val (thisValues, thisBS) = thisIter.next() - assert(!thisIter.hasNext) - val (otherValues, otherBS) = otherIter.next() - assert(!otherIter.hasNext) - val newValues = new Array[Z](thisValues.size) - val newBS = thisBS & otherBS - for( (k,i) <- index ) { - if (newBS(i)) { - newValues(i) = cleanF(k, thisValues(i), otherValues(i)) - } - } - List((newValues, newBS)).iterator - } - new VertexSetRDD(index, newValues) - } - - case _ => { - // Get the partitioner from the index - val partitioner = index.rdd.partitioner match { - case Some(p) => p - case None => throw new SparkException("An index must have a partitioner.") - } - // Shuffle the other RDD using the partitioner for this index - val otherShuffled = - if (other.partitioner == Some(partitioner)) other - else other.partitionBy(partitioner) - - val newValues = index.rdd.zipPartitions(valuesRDD, other) { - (thisIndexIter, thisIter, tuplesIter) => - val index = thisIndexIter.next() - assert(!thisIndexIter.hasNext) - val (thisValues, thisBS) = thisIter.next() - assert(!thisIter.hasNext) - - val newValues = new Array[Z](thisValues.size) - // track which values are matched with values in other - val tempBS = new BitSet(thisValues.size) - - for( (k, w) <- tuplesIter if index.contains(k) ) { - val ind = index.get(k) - if(thisBS(ind)) { - val result = cleanF(k, thisValues(ind), w) - if(tempBS(ind)) { - newValues(ind) = cleanMerge(newValues(ind), result) - } else { - newValues(ind) = result - tempBS(ind) = true - } - } - } - List((newValues, tempBS)).iterator - } // end of newValues - new VertexSetRDD(index, newValues) - } - } - } -*/ - -/* - def zipJoinLeftWithKeys[W: ClassManifest, Z: ClassManifest]( - other: RDD[(K,W)])( - f: (K, V, Option[W]) => Z, - merge: (Z,Z) => Z = (a:Z, b:Z) => a): - VertexSetRDD[K,Z] = { - val cleanF = index.rdd.context.clean(f) - val cleanMerge = index.rdd.context.clean(merge) - other match { - case other: VertexSetRDD[_, _] if index == other.index => { - val newValues = index.rdd.zipPartitions(valuesRDD, other.valuesRDD){ - (thisIndexIter, thisIter, otherIter) => - val index = thisIndexIter.next() - assert(!thisIndexIter.hasNext) - val (thisValues, thisBS) = thisIter.next() - assert(!thisIter.hasNext) - val (otherValues, otherBS) = otherIter.next() - assert(!otherIter.hasNext) - val newValues = new Array[Z](thisValues.size) - for( (k,i) <- index ) { - if (thisBS(i)) { - val otherVal = if(otherBS(i)) Some(otherValues(i)) else None - newValues(i) = cleanF(k, thisValues(i), otherVal) - } - } - List((newValues, thisBS)).iterator - } - new VertexSetRDD(index, newValues) - } - - case _ => { - // Get the partitioner from the index - val partitioner = index.rdd.partitioner match { - case Some(p) => p - case None => throw new SparkException("An index must have a partitioner.") - } - // Shuffle the other RDD using the partitioner for this index - val otherShuffled = - if (other.partitioner == Some(partitioner)) other - else other.partitionBy(partitioner) - val newValues = index.rdd.zipPartitions(valuesRDD, other) { - (thisIndexIter, thisIter, tuplesIter) => - val index = thisIndexIter.next() - assert(!thisIndexIter.hasNext) - val (thisValues, thisBS) = thisIter.next() - assert(!thisIter.hasNext) - - val newValues = new Array[Z](thisValues.size) - // track which values are matched with values in other - val tempBS = new BitSet(thisValues.size) - - for( (k, w) <- tuplesIter if index.contains(k) ) { - val ind = index.get(k) - if(thisBS(ind)) { - val result = cleanF(k, thisValues(ind), Option(w)) - if(tempBS(ind)) { - newValues(ind) = cleanMerge(newValues(ind), result) - } else { - newValues(ind) = result - tempBS(ind) = true - } - } - } - - // Process remaining keys in lef join - for( (k,ind) <- index if thisBS(ind) && !tempBS(ind)) { - newValues(ind) = cleanF(k, thisValues(ind), None) - } - List((newValues, thisBS)).iterator - } // end of newValues - new VertexSetRDD(index, newValues) - } - } - } - -*/ - - - - override def filter(f: Tuple2[Vid,V] => Boolean): VertexSetRDD[V] = { - val cleanF = index.rdd.context.clean(f) - val newValues = index.rdd.zipPartitions(valuesRDD){ - (keysIter, valuesIter) => - val index = keysIter.next() - assert(keysIter.hasNext() == false) - val (oldValues, bs) = valuesIter.next() - assert(valuesIter.hasNext() == false) - // Allocate the array to store the results into - val newBS = new BitSet(oldValues.size) - // Populate the new Values - for( (k,i) <- index ) { - newBS(i) = bs(i) && cleanF( (k, oldValues(i)) ) - } - Array((oldValues, newBS)).iterator - } - new VertexSetRDD[V](index, newValues) - } - - - /** - * Provide the RDD[(K,V)] equivalent output. - */ - override def compute(part: Partition, context: TaskContext): Iterator[(Vid, V)] = { - tuples.compute(part, context).flatMap { case (indexMap, (values, bs) ) => - // Walk the index to construct the key, value pairs - indexMap.iterator - // Extract rows with key value pairs and indicators - .map{ case (k, ind) => (bs(ind), k, ind) } - // Remove tuples that aren't actually present in the array - .filter( _._1 ) - // Extract the pair (removing the indicator from the tuple) - .map( x => (x._2, values(x._3) ) ) - } - } - + } // end of cogroup } // End of VertexSetRDD - +/** + * The VertexSetRDD singleton is used to construct VertexSets + */ object VertexSetRDD { - + /** + * Construct a vertex set from an RDD of vertex-attribute pairs. + * Duplicate entries are removed arbitrarily. + * + * @tparam V the vertex attribute type + * + * @param rdd the collection of vertex-attribute pairs + */ def apply[V: ClassManifest](rdd: RDD[(Vid,V)]): VertexSetRDD[V] = apply(rdd, (a:V, b:V) => a ) + /** + * Construct a vertex set from an RDD of vertex-attribute pairs where + * duplicate entries are merged using the reduceFunc + * + * @tparam V the vertex attribute type + * + * @param rdd the collection of vertex-attribute pairs + * @param reduceFunc the function used to merge attributes of duplicate + * vertices. + */ def apply[V: ClassManifest]( rdd: RDD[(Vid,V)], reduceFunc: (V, V) => V): VertexSetRDD[V] = { // Preaggregate and shuffle if necessary @@ -669,61 +631,29 @@ object VertexSetRDD { val values: RDD[(IndexedSeq[V], BitSet)] = groups.mapPartitions(_.map{ case (kMap,vAr) => vAr }, true) new VertexSetRDD[V](new VertexSetIndex(index), values) - } - + } // end of apply + /** + * @todo finish documenting + */ def apply[V: ClassManifest]( rdd: RDD[(Vid,V)], index: VertexSetIndex): VertexSetRDD[V] = apply(rdd, index, (a:V,b:V) => a) + /** + * @todo finish documenting + */ def apply[V: ClassManifest]( rdd: RDD[(Vid,V)], index: VertexSetIndex, reduceFunc: (V, V) => V): VertexSetRDD[V] = apply(rdd,index, (v:V) => v, reduceFunc, reduceFunc) - // { - // // Get the index Partitioner - // val partitioner = index.rdd.partitioner match { - // case Some(p) => p - // case None => throw new SparkException("An index must have a partitioner.") - // } - // // Preaggregate and shuffle if necessary - // val partitioned = - // if (rdd.partitioner != Some(partitioner)) { - // // Preaggregation. - // val aggregator = new Aggregator[K, V, V](v => v, reduceFunc, reduceFunc) - // rdd.mapPartitions(aggregator.combineValuesByKey).partitionBy(partitioner) - // } else { - // rdd - // } - - // // Use the index to build the new values table - // val values = index.rdd.zipPartitions(partitioned)( (indexIter, tblIter) => { - // // There is only one map - // val index = indexIter.next() - // assert(!indexIter.hasNext()) - // val values = new Array[V](index.size) - // val bs = new BitSet(index.size) - // for ((k,v) <- tblIter) { - // if (!index.contains(k)) { - // throw new SparkException("Error: Trying to bind an external index " + - // "to an RDD which contains keys that are not in the index.") - // } - // val ind = index(k) - // if (bs(ind)) { - // values(ind) = reduceFunc(values(ind), v) - // } else { - // values(ind) = v - // bs(ind) = true - // } - // } - // List((values, bs)).iterator - // }) - // new VertexSetRDD[K,V](index, values) - // } // end of apply - + + /** + * @todo finish documenting + */ def apply[V: ClassManifest, C: ClassManifest]( rdd: RDD[(Vid,V)], index: VertexSetIndex, @@ -774,6 +704,8 @@ object VertexSetRDD { /** * Construct and index of the unique values in a given RDD. + * + * @todo finish documenting */ def makeIndex(keys: RDD[Vid], partitioner: Option[Partitioner] = None): VertexSetIndex = { diff --git a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala index 2d74ce92e25ef..f2b3d5bdfe268 100644 --- a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala @@ -72,40 +72,22 @@ class GraphSuite extends FunSuite with LocalSparkContext { } } -// test("graph partitioner") { -// sc = new SparkContext("local", "test") -// val vertices = sc.parallelize(Seq(Vertex(1, "one"), Vertex(2, "two"))) -// val edges = sc.parallelize(Seq(Edge(1, 2, "onlyedge"))) -// var g = Graph(vertices, edges) -// -// g = g.withPartitioner(4, 7) -// assert(g.numVertexPartitions === 4) -// assert(g.numEdgePartitions === 7) -// -// g = g.withVertexPartitioner(5) -// assert(g.numVertexPartitions === 5) -// -// g = g.withEdgePartitioner(8) -// assert(g.numEdgePartitions === 8) -// -// g = g.mapVertices(x => x) -// assert(g.numVertexPartitions === 5) -// assert(g.numEdgePartitions === 8) -// -// g = g.mapEdges(x => x) -// assert(g.numVertexPartitions === 5) -// assert(g.numEdgePartitions === 8) -// -// val updates = sc.parallelize(Seq((1, " more"))) -// g = g.updateVertices( -// updates, -// (v, u: Option[String]) => if (u.isDefined) v.data + u.get else v.data) -// assert(g.numVertexPartitions === 5) -// assert(g.numEdgePartitions === 8) -// -// g = g.reverse -// assert(g.numVertexPartitions === 5) -// assert(g.numEdgePartitions === 8) -// -// } + + test("VertexSetRDD") { + withSpark(new SparkContext("local", "test")) { sc => + val a = sc.parallelize((0 to 100).map(x => (x.toLong, x.toLong)), 5) + val b = VertexSetRDD(a).mapValues(x => -x) + assert(b.leftJoin(a) + .mapValues(x => x._1 + x._2.get).map(x=> x._2).reduce(_+_) === 0) + val c = VertexSetRDD(a, b.index) + assert(b.leftJoin(c) + .mapValues(x => x._1 + x._2.get).map(x=> x._2).reduce(_+_) === 0) + val d = c.filter(q => ((q._2 % 2) == 0)) + val e = a.filter(q => ((q._2 % 2) == 0)) + assert(d.count === e.count) + assert(b.zipJoin(c).mapValues(x => x._1 + x._2) + .map(x => x._2).reduce(_+_) === 0) + } + } + } From 73bf8587e29d9ea531fae68931f306619aba5402 Mon Sep 17 00:00:00 2001 From: Kyle Ellrott Date: Mon, 21 Oct 2013 15:13:31 -0700 Subject: [PATCH 136/531] Fixing graph/pom.xml --- graph/pom.xml | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/graph/pom.xml b/graph/pom.xml index 1cd9cda98bc4b..11f0c4fe7f650 100644 --- a/graph/pom.xml +++ b/graph/pom.xml @@ -2,9 +2,9 @@ 4.0.0 - org.spark-project - parent - 0.7.0-SNAPSHOT + org.apache.spark + spark-parent + 0.9.0-incubating-SNAPSHOT ../pom.xml @@ -15,6 +15,12 @@ http://spark-project.org/ + + org.apache.spark + spark-core_2.9.3 + ${project.version} + provided + org.eclipse.jetty jetty-server From 14a3329a11d7b38e0fd28807aa434dae19ca52f6 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Tue, 22 Oct 2013 15:01:20 -0700 Subject: [PATCH 137/531] Changing the Pregel interface slightly to better support type inference. --- .../scala/org/apache/spark/graph/Pregel.scala | 69 +++++++++++++++++-- 1 file changed, 63 insertions(+), 6 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala index 7ad6fda2a4570..d1f5513f6aae3 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala @@ -10,6 +10,8 @@ import org.apache.spark.rdd.RDD object Pregel { + + /** * Execute the Pregel program. * @@ -34,12 +36,11 @@ object Pregel { * @return the resulting graph at the end of the computation * */ - def iterate[VD: ClassManifest, ED: ClassManifest, A: ClassManifest](graph: Graph[VD, ED])( + def apply[VD: ClassManifest, ED: ClassManifest, A: ClassManifest] + (graph: Graph[VD, ED], initialMsg: A, numIter: Int)( vprog: (Vid, VD, A) => VD, sendMsg: (Vid, EdgeTriplet[VD, ED]) => Option[A], - mergeMsg: (A, A) => A, - initialMsg: A, - numIter: Int) + mergeMsg: (A, A) => A) : Graph[VD, ED] = { var g = graph @@ -61,5 +62,61 @@ object Pregel { } // Return the final graph g - } -} + } // end of apply + + + /** + * Execute the Pregel program. + * + * @tparam VD the vertex data type + * @tparam ED the edge data type + * @tparam A the Pregel message type + * + * @param vprog a user supplied function that acts as the vertex program for + * the Pregel computation. It takes the vertex ID of the vertex it is running on, + * the accompanying data for that vertex, and the incoming data and returns the + * new vertex value. + * @param sendMsg a user supplied function that takes the current vertex ID and an EdgeTriplet + * between the vertex and one of its neighbors and produces a message to send + * to that neighbor. + * @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.'' + * @param initialMsg the message each vertex will receive at the beginning of the + * first iteration. + * @param numIter the number of iterations to run this computation for. + * + * @return the resulting graph at the end of the computation + * + */ + def apply[VD: ClassManifest, ED: ClassManifest, A: ClassManifest] + (graph: Graph[VD, ED], initialMsg: A)( + vprog: (Vid, VD, A) => VD, + sendMsg: (Vid, EdgeTriplet[VD, ED]) => Option[A], + mergeMsg: (A, A) => A) + : Graph[VD, ED] = { + + var g = graph + //var g = graph.cache() + var i = 0 + + def mapF(vid: Vid, edge: EdgeTriplet[VD,ED]) = sendMsg(edge.otherVertexId(vid), edge) + + // Receive the first set of messages + g.mapVertices( (vid, vdata) => vprog(vid, vdata, initialMsg)) + + var activeMessages = g.numEdges + while (activeMessages > 0) { + // compute the messages + val messages = g.aggregateNeighbors(mapF, mergeMsg, EdgeDirection.In).cache + activeMessages = messages.count + // receive the messages + g = g.joinVertices(messages)(vprog) + // count the iteration + i += 1 + } + // Return the final graph + g + } // end of apply + +} // end of class Pregel From 46b195253ecb54ff8a202a53773fc9388b2c753c Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Tue, 22 Oct 2013 15:01:49 -0700 Subject: [PATCH 138/531] Adding some additional graph generators to support unit testing of the analytics package. --- .../spark/graph/util/GraphGenerators.scala | 46 ++++++++++++++++++- 1 file changed, 45 insertions(+), 1 deletion(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala b/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala index 895c65c14c903..1bbcce50399ab 100644 --- a/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala +++ b/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala @@ -236,7 +236,51 @@ object GraphGenerators { } } -} + + + /** + * 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 + * order. + * + * @param sc the spark context in which to construct the graph + * @param rows the number of rows + * @param cols the number of columns + * + * @return A graph containing vertices with the row and column ids + * as their attributes and edge values as 1.0. + */ + def gridGraph(sc: SparkContext, rows: Int, cols: Int): Graph[(Int,Int), Double] = { + // Convert row column address into vertex ids (row major order) + def sub2ind(r: Int, c: Int): Vid = r * cols + c + + val vertices: RDD[(Vid, (Int,Int))] = + sc.parallelize(0 until rows).flatMap( r => (0 until cols).map( c => (sub2ind(r,c), (r,c)) ) ) + val edges: RDD[Edge[Double]] = + vertices.flatMap{ case (vid, (r,c)) => + (if (r+1 < rows) { Seq( (sub2ind(r, c), sub2ind(r+1, c))) } else { Seq.empty }) ++ + (if (c+1 < cols) { Seq( (sub2ind(r, c), sub2ind(r, c+1))) } else { Seq.empty }) + }.map{ case (src, dst) => Edge(src, dst, 1.0) } + Graph(vertices, edges) + } // end of gridGraph + + /** + * Create a star graph with vertex 0 being the center. + * + * @param sc the spark context in which to construct the graph + * @param the number of vertices in the star + * + * @return A star graph containing `nverts` vertices with vertex 0 + * being the center vertex. + */ + def starGraph(sc: SparkContext, nverts: Int): Graph[Int, Int] = { + val edges: RDD[(Vid, Vid)] = sc.parallelize(1 until nverts).map(vid => (vid, 0)) + Graph(edges, false) + } // end of starGraph + + + +} // end of Graph Generators From ba5c75692a61ae86496d4285da5d2e453ce88c36 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Tue, 22 Oct 2013 15:03:00 -0700 Subject: [PATCH 139/531] Updating analytics to reflect changes in the pregel interface and moving degree information into the edge attribute. --- .../org/apache/spark/graph/Analytics.scala | 110 +++++++++++++----- 1 file changed, 83 insertions(+), 27 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala index 92632db491beb..2093eec3110a1 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala @@ -11,43 +11,99 @@ object Analytics extends Logging { */ def pagerank[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], numIter: Int, - resetProb: Double = 0.15) = { - // Compute the out degree of each vertex - val pagerankGraph = graph.outerJoinVertices(graph.outDegrees){ - (vid, vdata, deg) => (deg.getOrElse(0), 1.0) - } + 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. + */ + val pagerankGraph: Graph[Double, Double] = graph + // Associate the degree with each vertex + .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 + .mapVertices( (id, attr) => 1.0 ) + // Display statistics about pagerank println(pagerankGraph.statistics) - - Pregel.iterate[(Int, Double), ED, Double](pagerankGraph)( - (vid, data, a: Double) => (data._1, (resetProb + (1.0 - resetProb) * a)), // apply - (me_id, edge) => Some(edge.srcAttr._2 / edge.srcAttr._1), // gather - (a: Double, b: Double) => a + b, // merge - 1.0, - numIter).mapVertices{ case (id, (outDeg, r)) => r } + + // Define the three functions needed to implement PageRank in the GraphX + // version of Pregel + def vertexProgram(id: Vid, attr: Double, msgSum: Double): Double = + resetProb + (1.0 - resetProb) * msgSum + def sendMessage(id: Vid, edge: EdgeTriplet[Double, Double]): Option[Double] = + Some(edge.srcAttr / edge.attr) + def messageCombiner(a: Double, b: Double): Double = a + b + // The initial message received by all vertices in PageRank + val initialMessage = 1.0 + + // Execute pregel for a fixed number of iterations. + Pregel(pagerankGraph, initialMessage, numIter)( + vertexProgram, sendMessage, messageCombiner) } /** * Compute the PageRank of a graph returning the pagerank of each vertex as an RDD */ - def dynamicPagerank[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], - tol: Float, - maxIter: Int = Integer.MAX_VALUE, - resetProb: Double = 0.15) = { - // Compute the out degree of each vertex - val pagerankGraph = graph.outerJoinVertices(graph.outDegrees){ - (id, data, degIter) => (degIter.sum, 1.0, 1.0) + def dynamicPagerank[VD: Manifest, ED: Manifest]( + graph: Graph[VD, ED], tol: Float, 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. + */ + val pagerankGraph: Graph[(Double, Double), Double] = graph + // Associate the degree with each vertex + .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 (initalPR, delta = 0) + .mapVertices( (id, attr) => (resetProb, 0.0) ) + + // Display statistics about pagerank + println(pagerankGraph.statistics) + + // Define the three functions needed to implement PageRank in the GraphX + // version of Pregel + def vertexProgram(id: Vid, attr: (Double, Double), msgSum: Double): (Double, Double) = { + val (oldPR, lastDelta) = attr + val newPR = oldPR + (1.0 - resetProb) * msgSum + (newPR, newPR - oldPR) } + def sendMessage(id: Vid, edge: EdgeTriplet[(Double, Double), Double]): Option[Double] = { + if (edge.srcAttr._2 > tol) { + Some(edge.srcAttr._2 / edge.attr) + } else { None } + } + def messageCombiner(a: Double, b: Double): Double = a + b + // The initial message received by all vertices in PageRank + val initialMessage = 1.0 / (1.0 - resetProb) + + // Execute a dynamic version of Pregel. + Pregel(pagerankGraph, initialMessage)( + vertexProgram, sendMessage, messageCombiner) + .mapVertices( (vid, attr) => attr._1 ) + + + // // Compute the out degree of each vertex + // val pagerankGraph = graph.outerJoinVertices(graph.outDegrees){ + // (id, data, degIter) => (degIter.sum, 1.0, 1.0) + // } - // Run PageRank - GraphLab.iterate(pagerankGraph)( - (me_id, edge) => edge.srcAttr._2 / edge.srcAttr._1, // gather - (a: Double, b: Double) => a + b, - (id, data, a: Option[Double]) => - (data._1, (resetProb + (1.0 - resetProb) * a.getOrElse(0.0)), data._2), // apply - (me_id, edge) => math.abs(edge.srcAttr._3 - edge.srcAttr._2) > tol, // scatter - maxIter).mapVertices { case (vid, data) => data._2 } + // // Run PageRank + // GraphLab.iterate(pagerankGraph)( + // (me_id, edge) => edge.srcAttr._2 / edge.srcAttr._1, // gather + // (a: Double, b: Double) => a + b, + // (id, data, a: Option[Double]) => + // (data._1, (resetProb + (1.0 - resetProb) * a.getOrElse(0.0)), data._2), // apply + // (me_id, edge) => math.abs(edge.srcAttr._3 - edge.srcAttr._2) > tol, // scatter + // maxIter).mapVertices { case (vid, data) => data._2 } } From e3eb03d5b54db894670376b9c01a0c6c61aae083 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Tue, 22 Oct 2013 15:03:16 -0700 Subject: [PATCH 140/531] Starting analytics test suite. --- .../apache/spark/graph/AnalyticsSuite.scala | 30 +++++++++++++++++++ 1 file changed, 30 insertions(+) create mode 100644 graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala diff --git a/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala b/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala new file mode 100644 index 0000000000000..864d51e3f662b --- /dev/null +++ b/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala @@ -0,0 +1,30 @@ +package org.apache.spark.graph + +import org.scalatest.FunSuite + +import org.apache.spark.SparkContext +import org.apache.spark.graph.LocalSparkContext._ +import org.apache.spark.graph.util.GraphGenerators +import org.apache.spark.graph.Analytics + + +class AnalyticsSuite extends FunSuite with LocalSparkContext { + + System.setProperty("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + System.setProperty("spark.kryo.registrator", "org.apache.spark.graph.GraphKryoRegistrator") + + val sc = new Sparkcontext("local", "test") + + test("Fixed Iterations PageRank") { + val starGraph = GraphGenerators.starGraph(sc, 1000) + val resetProb = 0.15 + val prGraph1 = Analytics.pagerank(graph, 1, resetProb) + val prGraph2 = Analytics.pagerank(grpah, 2, resetProb) + val errors = prGraph1.vertices.zipJoin(prGraph2.vertices) + .map{ case (vid, (pr1, pr2)) => if (pr1 != pr2) { 1 } else { 0 } }.sum + + + } + + +} // end of AnalyticsSuite From 0bd92ed8d07712b7d8bb06378d877eb9643ba05a Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Tue, 22 Oct 2013 19:10:51 -0700 Subject: [PATCH 141/531] Fixing a bug in pregel where the initial vertex-program results are lost. --- graph/src/main/scala/org/apache/spark/graph/Pregel.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala index d1f5513f6aae3..2e3f86a3f0aa6 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala @@ -50,7 +50,7 @@ object Pregel { def mapF(vid: Vid, edge: EdgeTriplet[VD,ED]) = sendMsg(edge.otherVertexId(vid), edge) // Receive the first set of messages - g.mapVertices( (vid, vdata) => vprog(vid, vdata, initialMsg)) + g = g.mapVertices( (vid, vdata) => vprog(vid, vdata, initialMsg)) while (i < numIter) { // compute the messages From c30624dcbb6f6999f4e4f592ac4379a18f169927 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Wed, 23 Oct 2013 00:25:45 -0700 Subject: [PATCH 142/531] Adding dynamic pregel, fixing bugs in PageRank, and adding basic analytics unit tests. --- .../org/apache/spark/graph/Analytics.scala | 14 +-- .../scala/org/apache/spark/graph/Pregel.scala | 51 ++++++---- .../apache/spark/graph/AnalyticsSuite.scala | 97 +++++++++++++++++-- 3 files changed, 127 insertions(+), 35 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala index 2093eec3110a1..dc1955a8353f8 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala @@ -35,10 +35,10 @@ object Analytics extends Logging { def vertexProgram(id: Vid, attr: Double, msgSum: Double): Double = resetProb + (1.0 - resetProb) * msgSum def sendMessage(id: Vid, edge: EdgeTriplet[Double, Double]): Option[Double] = - Some(edge.srcAttr / edge.attr) + Some(edge.srcAttr * edge.attr) def messageCombiner(a: Double, b: Double): Double = a + b // The initial message received by all vertices in PageRank - val initialMessage = 1.0 + val initialMessage = 0.0 // Execute pregel for a fixed number of iterations. Pregel(pagerankGraph, initialMessage, numIter)( @@ -49,8 +49,8 @@ object Analytics extends Logging { /** * Compute the PageRank of a graph returning the pagerank of each vertex as an RDD */ - def dynamicPagerank[VD: Manifest, ED: Manifest]( - graph: Graph[VD, ED], tol: Float, resetProb: Double = 0.15): Graph[Double, Double] = { + def deltaPagerank[VD: Manifest, ED: Manifest]( + graph: Graph[VD, ED], tol: Double, resetProb: Double = 0.15): Graph[Double, Double] = { /** * Initialize the pagerankGraph with each edge attribute @@ -64,7 +64,7 @@ object Analytics extends Logging { // Set the weight on the edges based on the degree .mapTriplets( e => 1.0 / e.srcAttr ) // Set the vertex attributes to (initalPR, delta = 0) - .mapVertices( (id, attr) => (resetProb, 0.0) ) + .mapVertices( (id, attr) => (0.0, 0.0) ) // Display statistics about pagerank println(pagerankGraph.statistics) @@ -78,12 +78,12 @@ object Analytics extends Logging { } def sendMessage(id: Vid, edge: EdgeTriplet[(Double, Double), Double]): Option[Double] = { if (edge.srcAttr._2 > tol) { - Some(edge.srcAttr._2 / edge.attr) + Some(edge.srcAttr._2 * edge.attr) } else { None } } def messageCombiner(a: Double, b: Double): Double = a + b // The initial message received by all vertices in PageRank - val initialMessage = 1.0 / (1.0 - resetProb) + val initialMessage = resetProb / (1.0 - resetProb) // Execute a dynamic version of Pregel. Pregel(pagerankGraph, initialMessage)( diff --git a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala index 2e3f86a3f0aa6..94dc806fc2814 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala @@ -43,15 +43,12 @@ object Pregel { mergeMsg: (A, A) => A) : Graph[VD, ED] = { - var g = graph - //var g = graph.cache() - var i = 0 - def mapF(vid: Vid, edge: EdgeTriplet[VD,ED]) = sendMsg(edge.otherVertexId(vid), edge) // Receive the first set of messages - g = g.mapVertices( (vid, vdata) => vprog(vid, vdata, initialMsg)) - + var g = graph.mapVertices( (vid, vdata) => vprog(vid, vdata, initialMsg)) + + var i = 0 while (i < numIter) { // compute the messages val messages = g.aggregateNeighbors(mapF, mergeMsg, EdgeDirection.In) @@ -96,27 +93,45 @@ object Pregel { mergeMsg: (A, A) => A) : Graph[VD, ED] = { - var g = graph - //var g = graph.cache() - var i = 0 - - def mapF(vid: Vid, edge: EdgeTriplet[VD,ED]) = sendMsg(edge.otherVertexId(vid), edge) + def vprogFun(id: Vid, attr: (VD, Boolean), msgOpt: Option[A]): (VD, Boolean) = { + msgOpt match { + case Some(msg) => (vprog(id, attr._1, msg), true) + case None => (attr._1, false) + } + } - // Receive the first set of messages - g.mapVertices( (vid, vdata) => vprog(vid, vdata, initialMsg)) + def sendMsgFun(vid: Vid, edge: EdgeTriplet[(VD,Boolean), ED]): Option[A] = { + if(edge.srcAttr._2) { + val et = new EdgeTriplet[VD, ED] + et.srcId = edge.srcId + et.srcAttr = edge.srcAttr._1 + et.dstId = edge.dstId + et.dstAttr = edge.dstAttr._1 + et.attr = edge.attr + sendMsg(edge.otherVertexId(vid), et) + } else { None } + } - var activeMessages = g.numEdges + var g = graph.mapVertices( (vid, vdata) => (vprog(vid, vdata, initialMsg), true) ) + // compute the messages + var messages = g.aggregateNeighbors(sendMsgFun, mergeMsg, EdgeDirection.In).cache + var activeMessages = messages.count + // Loop + var i = 0 while (activeMessages > 0) { + // receive the messages + g = g.outerJoinVertices(messages)(vprogFun) + val oldMessages = messages // compute the messages - val messages = g.aggregateNeighbors(mapF, mergeMsg, EdgeDirection.In).cache + messages = g.aggregateNeighbors(sendMsgFun, mergeMsg, EdgeDirection.In).cache activeMessages = messages.count - // receive the messages - g = g.joinVertices(messages)(vprog) + // after counting we can unpersist the old messages + oldMessages.unpersist(blocking=false) // count the iteration i += 1 } // Return the final graph - g + g.mapVertices((id, attr) => attr._1) } // end of apply } // end of class Pregel diff --git a/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala b/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala index 864d51e3f662b..f4a8c6b4c9f4d 100644 --- a/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala @@ -3,9 +3,45 @@ package org.apache.spark.graph import org.scalatest.FunSuite import org.apache.spark.SparkContext +import org.apache.spark.SparkContext._ + import org.apache.spark.graph.LocalSparkContext._ + import org.apache.spark.graph.util.GraphGenerators -import org.apache.spark.graph.Analytics + + +object GridPageRank { + def apply(nRows: Int, nCols: Int, nIter: Int, resetProb: Double) = { + val inNbrs = Array.fill(nRows * nCols)(collection.mutable.MutableList.empty[Int]) + val outDegree = Array.fill(nRows * nCols)(0) + // Convert row column address into vertex ids (row major order) + def sub2ind(r: Int, c: Int): Int = r * nCols + c + // Make the grid graph + for(r <- 0 until nRows; c <- 0 until nCols){ + val ind = sub2ind(r,c) + if(r+1 < nRows) { + outDegree(ind) += 1 + inNbrs(sub2ind(r+1,c)) += ind + } + if(c+1 < nCols) { + outDegree(ind) += 1 + inNbrs(sub2ind(r,c+1)) += ind + } + } + // compute the pagerank + var pr = Array.fill(nRows * nCols)(resetProb) + for(iter <- 0 until nIter) { + val oldPr = pr + pr = new Array[Double](nRows * nCols) + for(ind <- 0 until (nRows * nCols)) { + pr(ind) = resetProb + (1.0 - resetProb) * + inNbrs(ind).map( nbr => oldPr(nbr) / outDegree(nbr)).sum + } + } + (0L until (nRows * nCols)).zip(pr) + } + +} class AnalyticsSuite extends FunSuite with LocalSparkContext { @@ -13,18 +49,59 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { System.setProperty("spark.serializer", "org.apache.spark.serializer.KryoSerializer") System.setProperty("spark.kryo.registrator", "org.apache.spark.graph.GraphKryoRegistrator") - val sc = new Sparkcontext("local", "test") - test("Fixed Iterations PageRank") { - val starGraph = GraphGenerators.starGraph(sc, 1000) - val resetProb = 0.15 - val prGraph1 = Analytics.pagerank(graph, 1, resetProb) - val prGraph2 = Analytics.pagerank(grpah, 2, resetProb) - val errors = prGraph1.vertices.zipJoin(prGraph2.vertices) - .map{ case (vid, (pr1, pr2)) => if (pr1 != pr2) { 1 } else { 0 } }.sum + test("Star PageRank") { + withSpark(new SparkContext("local", "test")) { sc => + val nVertices = 100 + val starGraph = GraphGenerators.starGraph(sc, nVertices) + val resetProb = 0.15 + val prGraph1 = Analytics.pagerank(starGraph, 1, resetProb) + val prGraph2 = Analytics.pagerank(starGraph, 2, resetProb) + + val notMatching = prGraph1.vertices.zipJoin(prGraph2.vertices) + .map{ case (vid, (pr1, pr2)) => if (pr1 != pr2) { 1 } else { 0 } }.sum + assert(notMatching === 0) + prGraph2.vertices.foreach(println(_)) + val errors = prGraph2.vertices.map{ case (vid, pr) => + val correct = (vid > 0 && pr == resetProb) || + (vid == 0 && math.abs(pr - (resetProb + (1.0 - resetProb) * (resetProb * (nVertices - 1)) )) < 1.0E-5) + if ( !correct ) { 1 } else { 0 } + } + assert(errors.sum === 0) + val prGraph3 = Analytics.deltaPagerank(starGraph, 0, resetProb) + val errors2 = prGraph2.vertices.leftJoin(prGraph3.vertices).map{ + case (_, (pr1, Some(pr2))) if(pr1 == pr2) => 0 + case _ => 1 + }.sum + assert(errors2 === 0) + } + } // end of test Star PageRank - } + + test("Grid PageRank") { + withSpark(new SparkContext("local", "test")) { sc => + val gridGraph = GraphGenerators.gridGraph(sc, 10, 10) + val resetProb = 0.15 + val prGraph1 = Analytics.pagerank(gridGraph, 50, resetProb).cache() + val prGraph2 = Analytics.deltaPagerank(gridGraph, 0.0001, resetProb).cache() + val error = prGraph1.vertices.zipJoin(prGraph2.vertices).map { + case (id, (a, b)) => (a - b) * (a - b) + }.sum + prGraph1.vertices.zipJoin(prGraph2.vertices) + .map{ case (id, (a,b)) => (id, (a,b, a-b))}.foreach(println(_)) + println(error) + assert(error < 1.0e-5) + val pr3 = sc.parallelize(GridPageRank(10,10, 50, resetProb)) + val error2 = prGraph1.vertices.leftJoin(pr3).map { + case (id, (a, Some(b))) => (a - b) * (a - b) + case _ => 0 + }.sum + prGraph1.vertices.leftJoin(pr3).foreach(println( _ )) + println(error2) + assert(error2 < 1.0e-5) + } + } // end of Grid PageRank } // end of AnalyticsSuite From 8236d5dcc472f8e8ff19a5e8788f926047edbbd7 Mon Sep 17 00:00:00 2001 From: Kyle Ellrott Date: Fri, 25 Oct 2013 15:52:44 -0700 Subject: [PATCH 143/531] More changes to the graph/pom.xml to make it match the other subprojects --- graph/pom.xml | 21 +++++++++++++++++++-- 1 file changed, 19 insertions(+), 2 deletions(-) diff --git a/graph/pom.xml b/graph/pom.xml index 11f0c4fe7f650..fd3dcaad7c6c5 100644 --- a/graph/pom.xml +++ b/graph/pom.xml @@ -1,4 +1,21 @@ + + 4.0.0 @@ -8,8 +25,8 @@ ../pom.xml - org.spark-project - spark-graph + org.apache.spark + spark-graph_2.9.3 jar Spark Graph http://spark-project.org/ From 00e73833cce88d5f77eaadd741a9a4046a4f17a3 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Sat, 26 Oct 2013 15:10:30 -0700 Subject: [PATCH 144/531] Fixing a bug in reverse edge direction. --- .../main/scala/org/apache/spark/graph/EdgeDirection.scala | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/EdgeDirection.scala b/graph/src/main/scala/org/apache/spark/graph/EdgeDirection.scala index 99af2d54580bf..a1468a152baa9 100644 --- a/graph/src/main/scala/org/apache/spark/graph/EdgeDirection.scala +++ b/graph/src/main/scala/org/apache/spark/graph/EdgeDirection.scala @@ -6,9 +6,13 @@ package org.apache.spark.graph * the set of adjacent neighbors when running a neighborhood query. */ sealed abstract class EdgeDirection { + /** + * Reverse the direction of an edge. An in becomes out, + * out becomes in and both remains both. + */ def reverse: EdgeDirection = this match { - case EdgeDirection.In => EdgeDirection.In - case EdgeDirection.Out => EdgeDirection.Out + case EdgeDirection.In => EdgeDirection.Out + case EdgeDirection.Out => EdgeDirection.In case EdgeDirection.Both => EdgeDirection.Both } } From 08024c938c81b6590745fb87e95f8d352f5ea011 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Sat, 26 Oct 2013 15:42:51 -0700 Subject: [PATCH 145/531] Adding more documentation to the Pregel API as well as additional functionality including the ability to specify the edge direction along which messages are computed. --- .../scala/org/apache/spark/graph/Pregel.scala | 259 +++++++++++++++--- 1 file changed, 224 insertions(+), 35 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala index 94dc806fc2814..d45e351d6af5d 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala @@ -4,34 +4,87 @@ import org.apache.spark.rdd.RDD /** - * This object implements the Pregel bulk-synchronous - * message-passing API. + * This object implements a Pregel-like bulk-synchronous + * message-passing API. However, unlike the original Pregel API + * the GraphX pregel API factors the sendMessage computation over + * edges, enables the message sending computation to read both + * vertex attributes, and finally contrains messages to the graph + * structure. These changes allow for substantially more efficient + * distributed execution while also exposing greater flexibility + * for graph based computation. + * + * This object present several variants of the bulk synchronous + * execution that differ only in the edge direction along which + * messages are sent and whether a fixed number of iterations + * is used. + * + * @example We can use the Pregel abstraction to implement PageRank + * {{{ + * val pagerankGraph: Graph[Double, Double] = graph + * // Associate the degree with each vertex + * .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 + * .mapVertices( (id, attr) => 1.0 ) + * + * def vertexProgram(id: Vid, attr: Double, msgSum: Double): Double = + * resetProb + (1.0 - resetProb) * msgSum + * def sendMessage(id: Vid, edge: EdgeTriplet[Double, Double]): Option[Double] = + * Some(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. + * Pregel(pagerankGraph, initialMessage, numIter)( + * vertexProgram, sendMessage, messageCombiner) + * }}} + * */ object Pregel { - - /** - * Execute the Pregel program. + * 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 a fixed number (`numIter`) of iterations. * * @tparam VD the vertex data type * @tparam ED the edge data type * @tparam A the Pregel message type * - * @param vprog a user supplied function that acts as the vertex program for - * the Pregel computation. It takes the vertex ID of the vertex it is running on, - * the accompanying data for that vertex, and the incoming data and returns the - * new vertex value. - * @param sendMsg a user supplied function that takes the current vertex ID and an EdgeTriplet - * between the vertex and one of its neighbors and produces a message to send - * to that neighbor. - * @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.'' - * @param initialMsg the message each vertex will receive at the beginning of the - * first iteration. - * @param numIter the number of iterations to run this computation for. + * @param graph the input graph. + * + * @param initialMsg the message each vertex will receive at the + * on the first iteration. + * + * @param numIter the number of iterations to run this computation. + * + * @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 * @@ -42,6 +95,64 @@ object Pregel { sendMsg: (Vid, EdgeTriplet[VD, ED]) => Option[A], mergeMsg: (A, A) => A) : Graph[VD, ED] = { + apply(graph, initialMsg, numIter, EdgeDirection.Out)(vprog, sendMsg, mergeMsg) + } // end of Apply + + + /** + * 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 a fixed number (`numIter`) of 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 numIter the number of iterations to run this computation. + * + * @param sendDir the edge direction along which the `sendMsg` function + * is invoked. + * + * @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 each edge + * in the direction `sendDir` adjacent to 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 apply[VD: ClassManifest, ED: ClassManifest, A: ClassManifest] + (graph: Graph[VD, ED], initialMsg: A, numIter: Int, sendDir: EdgeDirection)( + vprog: (Vid, VD, A) => VD, + sendMsg: (Vid, EdgeTriplet[VD, ED]) => Option[A], + mergeMsg: (A, A) => A) + : Graph[VD, ED] = { def mapF(vid: Vid, edge: EdgeTriplet[VD,ED]) = sendMsg(edge.otherVertexId(vid), edge) @@ -51,7 +162,7 @@ object Pregel { var i = 0 while (i < numIter) { // compute the messages - val messages = g.aggregateNeighbors(mapF, mergeMsg, EdgeDirection.In) + val messages = g.aggregateNeighbors(mapF, mergeMsg, sendDir.reverse) // receive the messages g = g.joinVertices(messages)(vprog) // count the iteration @@ -63,25 +174,45 @@ object Pregel { /** - * Execute the Pregel program. + * 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. * * @tparam VD the vertex data type * @tparam ED the edge data type * @tparam A the Pregel message type * - * @param vprog a user supplied function that acts as the vertex program for - * the Pregel computation. It takes the vertex ID of the vertex it is running on, - * the accompanying data for that vertex, and the incoming data and returns the - * new vertex value. - * @param sendMsg a user supplied function that takes the current vertex ID and an EdgeTriplet - * between the vertex and one of its neighbors and produces a message to send - * to that neighbor. - * @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.'' - * @param initialMsg the message each vertex will receive at the beginning of the - * first iteration. - * @param numIter the number of iterations to run this computation for. + * @param graph the input graph. + * + * @param initialMsg the message each vertex will receive at the + * on the first iteration. + * + * @param numIter the number of iterations to run this computation. + * + * @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 * @@ -92,6 +223,64 @@ object Pregel { sendMsg: (Vid, EdgeTriplet[VD, ED]) => Option[A], mergeMsg: (A, A) => A) : Graph[VD, ED] = { + apply(graph, initialMsg, EdgeDirection.Out)(vprog, sendMsg, mergeMsg) + } // end of apply + + + /** + * 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. + * + * @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 numIter the number of iterations to run this computation. + * + * @param sendDir the edge direction along which the `sendMsg` function + * is invoked. + * + * @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 each edge + * in the direction `sendDir` adjacent to 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 apply[VD: ClassManifest, ED: ClassManifest, A: ClassManifest] + (graph: Graph[VD, ED], initialMsg: A, sendDir: EdgeDirection)( + vprog: (Vid, VD, A) => VD, + sendMsg: (Vid, EdgeTriplet[VD, ED]) => Option[A], + mergeMsg: (A, A) => A) + : Graph[VD, ED] = { def vprogFun(id: Vid, attr: (VD, Boolean), msgOpt: Option[A]): (VD, Boolean) = { msgOpt match { @@ -114,7 +303,7 @@ object Pregel { var g = graph.mapVertices( (vid, vdata) => (vprog(vid, vdata, initialMsg), true) ) // compute the messages - var messages = g.aggregateNeighbors(sendMsgFun, mergeMsg, EdgeDirection.In).cache + var messages = g.aggregateNeighbors(sendMsgFun, mergeMsg, sendDir.reverse).cache var activeMessages = messages.count // Loop var i = 0 @@ -123,7 +312,7 @@ object Pregel { g = g.outerJoinVertices(messages)(vprogFun) val oldMessages = messages // compute the messages - messages = g.aggregateNeighbors(sendMsgFun, mergeMsg, EdgeDirection.In).cache + messages = g.aggregateNeighbors(sendMsgFun, mergeMsg, sendDir.reverse).cache activeMessages = messages.count // after counting we can unpersist the old messages oldMessages.unpersist(blocking=false) From 6a0fbc037417575fec248c7fb2d2249bf031fa07 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Sat, 26 Oct 2013 15:44:19 -0700 Subject: [PATCH 146/531] Updating the GraphLab API to match the changes made to the Pregel API. --- .../scala/org/apache/spark/graph/GraphLab.scala | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala b/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala index 2f2a624592de1..b8503ab7fdb6c 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala @@ -33,14 +33,14 @@ object GraphLab { * @tparam A The type accumulated during the gather phase * @return the resulting graph after the algorithm converges */ - def iterate[VD: ClassManifest, ED: ClassManifest, A: ClassManifest](graph: Graph[VD, ED])( - gatherFunc: (Vid, EdgeTriplet[VD, ED]) => A, - mergeFunc: (A, A) => A, - applyFunc: (Vid, VD, Option[A]) => VD, - scatterFunc: (Vid, EdgeTriplet[VD, ED]) => Boolean, - numIter: Int = Integer.MAX_VALUE, - gatherDirection: EdgeDirection = EdgeDirection.In, - scatterDirection: EdgeDirection = EdgeDirection.Out): Graph[VD, ED] = { + def apply[VD: ClassManifest, ED: ClassManifest, A: ClassManifest] + (graph: Graph[VD, ED], numIter: Int, + gatherDirection: EdgeDirection = EdgeDirection.In, + scatterDirection: EdgeDirection = EdgeDirection.Out) + (gatherFunc: (Vid, EdgeTriplet[VD, ED]) => A, + mergeFunc: (A, A) => A, + applyFunc: (Vid, VD, Option[A]) => VD, + scatterFunc: (Vid, EdgeTriplet[VD, ED]) => Boolean): Graph[VD, ED] = { // Add an active attribute to all vertices to track convergence. From a2287ae1380af9de6c8b6e05d2c031de7d81fc71 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Sun, 27 Oct 2013 10:42:11 -0700 Subject: [PATCH 147/531] Implementing connected components on top of pregel like abstraction. --- .../org/apache/spark/graph/Analytics.scala | 135 +++++++++++++----- 1 file changed, 102 insertions(+), 33 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala index dc1955a8353f8..8feb42490da80 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala @@ -4,14 +4,54 @@ import org.apache.spark._ +/** + * The Analytics object contains a collection of basic graph analytics + * algorithms that operate largely on the graph structure. + * + * In addition the Analytics object contains a driver `main` which can + * be used to apply the various functions to graphs in standard formats. + */ object Analytics extends Logging { /** - * Compute the PageRank of a graph returning the pagerank of each vertex as an RDD + * Run PageRank for a fixed number of iterations returning a graph + * 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) + * + * @param graph the graph on which to compute PageRank + * @param numIter the number of iterations of PageRank to run + * @param resetProb the random reset probability (alpha) + * + * @return the graph containing with each vertex containing the PageRank and + * each edge containing the normalized weight. + * */ - def pagerank[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], - numIter: Int, - resetProb: Double = 0.15): Graph[Double, Double] = { + def pagerank[VD: Manifest, ED: Manifest]( + graph: Graph[VD, ED], numIter: Int, resetProb: Double = 0.15): + Graph[Double, Double] = { /** * Initialize the pagerankGraph with each edge attribute @@ -45,12 +85,42 @@ object Analytics extends Logging { vertexProgram, sendMessage, messageCombiner) } - /** - * Compute the PageRank of a graph returning the pagerank of each vertex as an RDD + * 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) + * + * @param graph the graph on which to compute PageRank + * @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 containing the normalized weight. */ def deltaPagerank[VD: Manifest, ED: Manifest]( - graph: Graph[VD, ED], tol: Double, resetProb: Double = 0.15): Graph[Double, Double] = { + graph: Graph[VD, ED], tol: Double, resetProb: Double = 0.15): + Graph[Double, Double] = { /** * Initialize the pagerankGraph with each edge attribute @@ -89,22 +159,7 @@ object Analytics extends Logging { Pregel(pagerankGraph, initialMessage)( vertexProgram, sendMessage, messageCombiner) .mapVertices( (vid, attr) => attr._1 ) - - - // // Compute the out degree of each vertex - // val pagerankGraph = graph.outerJoinVertices(graph.outDegrees){ - // (id, data, degIter) => (degIter.sum, 1.0, 1.0) - // } - - // // Run PageRank - // GraphLab.iterate(pagerankGraph)( - // (me_id, edge) => edge.srcAttr._2 / edge.srcAttr._1, // gather - // (a: Double, b: Double) => a + b, - // (id, data, a: Option[Double]) => - // (data._1, (resetProb + (1.0 - resetProb) * a.getOrElse(0.0)), data._2), // apply - // (me_id, edge) => math.abs(edge.srcAttr._3 - edge.srcAttr._2) > tol, // scatter - // maxIter).mapVertices { case (vid, data) => data._2 } - } + } // end of deltaPageRank /** @@ -113,16 +168,30 @@ object Analytics extends Logging { * lowest vertex id in the connected component containing * that vertex. */ - def connectedComponents[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], numIter: Int) = { + def connectedComponents[VD: Manifest, ED: Manifest](graph: Graph[VD, ED]) = { val ccGraph = graph.mapVertices { case (vid, _) => vid } - GraphLab.iterate(ccGraph)( - (me_id, edge) => edge.otherVertexAttr(me_id), // gather - (a: Vid, b: Vid) => math.min(a, b), // merge - (id, data, a: Option[Vid]) => math.min(data, a.getOrElse(Long.MaxValue)), // apply - (me_id, edge) => (edge.vertexAttr(me_id) < edge.otherVertexAttr(me_id)), // scatter - numIter, - gatherDirection = EdgeDirection.Both, scatterDirection = EdgeDirection.Both - ) + + def sendMessage(id: Vid, edge: EdgeTriplet[Vid, ED]): Option[Vid] = { + val thisAttr = edge.vertexAttr(id) + val otherAttr = edge.otherVertexAttr(id) + if(thisAttr < otherAttr) { Some(thisAttr) } + else { None } + } + + val initialMessage = Long.MaxValue + Pregel(ccGraph, initialMessage)( + (id, attr, msg) => math.min(attr, msg), + sendMessage, + (a,b) => math.min(a,b) + ) + + // GraphLab(ccGraph, gatherDirection = EdgeDirection.Both, scatterDirection = EdgeDirection.Both)( + // (me_id, edge) => edge.otherVertexAttr(me_id), // gather + // (a: Vid, b: Vid) => math.min(a, b), // merge + // (id, data, a: Option[Vid]) => math.min(data, a.getOrElse(Long.MaxValue)), // apply + // (me_id, edge) => (edge.vertexAttr(me_id) < edge.otherVertexAttr(me_id)) + // ) + } def main(args: Array[String]) = { @@ -238,7 +307,7 @@ object Analytics extends Logging { //val graph = GraphLoader.textFile(sc, fname, a => 1.0F) val graph = GraphLoader.textFile(sc, fname, a => 1.0F, minEdgePartitions = numEPart, minVertexPartitions = numVPart).cache() - val cc = Analytics.connectedComponents(graph, numIter) + val cc = Analytics.connectedComponents(graph) //val cc = if(isDynamic) Analytics.dynamicConnectedComponents(graph, numIter) // else Analytics.connectedComponents(graph, numIter) println("Components: " + cc.vertices.map{ case (vid,data) => data}.distinct()) From d6a902f309e914560ec9a49ca539c47b928d107a Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Mon, 28 Oct 2013 11:52:26 -0700 Subject: [PATCH 148/531] Finished updating connected components to used Pregel like abstraction and created a series of tests in the AnalyticsSuite. --- .../org/apache/spark/graph/Analytics.scala | 21 +++++- .../apache/spark/graph/AnalyticsSuite.scala | 65 +++++++++++++++++++ 2 files changed, 83 insertions(+), 3 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala index 8feb42490da80..acb9e3753f8e2 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala @@ -167,8 +167,17 @@ object Analytics extends Logging { * and return an RDD with the vertex value containing the * lowest vertex id in the connected component containing * that vertex. + * + * @tparam VD the vertex attribute type (discarded in the computation) + * @tparam ED the edge attribute type (preserved in the computation) + * + * @param graph the graph for which to compute the connected components + * + * @return a graph with vertex attributes containing the smallest vertex + * in each connected component */ - def connectedComponents[VD: Manifest, ED: Manifest](graph: Graph[VD, ED]) = { + def connectedComponents[VD: Manifest, ED: Manifest](graph: Graph[VD, ED]): + Graph[Vid, ED] = { val ccGraph = graph.mapVertices { case (vid, _) => vid } def sendMessage(id: Vid, edge: EdgeTriplet[Vid, ED]): Option[Vid] = { @@ -179,21 +188,27 @@ object Analytics extends Logging { } val initialMessage = Long.MaxValue - Pregel(ccGraph, initialMessage)( + Pregel(ccGraph, initialMessage, EdgeDirection.Both)( (id, attr, msg) => math.min(attr, msg), sendMessage, (a,b) => math.min(a,b) ) + /** + * Originally this was implemented using the GraphLab abstraction but with + * support for message computation along all edge directions the pregel + * abstraction is sufficient + */ // GraphLab(ccGraph, gatherDirection = EdgeDirection.Both, scatterDirection = EdgeDirection.Both)( // (me_id, edge) => edge.otherVertexAttr(me_id), // gather // (a: Vid, b: Vid) => math.min(a, b), // merge // (id, data, a: Option[Vid]) => math.min(data, a.getOrElse(Long.MaxValue)), // apply // (me_id, edge) => (edge.vertexAttr(me_id) < edge.otherVertexAttr(me_id)) // ) + } // end of connectedComponents - } + def main(args: Array[String]) = { val host = args(0) val taskType = args(1) diff --git a/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala b/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala index f4a8c6b4c9f4d..8d0b2e0b02b75 100644 --- a/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala @@ -79,6 +79,7 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { } // end of test Star PageRank + test("Grid PageRank") { withSpark(new SparkContext("local", "test")) { sc => val gridGraph = GraphGenerators.gridGraph(sc, 10, 10) @@ -104,4 +105,68 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { } // end of Grid PageRank + test("Grid Connected Components") { + withSpark(new SparkContext("local", "test")) { sc => + val gridGraph = GraphGenerators.gridGraph(sc, 10, 10) + val ccGraph = Analytics.connectedComponents(gridGraph).cache() + val maxCCid = ccGraph.vertices.map { case (vid, ccId) => ccId }.sum + assert(maxCCid === 0) + } + } // end of Grid connected components + + + test("Reverse Grid Connected Components") { + withSpark(new SparkContext("local", "test")) { sc => + val gridGraph = GraphGenerators.gridGraph(sc, 10, 10).reverse + val ccGraph = Analytics.connectedComponents(gridGraph).cache() + val maxCCid = ccGraph.vertices.map { case (vid, ccId) => ccId }.sum + assert(maxCCid === 0) + } + } // end of Grid connected components + + + test("Chain Connected Components") { + withSpark(new SparkContext("local", "test")) { sc => + val chain1 = (0 until 9).map(x => (x, x+1) ) + val chain2 = (10 until 20).map(x => (x, x+1) ) + val rawEdges = sc.parallelize(chain1 ++ chain2, 3).map { case (s,d) => (s.toLong, d.toLong) } + val twoChains = Graph(rawEdges) + val ccGraph = Analytics.connectedComponents(twoChains).cache() + val vertices = ccGraph.vertices.collect + for ( (id, cc) <- vertices ) { + if(id < 10) { assert(cc === 0) } + else { assert(cc === 10) } + } + val ccMap = vertices.toMap + println(ccMap) + for( id <- 0 until 20 ) { + if(id < 10) { assert(ccMap(id) === 0) } + else { assert(ccMap(id) === 10) } + } + } + } // end of chain connected components + + test("Reverse Chain Connected Components") { + withSpark(new SparkContext("local", "test")) { sc => + val chain1 = (0 until 9).map(x => (x, x+1) ) + val chain2 = (10 until 20).map(x => (x, x+1) ) + val rawEdges = sc.parallelize(chain1 ++ chain2, 3).map { case (s,d) => (s.toLong, d.toLong) } + val twoChains = Graph(rawEdges).reverse + val ccGraph = Analytics.connectedComponents(twoChains).cache() + val vertices = ccGraph.vertices.collect + for ( (id, cc) <- vertices ) { + if(id < 10) { assert(cc === 0) } + else { assert(cc === 10) } + } + val ccMap = vertices.toMap + println(ccMap) + for( id <- 0 until 20 ) { + if(id < 10) { assert(ccMap(id) === 0) } + else { assert(ccMap(id) === 10) } + } + } + } // end of chain connected components + + + } // end of AnalyticsSuite From 942de9843348a845f94b0654d6deafb948cf7e93 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Tue, 29 Oct 2013 10:19:49 -0700 Subject: [PATCH 149/531] Making suggested changes. --- .../scala/org/apache/spark/graph/Analytics.scala | 14 +------------- .../main/scala/org/apache/spark/graph/Pregel.scala | 2 +- 2 files changed, 2 insertions(+), 14 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala index acb9e3753f8e2..ddbd5becceb05 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala @@ -26,7 +26,7 @@ object Analytics extends Logging { * 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 + * PR[i] = alpha + (1 - alpha) * inNbrs[i].map(j => oldPR[j] / outDeg[j]).sum * } * } * }}} @@ -193,18 +193,6 @@ object Analytics extends Logging { sendMessage, (a,b) => math.min(a,b) ) - - /** - * Originally this was implemented using the GraphLab abstraction but with - * support for message computation along all edge directions the pregel - * abstraction is sufficient - */ - // GraphLab(ccGraph, gatherDirection = EdgeDirection.Both, scatterDirection = EdgeDirection.Both)( - // (me_id, edge) => edge.otherVertexAttr(me_id), // gather - // (a: Vid, b: Vid) => math.min(a, b), // merge - // (id, data, a: Option[Vid]) => math.min(data, a.getOrElse(Long.MaxValue)), // apply - // (me_id, edge) => (edge.vertexAttr(me_id) < edge.otherVertexAttr(me_id)) - // ) } // end of connectedComponents diff --git a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala index d45e351d6af5d..729eaa7eaef90 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala @@ -8,7 +8,7 @@ import org.apache.spark.rdd.RDD * message-passing API. However, unlike the original Pregel API * the GraphX pregel API factors the sendMessage computation over * edges, enables the message sending computation to read both - * vertex attributes, and finally contrains messages to the graph + * vertex attributes, and finally constrains messages to the graph * structure. These changes allow for substantially more efficient * distributed execution while also exposing greater flexibility * for graph based computation. From 77626d15071a19e5173cb682a07871382078873f Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Tue, 29 Oct 2013 11:05:42 -0700 Subject: [PATCH 150/531] Adding collect neighbors and documenting GraphOps. --- .../org/apache/spark/graph/GraphOps.scala | 94 ++++++++++++++++--- 1 file changed, 81 insertions(+), 13 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala b/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala index cecd3ff2913fc..d05711c9eb469 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala @@ -5,21 +5,62 @@ import org.apache.spark.SparkContext._ import org.apache.spark.util.ClosureCleaner -class GraphOps[VD: ClassManifest, ED: ClassManifest](graph: Graph[VD, ED]) { - +/** + * `GraphOps` contains additional functionality (syntatic sugar) for the graph + * type and is implicitly constructed for each Graph object. All operations in + * `GraphOps` are expressed in terms of the efficient GraphX API. + * + * @tparam VD the vertex attribute type + * @tparam ED the edge attribute type + * + */ +class GraphOps[VD: ClassManifest, ED: ClassManifest](graph: Graph[VD, ED]) { + /** + * Compute the number of edges in the graph. + */ lazy val numEdges: Long = graph.edges.count() + + /** + * Compute the number of vertices in the graph. + */ lazy val numVertices: Long = graph.vertices.count() + + /** + * Compute the in-degree of each vertex in the Graph returning an RDD. + * @note Vertices with no in edges are not returned in the resulting RDD. + */ lazy val inDegrees: VertexSetRDD[Int] = degreesRDD(EdgeDirection.In) + + /** + * Compute the out-degree of each vertex in the Graph returning an RDD. + * @note Vertices with no out edges are not returned in the resulting RDD. + */ lazy val outDegrees: VertexSetRDD[Int] = degreesRDD(EdgeDirection.Out) + + /** + * Compute the degrees of each vertex in the Graph returning an RDD. + * @note Vertices with no edges are not returned in the resulting RDD. + */ lazy val degrees: VertexSetRDD[Int] = degreesRDD(EdgeDirection.Both) + /** + * Compute the neighboring vertex degrees. + * + * @param edgeDirection the direction along which to collect neighboring + * vertex attributes. + */ + private def degreesRDD(edgeDirection: EdgeDirection): VertexSetRDD[Int] = { + graph.aggregateNeighbors((vid, edge) => Some(1), _+_, edgeDirection) + } + + /** * This function is used to compute a statistic for the neighborhood of each * vertex and returns a value for all vertices (including those without @@ -94,7 +135,16 @@ class GraphOps[VD: ClassManifest, ED: ClassManifest](graph: Graph[VD, ED]) { } // end of aggregateNeighbors - def collectNeighborIds(edgeDirection: EdgeDirection) : VertexSetRDD[Array[Vid]] = { + /** + * Return the Ids of the neighboring vertices. + * + * @param edgeDirection the direction along which to collect + * neighboring vertices + * + * @return the vertex set of neighboring ids for each vertex. + */ + def collectNeighborIds(edgeDirection: EdgeDirection) : + VertexSetRDD[Array[Vid]] = { val nbrs = graph.aggregateNeighbors[Array[Vid]]( (vid, edge) => Some(Array(edge.otherVertexId(vid))), (a, b) => a ++ b, @@ -104,12 +154,35 @@ class GraphOps[VD: ClassManifest, ED: ClassManifest](graph: Graph[VD, ED]) { case (_, Some(nbrs)) => nbrs case (_, None) => Array.empty[Vid] } - } + } // end of collectNeighborIds - private def degreesRDD(edgeDirection: EdgeDirection): VertexSetRDD[Int] = { - graph.aggregateNeighbors((vid, edge) => Some(1), _+_, edgeDirection) - } + /** + * Collect the neighbor vertex attributes for each vertex. + * + * @note This function could be highly inefficient on power-law + * graphs where high degree vertices may force a large ammount of + * information to be collected to a single location. + * + * @param edgeDirection the direction along which to collect + * neighboring vertices + * + * @return the vertex set of neighboring vertex attributes + * for each vertex. + */ + def collectNeighbors(edgeDirection: EdgeDirection) : + VertexSetRDD[ Array[(Vid, VD)] ] = { + val nbrs = graph.aggregateNeighbors[Array[(Vid,VD)]]( + (vid, edge) => + Some(Array( (edge.otherVertexId(vid), edge.otherVertexAttr(vid)) )), + (a, b) => a ++ b, + edgeDirection) + + graph.vertices.leftZipJoin(nbrs).mapValues{ + case (_, Some(nbrs)) => nbrs + case (_, None) => Array.empty[(Vid, VD)] + } + } // end of collectNeighbor /** @@ -139,11 +212,6 @@ class GraphOps[VD: ClassManifest, ED: ClassManifest](graph: Graph[VD, ED]) { * (v, deg) => deg ) * }}} * - * @todo Should this function be curried to enable type inference? For - * example - * {{{ - * graph.joinVertices(tbl)( (v, row) => row ) - * }}} */ def joinVertices[U: ClassManifest](table: RDD[(Vid, U)])(mapFunc: (Vid, VD, U) => VD) : Graph[VD, ED] = { @@ -158,4 +226,4 @@ class GraphOps[VD: ClassManifest, ED: ClassManifest](graph: Graph[VD, ED]) { graph.outerJoinVertices(table)(uf) } -} +} // end of GraphOps From 19da8820fcc33b8a4a5f11ee11526cc19c198a91 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Tue, 29 Oct 2013 11:06:06 -0700 Subject: [PATCH 151/531] Minor modifications to documentation. --- graph/src/main/scala/org/apache/spark/graph/Graph.scala | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/Graph.scala b/graph/src/main/scala/org/apache/spark/graph/Graph.scala index 8c7ee1fcefc86..b3253934e6c8e 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Graph.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Graph.scala @@ -12,10 +12,11 @@ import org.apache.spark.util.ClosureCleaner * manipulate the data associated with vertices and edges as well as the * underlying structure. Like Spark RDDs, the graph is a functional * data-structure in which mutating operations return new graphs. + * + * @note The majority of the graph operations are implemented in `GraphOps`. * - * @tparam VD The type of object associated with each vertex. - * - * @tparam ED The type of object associated with each edge + * @tparam VD the vertex attribute type + * @tparam ED the edge attribute type */ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { From d316cad9b1cb93e186edc31a438a5be253bda4fd Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Tue, 29 Oct 2013 13:58:04 -0700 Subject: [PATCH 152/531] Documented Graph.appy functions. --- .../scala/org/apache/spark/graph/Graph.scala | 281 +++++++++++------- 1 file changed, 179 insertions(+), 102 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/Graph.scala b/graph/src/main/scala/org/apache/spark/graph/Graph.scala index b3253934e6c8e..89e1b4ea01a81 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Graph.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Graph.scala @@ -7,13 +7,19 @@ import org.apache.spark.util.ClosureCleaner /** - * The Graph abstractly represents a graph with arbitrary objects associated - * with vertices and edges. The graph provides basic operations to access and - * manipulate the data associated with vertices and edges as well as the - * underlying structure. Like Spark RDDs, the graph is a functional - * data-structure in which mutating operations return new graphs. + * The Graph abstractly represents a graph with arbitrary objects + * associated with vertices and edges. The graph provides basic + * operations to access and manipulate the data associated with + * vertices and edges as well as the underlying structure. Like Spark + * RDDs, the graph is a functional data-structure in which mutating + * operations return new graphs. + * + * @see GraphOps for additional graph member functions. * - * @note The majority of the graph operations are implemented in `GraphOps`. + * @note The majority of the graph operations are implemented in + * `GraphOps`. All the convenience operations are defined in the + * `GraphOps` class which may be shared across multiple graph + * implementations. * * @tparam VD the vertex attribute type * @tparam ED the edge attribute type @@ -32,28 +38,28 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { val vertices: VertexSetRDD[VD] /** - * Get the Edges and their data as an RDD. The entries in the RDD contain - * just the source id and target id along with the edge data. + * Get the Edges and their data as an RDD. The entries in the RDD + * contain just the source id and target id along with the edge + * data. * * @return An RDD containing the edges in this graph * * @see Edge for the edge type. - * @see edgesWithVertices to get an RDD which contains all the edges along - * with their vertex data. + * @see edgesWithVertices to get an RDD which contains all the edges + * along with their vertex data. * - * @todo Should edges return 3 tuples instead of Edge objects? In this case - * we could rename EdgeTriplet to Edge? */ val edges: RDD[Edge[ED]] /** - * Get the edges with the vertex data associated with the adjacent pair of - * vertices. + * Get the edges with the vertex data associated with the adjacent + * pair of vertices. * * @return An RDD containing edge triplets. * - * @example This operation might be used to evaluate a graph coloring where - * we would like to check that both vertices are a different color. + * @example This operation might be used to evaluate a graph + * coloring where we would like to check that both vertices are a + * different color. * {{{ * type Color = Int * val graph: Graph[Color, Int] = Graph.textFile("hdfs://file.tsv") @@ -61,15 +67,16 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * .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. + * @see edges() If only the edge data and adjacent vertex ids are + * required. * */ val triplets: RDD[EdgeTriplet[VD, ED]] /** - * Return a graph that is cached when first created. This is used to pin a - * graph in memory enabling multiple queries to reuse the same construction - * process. + * Return a graph that is cached when first created. This is used to + * pin a graph in memory enabling multiple queries to reuse the same + * construction process. * * @see RDD.cache() for a more detailed explanation of caching. */ @@ -84,19 +91,19 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { /** - * Construct a new graph where each vertex value has been transformed by the - * map function. + * Construct a new graph where each vertex value has been + * transformed by the map function. * - * @note This graph is not changed and that the new graph has the same - * structure. As a consequence the underlying index structures can be - * reused. + * @note This graph is not changed and that the new graph has the + * same structure. As a consequence the underlying index structures + * can be reused. * * @param map the function from a vertex object to a new vertex value. * * @tparam VD2 the new vertex data type * - * @example We might use this operation to change the vertex values from one - * type to another to initialize an algorithm. + * @example We might use this operation to change the vertex values + * from one type to another to initialize an algorithm. * {{{ * val rawGraph: Graph[(), ()] = Graph.textFile("hdfs://file") * val root = 42 @@ -108,40 +115,42 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { def mapVertices[VD2: ClassManifest](map: (Vid, VD) => VD2): Graph[VD2, ED] /** - * Construct a new graph where each the value of each edge is transformed by - * the map operation. This function is not passed the vertex value for the - * vertices adjacent to the edge. If vertex values are desired use the - * mapTriplets function. + * Construct a new graph where each the value of each edge is + * transformed by the map operation. This function is not passed + * the vertex value for the vertices adjacent to the edge. If + * vertex values are desired use the mapTriplets function. * - * @note This graph is not changed and that the new graph has the same - * structure. As a consequence the underlying index structures can be - * reused. + * @note This graph is not changed and that the new graph has the + * same structure. As a consequence the underlying index structures + * can be reused. * * @param map the function from an edge object to a new edge value. * * @tparam ED2 the new edge data type * - * @example This function might be used to initialize edge attributes. + * @example This function might be used to initialize edge + * attributes. * */ def mapEdges[ED2: ClassManifest](map: Edge[ED] => ED2): Graph[VD, ED2] /** - * Construct a new graph where each the value of each edge is transformed by - * the map operation. This function passes vertex values for the adjacent - * vertices to the map function. If adjacent vertex values are not required, - * consider using the mapEdges function instead. + * Construct a new graph where each the value of each edge is + * transformed by the map operation. This function passes vertex + * values for the adjacent vertices to the map function. If + * adjacent vertex values are not required, consider using the + * mapEdges function instead. * - * @note This graph is not changed and that the new graph has the same - * structure. As a consequence the underlying index structures can be - * reused. + * @note This graph is not changed and that the new graph has the + * same structure. As a consequence the underlying index structures + * can be reused. * * @param map the function from an edge object to a new edge value. * * @tparam ED2 the new edge data type * - * @example This function might be used to initialize edge attributes based - * on the attributes associated with each vertex. + * @example This function might be used to initialize edge + * attributes based on the attributes associated with each vertex. * {{{ * val rawGraph: Graph[Int, Int] = someLoadFunction() * val graph = rawGraph.mapTriplets[Int]( edge => @@ -154,30 +163,35 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { /** - * Construct a new graph with all the edges reversed. If this graph contains - * an edge from a to b then the returned graph contains an edge from b to a. + * Construct a new graph with all the edges reversed. If this graph + * contains an edge from a to b then the returned graph contains an + * edge from b to a. * */ def reverse: Graph[VD, ED] /** - * This function takes a vertex and edge predicate and constructs the subgraph - * that consists of vertices and edges that satisfy the predict. The resulting - * graph contains the vertices and edges that satisfy: + * This function takes a vertex and edge predicate and constructs + * the subgraph that consists of vertices and edges that satisfy the + * predict. The resulting graph contains the vertices and edges + * that satisfy: * + * {{{ * V' = {v : for all v in V where vpred(v)} * E' = {(u,v): for all (u,v) in E where epred((u,v)) && vpred(u) && vpred(v)} + * }}} * - * @param epred the edge predicate which takes a triplet and evaluates to true - * if the edge is to remain in the subgraph. Note that only edges in which both - * vertices satisfy the vertex predicate are considered. + * @param epred the edge predicate which takes a triplet and + * evaluates to true if the edge is to remain in the subgraph. Note + * that only edges in which both vertices satisfy the vertex + * predicate are considered. * - * @param vpred the vertex predicate which takes a vertex object and evaluates - * to true if the vertex is to be included in the subgraph + * @param vpred the vertex predicate which takes a vertex object and + * evaluates to true if the vertex is to be included in the subgraph * - * @return the subgraph containing only the vertices and edges that satisfy the - * predicates. + * @return the subgraph containing only the vertices and edges that + * satisfy the predicates. */ def subgraph(epred: EdgeTriplet[VD,ED] => Boolean = (x => true), vpred: (Vid, VD) => Boolean = ((v,d) => true) ): Graph[VD, ED] @@ -187,16 +201,17 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { /** * groupEdgeTriplets is used to merge multiple edges that have the * same source and destination vertex into a single edge. The user - * supplied function is applied to each directed pair of vertices (u, v) and - * has access to all EdgeTriplets + * supplied function is applied to each directed pair of vertices + * (u, v) and has access to all EdgeTriplets * * {e: for all e in E where e.src = u and e.dst = v} * - * This function is identical to [[org.apache.spark.graph.Graph.groupEdges]] - * except that this function - * provides the user-supplied function with an iterator over EdgeTriplets, - * which contain the vertex data, whereas groupEdges provides the user-supplied - * function with an iterator over Edges, which only contain the vertex IDs. + * This function is identical to + * [[org.apache.spark.graph.Graph.groupEdges]] except that this + * function provides the user-supplied function with an iterator + * over EdgeTriplets, which contain the vertex data, whereas + * groupEdges provides the user-supplied function with an iterator + * over Edges, which only contain the vertex IDs. * * @tparam ED2 the type of the resulting edge data after grouping * @@ -211,35 +226,38 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { /** - * This function merges multiple edges between two vertices into a single - * Edge. See [[org.apache.spark.graph.Graph.groupEdgeTriplets]] for more detail. + * This function merges multiple edges between two vertices into a + * single Edge. See + * [[org.apache.spark.graph.Graph.groupEdgeTriplets]] for more + * detail. * * @tparam ED2 the type of the resulting edge data after grouping. * * @param f the user supplied function to merge multiple Edges * into a single ED2 object. * - * @return Graph[VD,ED2] The resulting graph with a single Edge for each - * source, dest vertex pair. + * @return Graph[VD,ED2] The resulting graph with a single Edge for + * each source, dest vertex pair. */ def groupEdges[ED2: ClassManifest](f: Iterator[Edge[ED]] => ED2 ): Graph[VD,ED2] /** - * The mapReduceTriplets function is used to compute statistics about - * 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. + * The mapReduceTriplets function is used to compute statistics + * about 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. * * @tparam A the type of "message" to be sent to each vertex * - * @param mapFunc the user defined map function which returns 0 or + * @param mapFunc the user defined map function which returns 0 or * 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 of - * the map phase. + * + * @param reduceFunc the user defined reduce function which should + * be commutative and assosciative and is used to combine the output + * of the map phase. * * @example We can use this function to compute the inDegree of each * vertex @@ -249,10 +267,11 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * mapReduceTriplets[Int](et => Array((et.dst.id, 1)), _ + _) * }}} * - * @note By expressing computation at the edge level we achieve maximum - * parallelism. This is one of the core functions in the Graph API in that enables - * neighborhood level computation. For example this function can be used to - * count neighbors satisfying a predicate or implement PageRank. + * @note By expressing computation at the edge level we achieve + * maximum parallelism. This is one of the core functions in the + * Graph API in that enables neighborhood level computation. For + * example this function can be used to count neighbors satisfying a + * predicate or implement PageRank. * */ def mapReduceTriplets[A: ClassManifest]( @@ -262,23 +281,25 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { /** - * Join the vertices with an RDD and then apply a function from the the - * vertex and RDD entry to a new vertex value and type. - * The input table should contain at most one entry for each vertex. - * If no entry is provided the map function is invoked passing none. + * Join the vertices with an RDD and then apply a function from the + * the vertex and RDD entry to a new vertex value and type. The + * input table should contain at most one entry for each vertex. If + * no entry is provided the map function is invoked passing 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 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. * - * @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 + * @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[(),()] = Graph.textFile("webgraph") * val outDeg: RDD[(Vid, Int)] = rawGraph.outDegrees() @@ -296,29 +317,50 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { // 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 Singleton contains basic routines to create graphs + * The Graph object contains a collection of routines used to + * construct graphs from RDDs. + * */ object Graph { import org.apache.spark.graph.impl._ import org.apache.spark.SparkContext._ + /** + * Construct a graph from a collection of edges encoded as vertex id + * pairs. Duplicate directed edges are merged to a single edge with + * weight equal to the number of duplicate edges. The returned + * vertex attribute is the number of edges adjacent to that vertex + * (i.e., the undirected degree). + * + * @param rawEdges the RDD containing the set of edges in the graph + * + * @return a graph with edge attributes containing the count of + * duplicate edges and vertex attributes containing the total degree + * of each vertex. + */ def apply(rawEdges: RDD[(Vid, Vid)]): Graph[Int, Int] = { Graph(rawEdges, true) } /** - * Construct a graph from a list of Edges. + * Construct a graph from a collection of edges encoded as vertex id + * pairs. * * @param rawEdges a collection of edges in (src,dst) form. - * @param uniqueEdges if multiple identical edges are found they are combined - * and the edge attribute is set to the sum. Otherwise duplicate edges are - * treated as separate. + * @param uniqueEdges if multiple identical edges are found they are + * combined and the edge attribute is set to the sum. Otherwise + * duplicate edges are treated as separate. + * + * @return a graph with edge attributes containing either the count + * of duplicate edges or 1 (if `uniqueEdges=false`) and vertex + * attributes containing the total degree of each vertex. * - * */ def apply(rawEdges: RDD[(Vid, Vid)], uniqueEdges: Boolean): Graph[Int, Int] = { // Reduce to unique edges. @@ -338,6 +380,20 @@ object Graph { } + /** + * Construct a graph from a collection attributed vertices and + * edges. + * + * @note Duplicate vertices are removed arbitrarily and missing + * vertices (vertices in the edge collection that are not in the + * vertex collection) are replaced by null vertex attributes. + * + * @tparam VD the vertex attribute type + * @tparam ED the edge attribute type + * @param vertices the "set" of vertices and their attributes + * @param edges the collection of edges in the graph + * + */ def apply[VD: ClassManifest, ED: ClassManifest]( vertices: RDD[(Vid,VD)], edges: RDD[Edge[ED]]): Graph[VD, ED] = { @@ -346,9 +402,21 @@ object Graph { } - /** - * Construct a new graph from a set of edges and vertices + * Construct a graph from a collection attributed vertices and + * edges. Duplicate vertices are combined using the `mergeFunc` and + * vertices found in the edge collection but not in the input + * vertices are the default attribute `defautVertexAttr`. + * + * @tparam VD the vertex attribute type + * @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 mergeFunc the function used to merge duplicate vertices + * in the `vertices` collection. + * */ def apply[VD: ClassManifest, ED: ClassManifest]( vertices: RDD[(Vid,VD)], @@ -358,5 +426,14 @@ object Graph { GraphImpl(vertices, edges, defaultVertexAttr, mergeFunc) } + /** + * The implicit graphToGraphOPs function 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. + */ implicit def graphToGraphOps[VD: ClassManifest, ED: ClassManifest](g: Graph[VD, ED]) = g.ops } // end of Graph object From 15958ca65a0ef52e9aac83301489fc30092552e2 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Tue, 29 Oct 2013 14:01:24 -0700 Subject: [PATCH 153/531] Reindenting documentation. --- .../org/apache/spark/graph/Analytics.scala | 76 ++++--- .../scala/org/apache/spark/graph/Edge.scala | 15 +- .../org/apache/spark/graph/EdgeTriplet.scala | 9 +- .../org/apache/spark/graph/GraphOps.scala | 101 +++++---- .../scala/org/apache/spark/graph/Pregel.scala | 209 +++++++++--------- 5 files changed, 220 insertions(+), 190 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala index acb9e3753f8e2..1b91141b17266 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala @@ -6,19 +6,20 @@ import org.apache.spark._ /** * The Analytics object contains a collection of basic graph analytics - * algorithms that operate largely on the graph structure. + * algorithms that operate largely on the graph structure. * - * In addition the Analytics object contains a driver `main` which can - * be used to apply the various functions to graphs in standard formats. + * In addition the Analytics object contains a driver `main` which can + * be used to apply the various functions to graphs in standard + * formats. */ object Analytics extends Logging { /** - * Run PageRank for a fixed number of iterations returning a graph - * with vertex attributes containing the PageRank and edge attributes - * the normalized edge weight. + * Run PageRank for a fixed number of iterations returning a graph + * with vertex attributes containing the PageRank and edge + * attributes the normalized edge weight. * - * The following PageRank fixed point is computed for each vertex. + * The following PageRank fixed point is computed for each vertex. * * {{{ * var PR = Array.fill(n)( 1.0 ) @@ -31,12 +32,13 @@ object Analytics extends Logging { * } * }}} * - * 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`. + * 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. + * 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) @@ -45,8 +47,8 @@ object Analytics extends Logging { * @param numIter the number of iterations of PageRank to run * @param resetProb the random reset probability (alpha) * - * @return the graph containing with each vertex containing the PageRank and - * each edge containing the normalized weight. + * @return the graph containing with each vertex containing the + * PageRank and each edge containing the normalized weight. * */ def pagerank[VD: Manifest, ED: Manifest]( @@ -54,8 +56,8 @@ object Analytics extends Logging { 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 @@ -85,10 +87,11 @@ object Analytics extends Logging { vertexProgram, sendMessage, messageCombiner) } + /** - * Run a dynamic version of PageRank returning a graph with vertex attributes - * containing the PageRank and edge attributes containing the normalized - * edge weight. + * 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 ) @@ -101,22 +104,24 @@ object Analytics extends Logging { * } * }}} * - * 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`. + * 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. + * 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) * * @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 containing the normalized weight. + * @return the graph containing with each vertex containing the + * PageRank and each edge containing the normalized weight. */ def deltaPagerank[VD: Manifest, ED: Manifest]( graph: Graph[VD, ED], tol: Double, resetProb: Double = 0.15): @@ -163,18 +168,19 @@ object Analytics extends Logging { /** - * Compute the connected component membership of each vertex - * and return an RDD with the vertex value containing the - * lowest vertex id in the connected component containing - * that vertex. + * Compute the connected component membership of each vertex and + * return an RDD with the vertex value containing the lowest vertex + * id in the connected component containing that vertex. * - * @tparam VD the vertex attribute type (discarded in the computation) + * @tparam VD the vertex attribute type (discarded in the + * computation) * @tparam ED the edge attribute type (preserved in the computation) * - * @param graph the graph for which to compute the connected components + * @param graph the graph for which to compute the connected + * components * - * @return a graph with vertex attributes containing the smallest vertex - * in each connected component + * @return a graph with vertex attributes containing the smallest + * vertex in each connected component */ def connectedComponents[VD: Manifest, ED: Manifest](graph: Graph[VD, ED]): Graph[Vid, ED] = { diff --git a/graph/src/main/scala/org/apache/spark/graph/Edge.scala b/graph/src/main/scala/org/apache/spark/graph/Edge.scala index 67b64540177fc..509a734338c32 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Edge.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Edge.scala @@ -8,8 +8,17 @@ package org.apache.spark.graph * @tparam ED type of the edge attribute */ case class Edge[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED] ( + /** + * The vertex id of the source vertex + */ var srcId: Vid = 0, + /** + * The vertex id of the target vertex. + */ var dstId: Vid = 0, + /** + * The attribute associated with the edge. + */ var attr: ED = nullValue[ED]) { /** @@ -23,10 +32,12 @@ case class Edge[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED] /** - * Return the relative direction of the edge to the corresponding vertex. + * Return the relative direction of the edge to the corresponding + * vertex. * * @param vid the id of one of the two vertices in the edge. - * @return the relative direction of the edge to the corresponding vertex. + * @return the relative direction of the edge to the corresponding + * vertex. */ def relativeDirection(vid: Vid): EdgeDirection = if (vid == srcId) EdgeDirection.Out else { assert(vid == dstId); EdgeDirection.In } diff --git a/graph/src/main/scala/org/apache/spark/graph/EdgeTriplet.scala b/graph/src/main/scala/org/apache/spark/graph/EdgeTriplet.scala index ef3aa199bdf41..aace6e54fe314 100644 --- a/graph/src/main/scala/org/apache/spark/graph/EdgeTriplet.scala +++ b/graph/src/main/scala/org/apache/spark/graph/EdgeTriplet.scala @@ -1,14 +1,15 @@ package org.apache.spark.graph /** - * An edge triplet represents two vertices and edge along with their attributes. + * An edge triplet represents two vertices and edge along with their + * attributes. * * @tparam VD the type of the vertex attribute. * @tparam ED the type of the edge attribute * - * @todo specialize edge triplet for basic types, though when I last tried - * specializing I got a warning about inherenting from a type that is not - * a trait. + * @todo specialize edge triplet for basic types, though when I last + * tried specializing I got a warning about inherenting from a type + * that is not a trait. */ class EdgeTriplet[VD, ED] extends Edge[ED] { // class EdgeTriplet[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) VD: ClassManifest, diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala b/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala index d05711c9eb469..8480ff29d34d0 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala @@ -7,9 +7,10 @@ import org.apache.spark.util.ClosureCleaner /** - * `GraphOps` contains additional functionality (syntatic sugar) for the graph - * type and is implicitly constructed for each Graph object. All operations in - * `GraphOps` are expressed in terms of the efficient GraphX API. + * `GraphOps` contains additional functionality (syntatic sugar) for + * the graph type and is implicitly constructed for each Graph object. + * All operations in `GraphOps` are expressed in terms of the + * efficient GraphX API. * * @tparam VD the vertex attribute type * @tparam ED the edge attribute type @@ -30,7 +31,8 @@ class GraphOps[VD: ClassManifest, ED: ClassManifest](graph: Graph[VD, ED]) { /** - * Compute the in-degree of each vertex in the Graph returning an RDD. + * Compute the in-degree of each vertex in the Graph returning an + * RDD. * @note Vertices with no in edges are not returned in the resulting RDD. */ lazy val inDegrees: VertexSetRDD[Int] = degreesRDD(EdgeDirection.In) @@ -44,8 +46,9 @@ class GraphOps[VD: ClassManifest, ED: ClassManifest](graph: Graph[VD, ED]) { /** - * Compute the degrees of each vertex in the Graph returning an RDD. - * @note Vertices with no edges are not returned in the resulting RDD. + * Compute the degrees of each vertex in the Graph returning an RDD. + * @note Vertices with no edges are not returned in the resulting + * RDD. */ lazy val degrees: VertexSetRDD[Int] = degreesRDD(EdgeDirection.Both) @@ -53,8 +56,8 @@ class GraphOps[VD: ClassManifest, ED: ClassManifest](graph: Graph[VD, ED]) { /** * Compute the neighboring vertex degrees. * - * @param edgeDirection the direction along which to collect neighboring - * vertex attributes. + * @param edgeDirection the direction along which to collect + * neighboring vertex attributes. */ private def degreesRDD(edgeDirection: EdgeDirection): VertexSetRDD[Int] = { graph.aggregateNeighbors((vid, edge) => Some(1), _+_, edgeDirection) @@ -62,29 +65,31 @@ class GraphOps[VD: ClassManifest, ED: ClassManifest](graph: Graph[VD, ED]) { /** - * This function is used to compute a statistic for the neighborhood of each - * vertex and returns a value for all vertices (including those without - * neighbors). - * - * @note Because the a default value is provided all vertices will have a - * corresponding entry in the returned RDD. - * - * @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 default the default value to use for each vertex if it has no - * neighbors or the map function repeatedly evaluates to none - * @param direction the direction of edges to consider (e.g., In, Out, Both). + * This function is used to compute a statistic for the neighborhood + * of each vertex and returns a value for all vertices (including + * those without neighbors). + * + * @note Because the a default value is provided all vertices will + * have a corresponding entry in the returned RDD. + * + * @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 default the default value to use for each vertex if it has + * no neighbors or the map function repeatedly evaluates to none + * @param direction the direction of edges to consider (e.g., In, + * Out, Both). * @tparam VD2 The returned type of the aggregation operation. * * @return A Spark.RDD containing tuples of vertex identifiers and - * their resulting value. There will be exactly one entry for ever vertex in - * the original graph. + * their resulting value. There will be exactly one entry for ever + * vertex in the original graph. * - * @example We can use this function to compute the average follower age - * for each user + * @example We can use this function to compute the average follower + * age for each user + * * {{{ * val graph: Graph[Int,Int] = loadGraph() * val averageFollowerAge: RDD[(Int, Int)] = @@ -138,7 +143,7 @@ class GraphOps[VD: ClassManifest, ED: ClassManifest](graph: Graph[VD, ED]) { /** * Return the Ids of the neighboring vertices. * - * @param edgeDirection the direction along which to collect + * @param edgeDirection the direction along which to collect * neighboring vertices * * @return the vertex set of neighboring ids for each vertex. @@ -164,11 +169,11 @@ class GraphOps[VD: ClassManifest, ED: ClassManifest](graph: Graph[VD, ED]) { * graphs where high degree vertices may force a large ammount of * information to be collected to a single location. * - * @param edgeDirection the direction along which to collect + * @param edgeDirection the direction along which to collect * neighboring vertices * - * @return the vertex set of neighboring vertex attributes - * for each vertex. + * @return the vertex set of neighboring vertex attributes for each + * vertex. */ def collectNeighbors(edgeDirection: EdgeDirection) : VertexSetRDD[ Array[(Vid, VD)] ] = { @@ -186,24 +191,26 @@ class GraphOps[VD: ClassManifest, ED: ClassManifest](graph: Graph[VD, ED]) { /** - * 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 should - * contain at most one entry for each vertex. If no entry is provided the - * map function is skipped and the old value is used. + * 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 + * should contain at most one entry for each vertex. If no entry is + * provided the map function is skipped and the old value is used. * * @tparam U the type of entry in the table of updates - * @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 only for vertices with a 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 + * @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 only for vertices with a + * 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") * .mapVertices(v => 0) diff --git a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala index d45e351d6af5d..47ce2ddb33076 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala @@ -5,18 +5,17 @@ import org.apache.spark.rdd.RDD /** * This object implements a Pregel-like bulk-synchronous - * message-passing API. However, unlike the original Pregel API - * the GraphX pregel API factors the sendMessage computation over - * edges, enables the message sending computation to read both - * vertex attributes, and finally contrains messages to the graph - * structure. These changes allow for substantially more efficient - * distributed execution while also exposing greater flexibility - * for graph based computation. + * message-passing API. However, unlike the original Pregel API the + * GraphX pregel API factors the sendMessage computation over edges, + * enables the message sending computation to read both vertex + * attributes, and finally contrains messages to the graph structure. + * These changes allow for substantially more efficient distributed + * execution while also exposing greater flexibility for graph based + * computation. * * This object present several variants of the bulk synchronous * execution that differ only in the edge direction along which - * messages are sent and whether a fixed number of iterations - * is used. + * messages are sent and whether a fixed number of iterations is used. * * @example We can use the Pregel abstraction to implement PageRank * {{{ @@ -46,18 +45,18 @@ object Pregel { /** - * 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 + * 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 + * 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. + * 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. + * on subsequent iterations if a vertex does not receive a message + * then the vertex-program is not invoked. * * This function iterates a fixed number (`numIter`) of iterations. * @@ -67,24 +66,26 @@ object Pregel { * * @param graph the input graph. * - * @param initialMsg the message each vertex will receive at the - * on the first iteration. + * @param initialMsg the message each vertex will receive at the on + * the first iteration. * * @param numIter the number of iterations to run this computation. * - * @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 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 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.'' + * @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 * @@ -100,18 +101,18 @@ object Pregel { /** - * 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 + * 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 + * 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. + * 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. + * on subsequent iterations if a vertex does not receive a message + * then the vertex-program is not invoked. * * This function iterates a fixed number (`numIter`) of iterations. * @@ -121,28 +122,29 @@ object Pregel { * * @param graph the input graph. * - * @param initialMsg the message each vertex will receive at the - * on the first iteration. + * @param initialMsg the message each vertex will receive at the on + * the first iteration. * * @param numIter the number of iterations to run this computation. * - * @param sendDir the edge direction along which the `sendMsg` function - * is invoked. + * @param sendDir the edge direction along which the `sendMsg` + * function is invoked. * - * @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 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 each edge - * in the direction `sendDir` adjacent to vertices that received messages - * in the current iteration. + * @param sendMsg a user supplied function that is applied to each + * edge in the direction `sendDir` adjacent to 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.'' + * @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 * @@ -174,18 +176,18 @@ object Pregel { /** - * 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 + * 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 + * 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. + * 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. + * 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. * @@ -195,24 +197,26 @@ object Pregel { * * @param graph the input graph. * - * @param initialMsg the message each vertex will receive at the - * on the first iteration. + * @param initialMsg the message each vertex will receive at the on + * the first iteration. * * @param numIter the number of iterations to run this computation. * - * @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 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 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.'' + * @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 * @@ -228,18 +232,18 @@ object Pregel { /** - * 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 + * 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 + * 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. + * 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. + * 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. * @@ -249,28 +253,29 @@ object Pregel { * * @param graph the input graph. * - * @param initialMsg the message each vertex will receive at the - * on the first iteration. + * @param initialMsg the message each vertex will receive at the on + * the first iteration. * * @param numIter the number of iterations to run this computation. * - * @param sendDir the edge direction along which the `sendMsg` function - * is invoked. - * - * @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 each edge - * in the direction `sendDir` adjacent to 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.'' + * @param sendDir the edge direction along which the `sendMsg` + * function is invoked. + * + * @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 each + * edge in the direction `sendDir` adjacent to 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 * From ede329336d2d072a91898a75c8ae2bdc7d0f671d Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Tue, 29 Oct 2013 14:50:12 -0700 Subject: [PATCH 154/531] Fixing a scaladoc bug in graph generators. --- .../scala/org/apache/spark/graph/util/GraphGenerators.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala b/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala index 1bbcce50399ab..2c652cb172b4a 100644 --- a/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala +++ b/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala @@ -17,7 +17,7 @@ import org.apache.spark.graph.Edge import org.apache.spark.graph.impl.GraphImpl /** - * @todo(crankshaw) cleanup and modularize code + * @todo cleanup and modularize code */ object GraphGenerators { From 08c7b040d6653baf6654682b8661619134c5fc96 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Tue, 29 Oct 2013 15:03:13 -0700 Subject: [PATCH 155/531] Documented the VertexSetRDD --- .../org/apache/spark/graph/VertexSetRDD.scala | 176 +++++++++++------- 1 file changed, 108 insertions(+), 68 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala b/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala index cb75da6c211cb..c7f20283da3d6 100644 --- a/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala @@ -41,9 +41,9 @@ import org.apache.spark.storage.StorageLevel /** - * The `VertexSetIndex` maintains the per-partition mapping from vertex id - * to the corresponding location in the per-partition values array. - * This class is meant to be an opaque type. + * The `VertexSetIndex` maintains the per-partition mapping from + * vertex id to the corresponding location in the per-partition values + * array. This class is meant to be an opaque type. * */ class VertexSetIndex(private[spark] val rdd: RDD[VertexIdToIndexMap]) { @@ -56,8 +56,8 @@ class VertexSetIndex(private[spark] val rdd: RDD[VertexIdToIndexMap]) { } /** - * Returns the partitioner object of the underlying RDD. This is used - * by the VertexSetRDD to partition the values RDD. + * Returns the partitioner object of the underlying RDD. This is + * used by the VertexSetRDD to partition the values RDD. */ def partitioner: Partitioner = rdd.partitioner.get } // end of VertexSetIndex @@ -65,19 +65,21 @@ class VertexSetIndex(private[spark] val rdd: RDD[VertexIdToIndexMap]) { /** - * An VertexSetRDD[V] extends the RDD[(Vid,V)] by ensuring that there is only - * one entry for each vertex and by pre-indexing the entries for fast, efficient - * joins. + * An VertexSetRDD[V] extends the RDD[(Vid,V)] by ensuring that there + * is only one entry for each vertex and by pre-indexing the entries + * for fast, efficient joins. * - * In addition to providing the basic RDD[(Vid,V)] functionality the VertexSetRDD - * exposes an index member which can be used to "key" other VertexSetRDDs + * In addition to providing the basic RDD[(Vid,V)] functionality the + * VertexSetRDD exposes an index member which can be used to "key" + * other VertexSetRDDs * - * @tparam V the vertex attribute associated with each vertex in the set. + * @tparam V the vertex attribute associated with each vertex in the + * set. * - * @param index the index which contains the vertex id information and is used - * to organize the values in the RDD. - * @param valuesRDD the values RDD contains the actual vertex attributes organized - * as an array within each partition. + * @param index the index which contains the vertex id information and + * is used to organize the values in the RDD. + * @param valuesRDD the values RDD contains the actual vertex + * attributes organized as an array within each partition. * * To construct a `VertexSetRDD` use the singleton object: * @@ -175,10 +177,18 @@ class VertexSetRDD[@specialized V: ClassManifest]( /** - * @todo finish documenting + * Restrict the vertex set to the set of vertices satisfying the + * given predicate. + * + * @param pred the user defined predicate + * + * @note The vertex set preserves the original index structure + * which means that the returned RDD can be easily joined with + * the original vertex-set. Furthermore, the filter only + * modifies the bitmap index and so no new values are allocated. */ - override def filter(f: Tuple2[Vid,V] => Boolean): VertexSetRDD[V] = { - val cleanF = index.rdd.context.clean(f) + override def filter(pred: Tuple2[Vid,V] => Boolean): VertexSetRDD[V] = { + val cleanF = index.rdd.context.clean(pred) val newValues = index.rdd.zipPartitions(valuesRDD){ (keysIter, valuesIter) => val index = keysIter.next() @@ -198,15 +208,15 @@ class VertexSetRDD[@specialized V: ClassManifest]( /** - * Pass each vertex attribute through a map function and retain - * the original RDD's partitioning and index. + * Pass each vertex attribute through a map function and retain the + * original RDD's partitioning and index. * * @tparam U the type returned by the map function * * @param f the function applied to each value in the RDD - * @return a new VertexSet with values obtaind by applying `f` to each of the - * entries in the original VertexSet. The resulting VertexSetRDD retains the - * same index. + * @return a new VertexSet with values obtaind by applying `f` to + * each of the entries in the original VertexSet. The resulting + * VertexSetRDD retains the same index. */ def mapValues[U: ClassManifest](f: V => U): VertexSetRDD[U] = { val cleanF = index.rdd.context.clean(f) @@ -214,9 +224,10 @@ class VertexSetRDD[@specialized V: ClassManifest]( valuesRDD.mapPartitions(iter => iter.map{ case (values, bs) => /** - * @todo Consider using a view rather than creating a new array. - * This is already being done for join operations. It could reduce - * memory overhead but require additional recomputation. + * @todo Consider using a view rather than creating a new + * array. This is already being done for join operations. + * It could reduce memory overhead but require additional + * recomputation. */ val newValues = new Array[U](values.size) for ( ind <- bs ) { @@ -229,16 +240,16 @@ class VertexSetRDD[@specialized V: ClassManifest]( /** - * Pass each vertex attribute along with the vertex id through a - * map function and retain the original RDD's partitioning and index. + * Pass each vertex attribute along with the vertex id through a map + * function and retain the original RDD's partitioning and index. * * @tparam U the type returned by the map function * - * @param f the function applied to each vertex id and vertex + * @param f the function applied to each vertex id and vertex * attribute in the RDD - * @return a new VertexSet with values obtaind by applying `f` to each of the - * entries in the original VertexSet. The resulting VertexSetRDD retains the - * same index. + * @return a new VertexSet with values obtaind by applying `f` to + * each of the entries in the original VertexSet. The resulting + * VertexSetRDD retains the same index. */ def mapValuesWithKeys[U: ClassManifest](f: (Vid, V) => U): VertexSetRDD[U] = { val cleanF = index.rdd.context.clean(f) @@ -267,16 +278,16 @@ class VertexSetRDD[@specialized V: ClassManifest]( /** - * Inner join this VertexSet with another VertexSet which has the same Index. - * This function will fail if both VertexSets do not share the same index. - * The resulting vertex set will only contain vertices that are in both this - * and the other vertex set. + * Inner join this VertexSet with another VertexSet which has the + * same Index. This function will fail if both VertexSets do not + * share the same index. The resulting vertex set will only contain + * vertices that are in both this and the other vertex set. * * @tparam W the attribute type of the other VertexSet * - * @param other the other VertexSet with which to join. - * @return a VertexSetRDD containing only the vertices in both this and the - * other VertexSet and with tuple attributes. + * @param other the other VertexSet with which to join. + * @return a VertexSetRDD containing only the vertices in both this + * and the other VertexSet and with tuple attributes. * */ def zipJoin[W: ClassManifest](other: VertexSetRDD[W]): VertexSetRDD[(V,W)] = { @@ -299,17 +310,18 @@ class VertexSetRDD[@specialized V: ClassManifest]( /** - * Left join this VertexSet with another VertexSet which has the same Index. - * This function will fail if both VertexSets do not share the same index. - * The resulting vertex set contains an entry for each vertex in this set. - * If the other VertexSet is missing any vertex in this VertexSet then a - * `None` attribute is generated + * Left join this VertexSet with another VertexSet which has the + * same Index. This function will fail if both VertexSets do not + * share the same index. The resulting vertex set contains an entry + * for each vertex in this set. If the other VertexSet is missing + * any vertex in this VertexSet then a `None` attribute is generated * * @tparam W the attribute type of the other VertexSet * - * @param other the other VertexSet with which to join. - * @return a VertexSetRDD containing all the vertices in this VertexSet - * with `None` attributes used for Vertices missing in the other VertexSet. + * @param other the other VertexSet with which to join. + * @return a VertexSetRDD containing all the vertices in this + * VertexSet with `None` attributes used for Vertices missing in the + * other VertexSet. * */ def leftZipJoin[W: ClassManifest](other: VertexSetRDD[W]): VertexSetRDD[(V,Option[W])] = { @@ -332,19 +344,21 @@ class VertexSetRDD[@specialized V: ClassManifest]( /** - * Left join this VertexSet with an RDD containing vertex attribute pairs. - * If the other RDD is backed by a VertexSet with the same index than the - * efficient leftZipJoin implementation is used. - * The resulting vertex set contains an entry for each vertex in this set. - * If the other VertexSet is missing any vertex in this VertexSet then a - * `None` attribute is generated + * Left join this VertexSet with an RDD containing vertex attribute + * pairs. If the other RDD is backed by a VertexSet with the same + * index than the efficient leftZipJoin implementation is used. The + * resulting vertex set contains an entry for each vertex in this + * set. If the other VertexSet is missing any vertex in this + * VertexSet then a `None` attribute is generated * * @tparam W the attribute type of the other VertexSet * * @param other the other VertexSet with which to join. - * @param merge the function used combine duplicate vertex attributes - * @return a VertexSetRDD containing all the vertices in this VertexSet - * with `None` attributes used for Vertices missing in the other VertexSet. + * @param merge the function used combine duplicate vertex + * attributes + * @return a VertexSetRDD containing all the vertices in this + * VertexSet with `None` attributes used for Vertices missing in the + * other VertexSet. * */ def leftJoin[W: ClassManifest]( @@ -581,8 +595,8 @@ class VertexSetRDD[@specialized V: ClassManifest]( object VertexSetRDD { /** - * Construct a vertex set from an RDD of vertex-attribute pairs. - * Duplicate entries are removed arbitrarily. + * Construct a vertex set from an RDD of vertex-attribute pairs. + * Duplicate entries are removed arbitrarily. * * @tparam V the vertex attribute type * @@ -592,14 +606,14 @@ object VertexSetRDD { apply(rdd, (a:V, b:V) => a ) /** - * Construct a vertex set from an RDD of vertex-attribute pairs where - * duplicate entries are merged using the reduceFunc + * Construct a vertex set from an RDD of vertex-attribute pairs + * where duplicate entries are merged using the reduceFunc * * @tparam V the vertex attribute type * * @param rdd the collection of vertex-attribute pairs - * @param reduceFunc the function used to merge attributes of duplicate - * vertices. + * @param reduceFunc the function used to merge attributes of + * duplicate vertices. */ def apply[V: ClassManifest]( rdd: RDD[(Vid,V)], reduceFunc: (V, V) => V): VertexSetRDD[V] = { @@ -635,7 +649,14 @@ object VertexSetRDD { /** - * @todo finish documenting + * Construct a vertex set from an RDD using an existing index. + * + * @note duplicate vertices are discarded arbitrarily + * + * @tparam the vertex attribute type + * @param rdd the rdd containing vertices + * @param index the index which must be a superset of the vertices + * in RDD */ def apply[V: ClassManifest]( rdd: RDD[(Vid,V)], index: VertexSetIndex): VertexSetRDD[V] = @@ -643,7 +664,15 @@ object VertexSetRDD { /** - * @todo finish documenting + * Construct a vertex set from an RDD using an existing index and a + * user defined `combiner` to merge duplicate vertices. + * + * @tparam the vertex attribute type + * @param rdd the rdd containing vertices + * @param index the index which must be a superset of the vertices + * in RDD + * @param reduceFunc the user defined reduce function used to merge + * duplicate vertex attributes. */ def apply[V: ClassManifest]( rdd: RDD[(Vid,V)], index: VertexSetIndex, @@ -652,7 +681,19 @@ object VertexSetRDD { /** - * @todo finish documenting + * Construct a vertex set from an RDD using an existing index and a + * user defined `combiner` to merge duplicate vertices. + * + * @tparam the vertex attribute type + * @param rdd the rdd containing vertices + * @param index the index which must be a superset of the vertices + * in RDD + * @param createCombiner a user defined function to create a combiner + * from a vertex attribute + * @param mergeValue a user defined function to merge a vertex + * attribute into an existing combiner + * @param mergeCombiners a user defined function to merge combiners + * */ def apply[V: ClassManifest, C: ClassManifest]( rdd: RDD[(Vid,V)], @@ -703,9 +744,8 @@ object VertexSetRDD { /** - * Construct and index of the unique values in a given RDD. - * - * @todo finish documenting + * Construct and index of the unique vertex ids. This can be used + * as an index when building a vertex set. */ def makeIndex(keys: RDD[Vid], partitioner: Option[Partitioner] = None): VertexSetIndex = { From 38ec0baf5c9033a9e9e9bb015d95357d8176e022 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Tue, 29 Oct 2013 16:27:55 -0700 Subject: [PATCH 156/531] fixing a typo in the VertexSetRDD docs --- .../src/main/scala/org/apache/spark/graph/VertexSetRDD.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala b/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala index c7f20283da3d6..a64eb6a99cf8f 100644 --- a/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala @@ -744,8 +744,9 @@ object VertexSetRDD { /** - * Construct and index of the unique vertex ids. This can be used - * as an index when building a vertex set. + * Construct an index of the unique vertices. The resulting index + * can be used to build VertexSets over subsets of the vertices in + * the input. */ def makeIndex(keys: RDD[Vid], partitioner: Option[Partitioner] = None): VertexSetIndex = { From 41b312212094b2accd650813dd45e1767b5465fe Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Tue, 29 Oct 2013 20:57:55 -0700 Subject: [PATCH 157/531] Strating to improve README. --- README.md | 128 ++++++++++++------- docs/img/data_parallel_vs_graph_parallel.png | Bin 0 -> 199060 bytes docs/img/edge-cut.png | Bin 0 -> 12563 bytes docs/img/graph_parallel.png | Bin 0 -> 92288 bytes docs/img/tables_and_graphs.png | Bin 0 -> 68905 bytes docs/img/vertex-cut.png | Bin 0 -> 12246 bytes 6 files changed, 81 insertions(+), 47 deletions(-) create mode 100644 docs/img/data_parallel_vs_graph_parallel.png create mode 100644 docs/img/edge-cut.png create mode 100644 docs/img/graph_parallel.png create mode 100644 docs/img/tables_and_graphs.png create mode 100644 docs/img/vertex-cut.png diff --git a/README.md b/README.md index 139bdc070c5b7..a54d4ed58780b 100644 --- a/README.md +++ b/README.md @@ -1,35 +1,76 @@ -# GraphX Branch of Spark +# GraphX: Unifying Graph and Tables -This is experimental code for the Apache spark project. -# Apache Spark +GraphX extends the distributed fault-tolerant collections API and +interactive console of [Spark](http://spark.incubator.apache.org) 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. + + +## Motivation + +From social networks and targeted advertising to protein modeling and +astrophysics, big graphs capture the structure in data and are central +to the recent advances in machine learning and data mining. Directly +applying existing *data-parallel* tools (e.g., +[Hadoop](http://hadoop.apache.org) and +[Spark](http://spark.incubator.apache.org)) to graph computation tasks +can be cumbersome and inefficient. The need for intuitive, scalable +tools for graph computation has lead to the development of new +*graph-parallel* systems (e.g., +[Pregel](http://http://giraph.apache.org) and +[GraphLab](http://graphlab.org)) which are designed to efficiently +execute graph algorithms. Unfortunately, these systems do not address +the challenges of graph construction and transformation and provide +limited fault-tolerance and support for interactive analysis. + +![image](http://docs/data_parllel_vs_graph_parallel.png) + + +## Solution + +The GraphX project combines the advantages of both data-parallel and +graph-parallel systems by efficiently expressing graph computation +within the [Spark](http://spark.incubator.apache.org) framework. We +leverage new ideas in distributed graph representation to efficiently +distribute graphs as tabular data-structures. Similarly, we leverage +advances in data-flow systems to exploit in-memory computation and +fault-tolerance. We provide powerful new operations to simplify graph +construction and transformation. Using these primitives we implement +the PowerGraph and Pregel abstractions in less than 20 lines of code. +Finally, by exploiting the Scala foundation of Spark, we enable users +to interactively load, transform, and compute on massive graphs. -Lightning-Fast Cluster Computing - ## Online Documentation You can find the latest Spark documentation, including a programming -guide, on the project webpage at . -This README file only contains basic setup instructions. +guide, on the project webpage at +. This README +file only contains basic setup instructions. ## Building -Spark requires Scala 2.9.3 (Scala 2.10 is not yet supported). The project is -built using Simple Build Tool (SBT), which is packaged with it. To build -Spark and its example programs, run: +Spark requires Scala 2.9.3 (Scala 2.10 is not yet supported). The +project is built using Simple Build Tool (SBT), which is packaged with +it. To build Spark and its example programs, run: sbt/sbt assembly -Once you've built Spark, the easiest way to start using it is the shell: +Once you've built Spark, the easiest way to start using it is the +shell: ./spark-shell Or, for the Python API, the Python shell (`./pyspark`). -Spark also comes with several sample programs in the `examples` directory. -To run one of them, use `./run-example `. For example: +Spark also comes with several sample programs in the `examples` +directory. To run one of them, use `./run-example +`. For example: ./run-example org.apache.spark.examples.SparkLR local[2] @@ -37,18 +78,19 @@ will run the Logistic Regression example locally on 2 CPUs. Each of the example programs prints usage help if no params are given. -All of the Spark samples take a `` parameter that is the cluster URL -to connect to. This can be a mesos:// or spark:// URL, or "local" to run -locally with one thread, or "local[N]" to run locally with N threads. +All of the Spark samples take a `` parameter that is the +cluster URL to connect to. This can be a mesos:// or spark:// URL, or +"local" to run locally with one thread, or "local[N]" to run locally +with N threads. ## A Note About Hadoop Versions -Spark uses the Hadoop core library to talk to HDFS and other Hadoop-supported -storage systems. Because the protocols have changed in different versions of -Hadoop, you must build Spark against the same version that your cluster runs. -You can change the version by setting the `SPARK_HADOOP_VERSION` environment -when building Spark. +Spark uses the Hadoop core library to talk to HDFS and other +Hadoop-supported storage systems. Because the protocols have changed +in different versions of Hadoop, you must build Spark against the same +version that your cluster runs. You can change the version by setting +the `SPARK_HADOOP_VERSION` environment when building Spark. For Apache Hadoop versions 1.x, Cloudera CDH MRv1, and other Hadoop versions without YARN, use: @@ -68,17 +110,18 @@ with YARN, also set `SPARK_YARN=true`: # Cloudera CDH 4.2.0 with MapReduce v2 $ SPARK_HADOOP_VERSION=2.0.0-cdh4.2.0 SPARK_YARN=true sbt/sbt assembly -For convenience, these variables may also be set through the `conf/spark-env.sh` file -described below. +For convenience, these variables may also be set through the +`conf/spark-env.sh` file described below. -When developing a Spark application, specify the Hadoop version by adding the -"hadoop-client" artifact to your project's dependencies. For example, if you're -using Hadoop 1.0.1 and build your application using SBT, add this entry to -`libraryDependencies`: +When developing a Spark application, specify the Hadoop version by +adding the "hadoop-client" artifact to your project's +dependencies. For example, if you're using Hadoop 1.0.1 and build your +application using SBT, add this entry to `libraryDependencies`: "org.apache.hadoop" % "hadoop-client" % "1.2.1" -If your project is built with Maven, add this to your POM file's `` section: +If your project is built with Maven, add this to your POM file's +`` section: org.apache.hadoop @@ -89,28 +132,19 @@ If your project is built with Maven, add this to your POM file's ` ## Configuration -Please refer to the [Configuration guide](http://spark.incubator.apache.org/docs/latest/configuration.html) +Please refer to the [Configuration +guide](http://spark.incubator.apache.org/docs/latest/configuration.html) in the online documentation for an overview on how to configure Spark. -## Apache Incubator Notice - -Apache Spark is an effort undergoing incubation at The Apache Software -Foundation (ASF), sponsored by the Apache Incubator. Incubation is required of -all newly accepted projects until a further review indicates that the -infrastructure, communications, and decision making process have stabilized in -a manner consistent with other successful ASF projects. While incubation status -is not necessarily a reflection of the completeness or stability of the code, -it does indicate that the project has yet to be fully endorsed by the ASF. - - -## Contributing to Spark +## Contributing to GraphX -Contributions via GitHub pull requests are gladly accepted from their original -author. Along with any pull requests, please state that the contribution is -your original work and that you license the work to the project under the -project's open source license. Whether or not you state this explicitly, by -submitting any copyrighted material via pull request, email, or other means -you agree to license the material under the project's open source license and -warrant that you have the legal authority to do so. +Contributions via GitHub pull requests are gladly accepted from their +original author. Along with any pull requests, please state that the +contribution is your original work and that you license the work to +the project under the project's open source license. Whether or not +you state this explicitly, by submitting any copyrighted material via +pull request, email, or other means you agree to license the material +under the project's open source license and warrant that you have the +legal authority to do so. diff --git a/docs/img/data_parallel_vs_graph_parallel.png b/docs/img/data_parallel_vs_graph_parallel.png new file mode 100644 index 0000000000000000000000000000000000000000..d9aa811466327dd6960d3b2552112b5259c02a42 GIT binary patch literal 199060 zcmcG$Rajh2w=Imj1c%^(KyU~U+$}f+cL)~Ty>Yjo3GTt&-QC^Y9U5<>o1ZuD+5g_( zzBqU1qM!Bj>Q%F5jafBU%`w*Ma3uvPbd>ifP*70l(qF~DK|#S>KtVwdAtAh-!Gf{_ zynSJ@5EE09789dTa5i+t6@moPiF-(d}DUSkyPe&aSTR zu5pSn!!g5*WL@1Y!gpJ}P;efP9>o;z@c^v@T}ar?WvHJ~)qYaMFf?Zu!4;igz9Y{6 zgHOxO%K9E^d3paMys*Onk8s1+Z(owzq5GIokR1^s$f0WwxAq-= zvTg4rH{d0=FMbX;TRZGoxP=go5=Dfv!03mCMGi84{s-s?#4v{}*zWDoPp$}*AyuE6 z>x$4uj7>h98B+8~C9_Ed_;>r#!NQJ`2PI3h!1ltBA|ODDk=QuJ5D=if5kkD3pn5Bn z?9M)x)x9BzZ2wiu2?`1|{GSgr*YIdL6qGQOw77_>JM@`1s-ar{^eY@Tw*EHwIyg+N z3%d-&Xh?jvxRV#6K8+tJN}g>aBre48oq>MkTa_>BoVu`%lK(x9O?0`i4zy4Z%Dg7J z3_uwAqu)CiYGD}U^w+Vo37*4DFFRW@u3`BL#=Wu3_2uQ1&5M+b+J`kK6%x5@X)Gl; zTxbd5|2|;9_g6ic{Q%Tncv&#al=?uiT`QkpQ~Wq zGGPDHLWD58urTa{8SB@Q-2Z9$pR4>^C;r2aHw)QB6xse>?1a>6%Kz=(-~Ok%^bY^S zw>Jw@Xz!-F+KJHOON##6=fC~`*s5^)A0zy0;RgcJkF5tn5g}NP6R_G z6%+VOo`T3y=1JOcE=Z`=@H>P`=?;2LO{Fa~D#*c20EVTlgJ`7ZV|`P6%{b~AKMA6p zMo=0-sbHG~Smu3e=aWTgc>!0b^_yzoydw$Xh(qwqCP>lV~vNC`B z`{hEh#j%Fi03XvCaP4+5r(G|krsPl1d0>A+RAtI7jJQ2#bZB@5qH=Y`#H+dnftsNJ z{hN8uqEsb$PJ6qe?N35zIw+QOZP`!@2TcAcvzI|^8V1ORnHzWnfex&f!Sd2rh6~u` zO83{sRqX}t1ZBY=oeFxB8EPrmO{vFp>WSiwQw#^-D*lk)woIYb;Dk%IR2p<T zEq^1T>VH@C;|y|=V=2lxFJ=0EccT~=#;dcMS3bBSP^x>}b%LMP}MUO$zG26cdG zX%CM>)b=4Yg%w!B#(FpZ${}j!aFl!NF4v~QcOP7h*#;LZTi8U(+`NxiW#;S1&$0Wy zoailU8Nz*GB1F_`9J`ttN`Z!drr$o?l@g74Z<}oHe2hvps1TBr?)XW!5tr33G-3)9 z!vC0RMFQt6@Pj11K1ovSEu%>Ww%C&1k+YF4B;xm=8Qga4-H#gGpFW?`R5g^-f9aho zEpRIJA!54hErT{L>;-7+@stNQLK4snH1NEBz2y`r%Bt79W&6kTYY>U>MAR>(rx{q* zCMi8~(2yTA)xdWs1LTCzlilcX)8EIQQ@WD$&3AKM?L1MwgV4aP)DOw02+2??l`nLF zsZVLWznESktM#_hUfFvy;9uA3*2k?R-x6p%6!cnp)ALAiMb2`qdao>I)U^QBx%xb1 z0t%5!H28EX?MpEWObgi@fr!l}@SpSuBpfUboi)66B7V%%pt7fYu^v9)9midX<3Z*w=|%zyLR;% zbUtEHF(6CBKaJFS(J^Q|h z(th`5s4lzPI;Y?$)P3CAP?6Y$U~D86S>>_Z7+yx;uZuN|RaY|@)yK?f>G7Zr zZ!cl{5Azk91*Ac_n49)^NAv9&X-RVJ*2SW7jrbyJtG3YTC5Ef3l|84Ys9P#8`SB3g z3O^R}7{4mYYMD7%L^f*MIv5vUoG*CR{yW~GIXfT+?#vBj8xRZ{u><_=Id(d_!>J}b z9gKs7;o*spN2T|e9?Ir>lG|{H^9h{aP(C>&)7iN%s;{}@oRD%cNfW)E7VW%Pmi0*BtZV!>om?lxswaK)m`FsbOy~>W zt9o4uzjLytD1sS-i7y3|XD~J2eW+ew zSPjx3F!Fb*SYcE!`xDpU8!p^dd7BCS)oDn%*xf99ljE-5Q+;&zoIjN-ELu8h9nHCk zBTIZ)3SM1e`wVNxba5f!^eb+&k2|%8wnGt3N^_*O)7}DKYVk}Y5;;3$I;ng%-E@&?Em1?03!I@ zAw{oE+7fH(;UCCod0@uq!`tULP<;a54%kG`J3`^saCCJ1um+TO{vsK^VYQNVTUvMg z?h^}(+fOoq5nnYmHTlA?Vz;{bTXN2@^CaeZzBqZ5PdCGIaGhHxTi+=!)jj>{KjImwHW~od|dyOU>a|@+ybMqbayXwY1-v16>4w<)1tX5;r;=+t7;xi}YyDI{4Na(86WAGKRj2Y?xc1pB4#jH97vm82-d8T0RCd!5aat~} znG05KCyh%aEK(MowO~fXL6h9&?vc#u&pRxwwtCWhszyT7ZGOEBaOUHPGJ8p z+@p;M_+%!|rZQ8R?2F+LYB`Ir$}s#N3l`MQ0`|G(r+=5i4-#jlqFL)GFDmr4U5sM9 zBR7&Oj5W41oxKb7xjQO-JE_JMzcq8IdI0A@bqIJtQy;F)bh-#SgWJmD$O%$^!qvdk z9n5Goavdh(Ro;Sb^s`)y(OlQtJ3rwNwXd-JmaGg#xz~3mjUD%>5|zJE)!WxcB-PFm zPTP4)zqz-2bAXkY(BWZvK1TNg2wo%f{bpSvcjQaA*ToF0!NCaiW+WFJTf4+CuN{|A zrt!O-Kv6sH4+0P>)8P&>)yp&&rXom*ZJV|?7*@Y}oaI+n;i|;x&sg=sMvekenhuJB zE0x{bG4^RtE;7zF`l4QGmGOZE>(q9V7XKs+F9pYaWC&6;aiLN-_7GuM$E zBn}#weMH0PHsqR#jw+Ndm0HlNG2$9+qaIX-eSRk!LohF&iEPJb`j(f5s~ciSoM#Y`^v1owO}@qJnj*3-Ba9Loy;K-Fee=Grhe znS}w~_358i2wmF68Bz8zq#aTX3?~p-=>}J|wtY^dbN{`7&zmtj$^nxK9C<3i6I7T5AdaVm4Px(@t;y*%sjDTFU!e0Whhv+=0^cS_M@ju}^1`Yiv~&(Ll;Sz5 z86aNqc8$Y3 zM=~ZT^RG7gWm@I#Oe2>)9ye1ZQ<=^46#P?Q%h^igcv~!!2|af9jsm+#zLxAr`<(@2 zY5}9qmHg~cR#5vzh-qz`&176svzH{L=GjIap^yDz&{W3b8A$Kd?YGan#c((#Keb}*?mJycPt{Hw{gf*LS(I=_3Nrr71oRgeL*h=K0=^8F!o+5SmQW!=ZIU_A6V z!8RaL)TgEe2IaJVbvMF|{pzUk!BoW%@}fi8z5)*|q;qa>6nL>S(2Gxy%xi`%g80(! zls09-HFUW2!JzoV6bLE(e&0@vd918EiHX>t@rS^dW`iScLSSgAUqg&cjyDJA?zf6l ze?;SinAQsT4MPqG1=t4~GCyq>gKDAzeSuk<&ko0bs#y5c&-LLoo&)Y5i6aeXObeTB z+T@~G2tA`XNBT=QXPH~c?#h*u&_@a1me?rtN7u z)uu@3IJPJ)&=BD)B7#9Qx_UUbYHI4nZF(pG+TGZJpT9jYQPeI-psrXg7~3?ZmTayZ zX{v09SAE2Crf8d)S+l`+_gqfn3|4x4qiX`Y{Hh;zBNc44y=Vcx8#Fw!+^I-DMzwa) z!;qd7@2HLVdP+?}FdhTXLajsyKSjCuyM! z&9|u!{}7|#?^K!IbvgPg)nXKNtNDsx*uu}%Usm!kxu(s0yMA5G-_uLGp8WUvooOJN`ol=NKhj|$Smg64w@Fy?pHj}zjX0LqTDM|eu^#iGtw ze9v4Y4l zQ&SN3i_c@WMilm@%$k;uua1#{mNs$lK7-vY1Lv&d$9`Q{Tg8M{E$dQ@ZzpGc^5cuU zIij7QLp3M3)!QTWP2*~vWnAx^C|N@bnbyO8gcQ7gAXt%` z#PydH&z_-+svE{i-mJik(#$*`OU^p9W+g_d!f8gwAMVaVDGf%@OM!A zlfUXe4K5QJ8F%C9r$tpDQu?fPR%jghmX@OW8!QIA<9`1v4*L)y+Ez1_@{Krjs7*`Z z3&NaA!MG#-_Jq{-Pj4d;=kYlvw=c{Q*w*HX&iLibhRwk@{YJS76RAgxn@8sph|hFm zY5OJ;HHdbM2M0tRxWIwd&1^E|Ne2R`O{TF4dSW4Hqz!<$4MF_}!GCHS1wBFduo#Se zog^LFOk#&Y0d?%SKLcBC^J2ru#Cm!3qF4EdexJQ!rzTN=eSmQR;I#m@j(H}Vs!p#!;^eivfDqw z#Vh}<7+%Pub9&hnt=~dMm_=@+SNWG!wditT3Ja3yP0CjyXgadA3`bHW39=!-4Cn=o z+}G7{9Jvl0+iMd%-m!BuolpRjsz|d}Q|xUw$===MLxwubWB1f@+gzv0xkgU66KDws zS`Fe>aXGN8rQt|F7#cMAa^aq@qYj=a!l?4meyCggAd#&_sicPR9C_cp0UPQuQCKg# z%i>Z{Z-tnWqL*Wp#2kw?8$K|-H2WF;i(C&6@gr_B zpA4PhOjt{LN1d8k67!}P1bxW1`-R*uHu;yq|8LgaKS=zIg9{?TSCjc)IsuYIHe1#Z z>bcf`HF;AvU=cBv%rl@{w%FH>kB^bdEB}=`gk`1PjQketvCR4_O}K~~&X=eu>FSP^ zSGHd89&L8|l76&c|5xPjcc;jX8R>(S_?7d&v<42=f4G0vE_(g{jrJMvz}O%{4J`+Dh0StrkXI&UbE z{rD!Ac-@fl&FlVGCq+0gg?g?6axG;F^7V0V1K^;9c`A~_38>Wj|3Uq$WPF0*d{br~ zQa|Ja{?GUSItb(ZWA*=ei`X4Q%N|I*)&Bt!P9nRg2uUHGyL;vCY#C>z(Jm`HwM6@` zQ`G-JMmxw|`s=@Z{8#fCCxP}>El9!4E~)$v^kYfhNXtx;-7@w6RB1Sz{!woZV2e)v z6N~?gee|`oN2rpg|Lk|z|4r%|W}C3g{}1s>m|cpZKIN2L#oJT9y`w~sMh>|3?p zmnAENV|DLhL-WUPc@4Jzxel3x=>l^EM_HT>{xzG^*7_y&PyQM67Yi7{srXlRZ;N}b!Jx5=?$%^C8RQgPO^x|$mZ44idJSL zShwp(+$M4%W#ra-^59GTB>3_vhk@H}Xkp^j!{EmstsB&llN!(M-2Mf*eZ;(w`yMp8 zcRT{&1Sgje?XE>HNnZU!=knHUc;*v)_m%Ik4W5sVUym zO^av4cW1owzu(r-D#EppO~huR0&QWa=xj+WYQ7sFKg%Na2&L%MFm|lIRpfX+{?6|* z;-4oKBWh_`a?yf_s5<$kwNZS4*+VTYDcKr|qte}M)$#fvPo9qmaLs*e%f`gKS%$U> zeDSx`FAWMko%3ONFDdN&B4&qc$>)px@u0XTT1&>cy|7_Iu0=CP0`X#y;pfd({N;|g zn+aflMi7l=jIZXSv5e@?Jt&J*V#`upBX?%UV1)Gm^fr$goK~0I3VsXPx%1H@f{o5k zR(Ix)e{uyHuRW@FAi%A>gWO)2B%$duF2d=beCL^Q;0t*iom|Afi>L>S!a&5|OF{x( zBc9<;XWxTIO$T-98>prkh2abyV#R;*DrKD9;NO~)bmcD+zMe;n$>hM_5&A03ocL@!>{B5u*utNerMu=* zh3NIcshtda(&tG$fXwNdL<5RUK!FlI*Ly%?$ywD__6Qa!{|@|R0mjoB z6vFlYagzm0WRh={35jQ_AL`0Lh(pkghubj29wH@cK_zY+L8t>F=MQ&##NqvT$2>W7 z-{{6inE{MpR-Tr-zAz=G*_@5Drfi;}%}~K!e+;_CCDJuV!}b2dq%6}z|2atUN8akB z_OZx6Y0-a$O4w)o#^xYpG3dbqnwG?Vn`!f~6pUNegxYKp{>}2r%13W7i~JX>f2-~d zLiHYZ!%~;P3{YB_o`aDU{R@g5{R2#`2Ht2lK-I(l9y=Q~dcG5g{PFJPF1Y;ph zBfj-DBKHr{8Y$n>+fJ}bG7T4S1!B&vA`*6_1~oW-Z5Y~>fF2$H_?0LvA#!Hq_d0op zFc>UNM)xs}YVMpbg5P$JuWEg06>#=ISE==PvYmuMJ@Q|T?+NuwA@&zfN;#XXBL|2b z*5OZk8ZT1!DBI^-cK`lF+xlgAg;u1v!oyWw3IK;QAY0W4Lis7Gcg9pPqYkVqY7JWS z+%t{H=hy$;BG)13l<_Kv6aP64*S|H&`j7O9KE)93O)iqIb9ZFW)72;{Z+jSc6*YbZ zf#vfKK}M0FTqe`a!||jO@h0W>*ACBPh7uK|quF=&&vVmde`9bbn#}{^zh0&(%Om7@ z1az!a?i!pvUPh*!>{Fy#?7m^LE^S2mhjS{Q2c@i4^@|f%|;Ifl8W%Q!SWZL)Ml}gyA*f&n3W%YBft+UnFXDt?ie`A6PH#8Yu zi8HbTBPj3fGy4qC(lBjGuo+i*wj#d)?UJN$4T~pyuul@^U{o5&Rvr1k54y`MNARLY~uTR0Qc@+=!GAeSan7>J+Gc^1vDNB%8bY(`QH| z@s)FKXs}i4WRM1I!#URwfd2ZQP7U9xcit4v5QEXuFXaayBJSB551NX$$MO z({mTwx%=r)J+d8^|N3Qavxw=(E-eCIWCYhqd$+^9bBqjrAX!_>w%Cu?W|rL?L^z{Rs#dNjtV1ZhJdV9h{w>&QOS?E#M#|>WLLx|p^-eraVgPVrr?S3C> zEJ8-hgP(r;gx^McD^%;W=M(>=6fr#bNh!wYj`RLeoeX!qgRmddx3iV=59$w=cSP`Q5RWtD(yTnt?qhX{BIJ&AbPnL`=JmW-OG3bH4~j zzo-TqN~%2e^aVSpbhmE!sn_(5DZ0G+6dbnw6zJ0N7ffO=`l3OQ2A#>HB zeU>wopk>teyYgcY-|$18Eab}K%D;7UsY*)d?2mKCdtOT9kYdt9hXd?<@yi&8Z>*p` z-v+4rvS+L{PXm9R%H69C4a}}CGNKAOji__UmeNn!sEr%0v}m60RBa*tt%vf#Z1!qR z0VCV>UKJr(ZCU6HZsq>A=g|VzTxO@nLT;4nv&cM-X&-h%HBo#5J}S$<-PvQ^k(^KF z=&lk=q9yB`sJcHkL|)wd>?$Gmf%i6%4>s>Ee|UxJ0H0U0-pXBu6etKzI;CtgXUttK zw$gt4+L3Ur4EPa!&UO9agk1CH>qiariAZMU1-a`-yj7!xMkm8wc&DvAbgil3%bWIt z%(btkl%%L)?A4nv_!i@<)eda@u9I$C!SBb~m3Pg%K*)_fVU*tE(Nr;fMse=D2Ksdf zyT9lNE8Cze*FAV*^r@f44zX)JZu}HXi<)H=g0vyjR7^tlgI6xFmxhL9O|84gAAJ01 z0`|^1yP;_g)`f9y0tYMoe9^$T^s_Ti_77|F?kfcVc7FFeIA`_Qt^Umoq!e0`q7S4) zDc^FN4^>4gouYRS9cLC7LeIbaNwopj&Zy{-F@*OMd1!t=eKUj^59RQ&H^7WD&=gN?~m#j+N!tj~79jP7Uq!S+UG z^izd7JDW2rvA#lRSpX11*~_k~`}Q4-`~6|#&1QM0J04jI2lkKC{CmMk&}osN&=Gre zOzIQBr2H4SZ?eZjZ->2h!6+2MsmRJkk~3Iz->48Q*yriWTSp_sVwE`F=>BZ#kawqf zR=I?LO1lXqn>O}L>KPZA0k1Jo6Z;~_~SEbOz{JoAu(^l=@?R1UR4u(A{LyS0y8$QEepwP zk-|Ok)tb(f4|7xz=3GYbE6_vLmk&u$KCc-UH^Wt75Ey_bKXwP(wtjNGIs41*Bkk5bB0G9EivG9FYJ zxceJX@^jc8k1>{?SV-azJuS^ow0C74OMQ6su45Faq3`82w) z_oA=x5u7f>awl0po8?{<v7=X{kyU+8Q6h)-XJ6#YI9D4)2_y!R5~0!2oahMq!Lhx286pi?YsvX_fyGBiWS zt-Jit4T;n$(E3LTsaqoux;V7bT@fKejQmTAs)PcfiYQ_Pv1`Jtc}IRXW$LnU$pzhdIBK-;W$7_)17vT6b=j zH-o8HwEa}i@abJqO;un8g^JG#>oLz+be~j`#|=%oFE@PPEBGoIa(aA20vXP`DNHQK zQ?sVQ&PNOWWS;rk<+kI44e|nz?tNXZU@vsSUQetk_!rpqIN^e#%aBpOhLduz$qXl7RWn%rX_ zNaO#);(Vo6#Kl=8I%zN^`u>rv8utM67Z&X^7R&Tc>-@HQ-T?5Q7@B`k2>%y_<>%P7 zAA*y&!-5enZ6MDa)#w^c64FmGF1?qYC^V@B_NxgDIV7W&7dPLZ0s4s6zGXZI5*_Z2 zV$lja&sKK6uum_lI6poZ7U4tk>MKL6^dqY_HPI)k0iZj~B%n3X9q4kq#m7dE)Hdg2xPv-txE;+7o%tdFaDH7hLgF!$`*66I zgSowLPS6#9xY#A40&lM82pdKEDgat}2U#iDr~L-~s{Msn-ve_izETgxYY|Me@Oo+t z+RMHg*F%U7V&hWuo7O-NX&ABraC1F8zR27kPcpq^9L~1pD!w8--P+|THckT5I6uIT z2x+0myjpUiJwiB|Tn%|%ytm~vf$4>(*$kbi2{ej9jJgR?@&WjF43D2bX@Y4rAuqL6 z5bsik`^U%1s1Pd_3s|q{h&93f!yAj}65d-8aX^Rns+-YVaACFHTKd3rv7fxJJDNTn z8Qg22!9`i$CGm^rJ&C5@=@x}9V$P`{q1T{ZEZ@loLw2g4uarx-Ju1L4cG;XMjIT)@ zf!jA?%)u;X2AFvWSwV$lUNHBCnEpDKWSHhnao<27g3fwe{Fh^ILf|-g4?02GWviCT z`c;yL(#rt@@Pmdz&~t}3*5S)`a9N$`I4GHOjdho;kIe5{QKPpq^MxVZVII?HZ3q(U z2!aLmwq_8Vxpcnox9|i8Mwa|URrv_r*f95Q1!fg_{5u=@mRm1nA zO9nhsmg8HSY|Oq<{?+x zHoC#R@%&?c@O||=??#(syVJM_n9B32G{+2Osy3%4!_?Y)mC1!kjNiOY&6OmFDc&+hm=Mg{V^R}2Vxj3VjkZoSlX{nDg7!0a89R-AZpe5-Ha|E8 zvB3$zCvcn4>vAT*Jn=4x1z){I@VEPh)|E+^!2_W5sh z_#RU0+Io1{U(wD9Jnmdo+F22Au<;r%V+tG)I$Jw7NP_VedKzI4WO%g8dS!WX&to5M zq=C0_p0r@X-L!KkM;H989>XT6HPMc#82Lk+)i&Ze2K z;GEA%+(lOTy68D+dw0;&^q)84H$0^4HUG9mPzSP?`WEVAB2%8*F zCYqZbC+nlf_RTYo@5b;EidNlfB&5b(qj(ka6}m(FI+d>o1|d2N)t7}EsuhE`%4*WQ zYNt4~8PL0HyhU8QsOcTOVZ-Z)*Tq?D*k5p7<)ZlUX1q%-#&Qhqw=F`~w z8>O=Y5mr!u+`Gl%d1IAWFAK76WDv3#dHf6c-XK-PVsr=5h`Bg5mEPvIF4$$?vwH0y3x)C+E%O|b`ik4mGofxtXbY&oTb7FKw(9wrL@MA$K|N!wu_s>vf(r{wfhgV0Ve!~YHGVPyb(IjG zEAR@H_0dVA8)3+Q6Jm^bwxojMYw#tq&rr@D$$*83?f~QzlpN?;t$nztO%pJE&~%hJTf>lY6^Ir=t1$zCLe);+{MqeD+=ZkSUXqxf0_m zww$@9*%08P-gaGwx(HyuRl0RMg?wW|Yt1N_v-8Prh_{yE^|z#@q+-+2z%j6Ug|^ve zbgXsDE=$!ha}j!CziTtAnShzMlErz5)u&nP0RpU-M&0T~motJr&>uB`xKpD{LWJO;W{wz=nc+hdU#rr~;ce~)8nd;veu}U7Y z>CUp|o#zqsv3~Q1`vJy^xDJxyOSpA`xZ@`*rU3Ix-`GjB*F6Lu{$Gls7(PCTdx#66 zm;#8+XQ_6@QLR(&*rpW#fsH4_#~{K9rr3w+tnD;{tl-X;73i7@2%B}kd{bNFWgz>f z19t*DRn~*1TkDs7m(tV(pZ9=Mk#HW2t}7((=%@ZxU~ef)AbX2j*JGqT<%GGxi;KH8 z7#hT63jqzjrO*x(su*Aw+%Zek}k=RT}%S0c8SnHo-Hf!pa`NIwh|$lBN&o&XeNJAo-Yx3Qqg87$nIxYOs|f#J|nlaUXszoNVt$gV+t6%n&2}(5|iW< z=!zu95AniOG6P~|2L$eLqt1-g)0Ce1dY2G=C;4yB%%G7=%fV%zNsnTl@J}YwJn8=? z+X!JW7KI%)#Qa5vQ_z5yMu@gU;Rl?@z@)LzUJtdyhJQp11R(~H;M3+CNIG~ z5k5p6>`hNAGJUwWpwZw7?77NO&6#^z&~WV zH(TqKporN|@iCv_=~iOQ7{L3+iyKEEtj8g;+mTpX6SStd2A^J}bjwMNe)9X`$N;&a z1#Nt%p9m0oxq#;8gLR?id=w_KVt}|CdePr63xoLSL2EocCzSWZj~O9rC)>3swSxKPw-oLrN5Hwi^29-nr;Gv!tg;?rZMWZbtFo z$B!#mfmZgUKnTbEvxYG{>%GJeb%JcOuf2ynVDSV8<4~vtGO(BR@d0IW#Y`BQBS3xBVz`k zK1sU=hBR4jLZ{Y0d4|lY(=0+2z8u&C@iB2$6+^B%d)#R|jqmNI|k6@2!)P8EC&y91^#!bFkR7B@l znwl(5VxQ!BJBFk&@-<&(VvDjC8_ZpRhKeyHOXyyLtFWeCn>7Y|)p$K!-aAqX=E*V3 zXBLD!UVCtB>zYWV>SsaU+cwSy!ecP^)z1R}8#aS{`#hMUNj+pr+rG@9%~u!hv_9J- z@=FyB_l2?O0fBqr-*;5_msC`Wr2bZbJHb$VM7a3l!ys_YZMf-9O_82FU}N~hVpfBN z*r+86K)Y%hMHk75>=$-?i7po zy5jndeExB4e!H0mGe_}KA2y53@>MJAG?cF-?)-8pgR18pX-pDsHE(Z$0Q538tNP(h z3h{aOYgUI20ie$_7o4pa^Mg`O>S%*Ni?s?LjbUWW6-cobJbA&K9I|rxvo*^1va&a- zwHAnT_z?)2p{sJAQNoq6iQM+}>7_D1neu*NTm-XKZFA%Nl!0p2Cf~*flB?~{~ z1VepvH@+V7z8)=EnxlCs`P9b!_?gF)_vk`ndqIRFvWP_*0?8lClZtrg%Je35I*UsV z#O)DM-uxxu_)^0No6y0^c4?HHbVE`0bfaT$0@F?sNRJkiqM}ZV?~mxUlH;YQNHoC~ zcrIJQSes5?@5C$Zm;FXkbhr6;eVOfx4;O`xkKtDa>c8UwGA}E99mdJWRljr%x=ZHL z&9rW*bsll;vhwvH%X~an)eKgkp`Ol*pPOUiAb?4yuoN^iCbAXk3tKCnJ-Zmd=k~Me zUpAAl&u8^|WLvV6Md;|nJ%E39ZLT%kf`?ZMyBi$2;(53)UsfpjX(8l@L{q`t$!Q`) zZ+uY&cJ*9KRryC+$gV(&tt_pd9F#nxs#du&sD6{Q*U5XG%1kf-kTJP?t6!#BMD;8~ z0R-d^c7AYW-Jb5`fwd8Nww+{BT~gY<7G$(XFy2QVM7QIADtB*=12+u5QDynmkz_g9 z#qiyH2Rgh=3lHc0=mZmyDg^rfNj7%QJ)3BRNW<}(?fCHilDc`k_#lbO`lU;xnD?73 z;3v89x=H)i&#U7PwCi{mKV4*Gbhw?Mb}YFHhyDnFmI(4|Cyt^V&$KNQyT2(SurI?d z3!U8$8ZGS__^Uk#z&vt+i9%cp*6$?_W)rOrc5V4OM2AKqErmu${cc!VNSxrSPBzI! zB}SB9I9&n)fVU4P`?a<@n6Diji5@@;GWXq$+3n|1EW0)y=fUAGG_9Ye>Bc>LHxgp- zq+GjJlCGE>L1!NBWB$=`%W6{gA0m83o=tq5k6eBqS5eh-0VC!0h6J=mpc^=Utj7E#lUSem!-z9 zKRq>NT2lCrlMw_a#wA}#|v z>UU`8kLTtK+u@*(P)zVu-#=jk+Vi2|5?`VMVlV=)*#YN7t-)K5dwzY!^*P|ice2HB z%z6?W!t{7{ii*C$%otEA z?X~9@fI~XdfeV%aXF{$UDE-N;SX`d0P+*dack1JovhT*&##E#vjcy&CS~VD+>qn>2 zfhEX(vlR&3yJFqhVdQ>(xf#1LTO*fd1tXXSpIY5kth`cMQyy&hZj-dhV$w8_3+3&5 z?cFhKC!q9VF;_zFH9`Zsm10e^14~Z+)wo?Gv%d&#o(Jj09i9qc;%Q6n?`_fxKEdvd z>5h)mGN&PV9~Yjo)t^tr9EUu(vgM0WXVIF5g!8#Q2AbvwwvaS*T0NO|M7bIXwin#D z%A#8g(Fd)_@a~ZDknI1IBh{MSm%G7Ru0gRY?BLB(wD}ZQ=gKU2;6Pkabr&)Tlp(gI zl?X|R2?(zoGEp>~vc=h++Sko1C5Xy3VTdVu_XWII8LP#wwB zDipEoHpRh1rFBj+gzb>9QR*Dc@3o@&YBkf;$ZHmLroiACCE1O+G}H-Stev_;>8~(6 z?DGHhUjR69%+N7UHv?EhCU`OlW3*W}*P3Cl z09usgtP9wOLsw!V!;L;x>Xw81%obpC*PoN6s6S<|J0s<(u1<_&C1nI+gsB*1eEZm>V%g4$L!M*14tK zo$$|c8&T2ItqAwEwvMEl^YO9N5?czh-9iFw4)-+qDM65MoZ{e4xeLPmjkuGx&m(E5 zk)8>WA|wAF0P8>$zwm7_>E^-nZUc>TCoXTEzu2(N`Jb=j zH^`%j=m=k$@GvvZ8#@7CsYS0Mqs|BuV+&Vuem`tlw;6xWPn0qBWG(T^KgmmbW52Q4 zpO<=1elTaw^{5Ca0*XLg5D0oOPeye<-*8pBevZzg`@EfREQFEgU15I-6X3TS?}F2P zfkpbjufdP$%_jJ}8uvW57Wb?DvP@s%^Ggd~hs~dvk3OZ0I8sv$U-&d7oVRo=mvOTIJL`z z(Nmv+Lq7XMd_KNa#!E&{#-LH=%`c448aQdi`|X!DhqoX;d+kTVQW$}H8hY9aTl;gq z2;=xazm%ooV>gt#4Wt_;vRUjVTIIzY z?l@zyY_sjT$fXJCx|#A*C>7c(E*&T|1z?VNCBiA>@Y<&a8VC{}BVfN#Pq`b(NywsCylZaA(B*oCm<7HfR;6Jq6 zZPF>4;77K>Q{TPZIAjdFE6cXRbMyWH2i*1mrKJb?F5RzF<&mB6*T=pASB#x$gyCzw zzImRxP)R@Hr6f9#9=)!*-M&Y4EsPyc31JpT^s zYOpX}($TNK8Edfm86MLg%FD=tJ8Z!G;*TzdAIYyj+;k^4*X{qx0QAd#vD(7PhyUA z9ItKg?7U_8-W;PV$=&(J8rV`?%JNAkl6eYBq2T2iaDMg#Vu69v*1>iC+G36A*I+<3 z-u$IA!GZEJ*D269d71xa{6!W~^Ut&w-AP6=rl|MxEeQ;|JETMdPkEx0Bwy2dZdA=7 zDBoqpt4tn#k+O3}n2+6Fy!&F_2r?>);O!?D+8YRzGyE3ll_5x3!2uH0{s5nM zO5-9Ixp$d+^_o)z6aht`_6U%tqJKf3GhRAnUEE9R^K+f4@=+U$_=`k=M_}A2pf@!^TX6bJ`z+ zV@J!-MsIC`OnOqDl4;-h;yWYYlZ*HiMZl*w>*}?ppHw1kyd-&|Ufx^t`dvR0o_}aR z-1N{=c=4ee{L6xIcU%j-@uTTIA8v#tD`-5j;gwI;z(r=?%N2h3tv{Og&JUjpWAMwy zB;P-OT*hiMrWI*-?x^W_6POJ%e|RM<#Ci4T_$%R2Y&-0lTo`oznRqi+4u^JahizN( z;Gg*BAb%QmZfBHWEEHPCB~oU%{rdHDV~(M2%u#|d$JCIR<2x|G#vGsj&Mk1h?W1t{ z6u1ii1`f}G5tj_b$6-p~pLo;7Uq;2QzKAi$Ge72!Ui@lC$b0rvrRYwS{!swW^WWaQ z((bz{6H%tUw3E_gqe&6C^T#%JWN9M1DRd5nOd{e5P9nEmH5tErx8FV;_}AcMLJ}os z_peuu!)@2z0UH)i#Xlo-;FyQu=aat*YerlFzow!KCc-oGABCa#qFCpgYvHMxpTUbu zX5wp{3gE@Xfkot+DKPb(d*G`>PA8ATPQ|z{{FSq;Y{RkHqd?)Zz_$O)L@QE<-e|`uz;IHA~3#bqisfkQO?6)0BncseD9mF@5?8rT8 zsUn~VC<37fkZ%$HEB=Q$s`2^h?Tw7W^?7zDJya&04ZI*523>qPT>8Z;;fmpx!nwV> z7$)l%r7xl6^ZI}NXRtN@D=_bYyI|f5d==ngvuj$&JLXxCd*2k8Fd-KPcBeO}+u+6O z0@vIE!&WTCHrURm&ZxV757VcNhqGy2V8=;@Q9?z~r{9%#7{0jd7jXZ7NA~HH?)j1R z>84ex-LvnGuf!1@OpWj8q?$LVwDGW|TC%m*<&)u)_lLtXvwsW^JhK)yEcb2tKo0!$ z@h`%QcTa`F9v!eJ(b>o*d_I2bDwsF@Zum9hvxF-IpK+~&!uWV&N-X`LBi=8C z*^lyBt2BzdkC=K{w^A}-_IAe@Eiur&pjEsjt(0?8oJLqiE+R#`D zCM+W`r)2I)V9xv@^TpUD^T5?3$ucPjfH%S|#Q3VJDkv)}gH4-vK*6q7aO>4W%p3_$ zni%%V1S3j`ClTfavS58sT#!Fl&+RKq;j_I5aTg*De_FZ=bnl2CHKCprhjzn(5i&U6*dq zwS78YXjBqxACyl#{uJc**HJ#$es^&hy!`qWm~d5Z=-IO;q^GAtt5!+zG*=KBJy!%0 z0)eGVmqKwdeP|AF-F4SN|Ngi@Ctx~x@+2HNas>7lAAn!KvIp+HsZ(cYi@)GW?I;=frSSZ?PB3LfF1)_)IXE}9 z0uC3KKpDMr9k)+1d-Q_L^vFB>LH(S{aws{l4~|um{4{9Wxhr(*5V}rq$Mz7`KCos6 z>-}vhPj;Leo6-${I& z<)Kxy~R8@vCM+uHk=zNG2a~yepK74M(9dN_1 zx4^H*4uIo0wuj0NLlx_n3ElC=)i*r#)-Q-p5JR!Q3a|IamTiO|U4yYgkM7W+18$DC zj(kl7WweBIKjtmLo`^QcHDo4*agM|&lfb(O%$>T~r8f1H_v)q0bR%+wVl7sP49eQJ za*-1wh+a^LHy}(x|Qee-}DTS-o7(rbm-#RK;^_I6W^R0ZqYBM z{i$uCTkpQu$T%&r$KQWwK2*N&8%knoU_Y|?Wg3ZkR0I@(L_px?n{Rfn&*ro+lFy^R zm_E<-*T&}=U(|3uKfP0TI0OH@Bz*FlvufEXDy7VAFav&E1Wqp1;>~D>0Q!iYbIGI5(gF!vfxsOu6(kGQ0e0mTE zonk`!C5vZbYmM#aHORDMX#u?2cRw9JMeyu5{fbW1f^;r{7iM74j_Ie@kgdD52|pw{>W(r&#w0`8r2f9?T0}rF$s!Yj&dqc* z(~{1KpC&3(I+r|AH~I3C70T_A(e@YCUgp11$0#dh*o?&ZPPu;;?d#t^;k`@BDuBfB zTV+uX=C}GJg#A{vnh?>qCcMg{2q*%zM<6x&YfWk|tPxv~@1uYAzjb`5KqC4)a>_xU zZ?TdEn*BDFp|3YS+$}La-11YcStD#c62os9nv4CO`c1y(9e(FI^WOFgX2H!v@urK` zOW&I7O)eTAcOXoLc`-|r61DTUH-i;E{t0BiQ5+Q&y@i?zGRn=lSt(*(tR_k+PWJob z93wcnj!E$x0tMq7kmJ}2<$2|If|#c|npjc!Y2NwfhvbQJ9@`5=A*NI))|km=DCN1G zP?C!U+@34EyghYw7yamJXPw!#*}gF!ji1ViU+T$L%Jao!?QUxONu8P`!mn)epYg3b z`z@|DX{9`hfFh6#2#Akt4nEKLU>p_9NS-MciN6+mTB6Svn}haKdDGi}5b}O$ciU%A zfUfr9A>CN-Bv|vU$xPU0YkSk%@7FB3@<|c6^A(1(jGO@F#R<~H6UV@lFc`>EKuiIx z{U>>%oM(ByK%aQ&LMA!HgGngmxqnMp2gwt~KRO8Y=c$a#QKV4g}ty^&4!pX7;hz5(nf7-FSUr%pj-WhGP`EP~>{KMWOz_X3q^ zQUnwMMSu}#Mm~>yIvwy&@=VDP^5K?Jo;|dsly3<>Uu+KAFLn!U3j2?~^S^K(eNmKl z+b{2K*HdqerS+Z1fYwf07cmK?Jk<%MJe|3eE4(pJMEiNIXA5Z38C!*l-9p7~p<=gC z*KWa~p7yg^NqUe^;##!%?nE-z6+1&Rm`~|~^I7lD^HoL#=Grq2DDN_1hV!<7Po)g2 zY+%cj`Z$GkR7RANZ4Zlo#ntqwpjI}9eD`C%zE|u7c6;eiH zgPPOavuk>DU-KRN3hGk)?JJ&ZyL^7tN2;4b9jxf679E8%$q`=wEP$VW0<^=%T z%bg|jh&;D3Hp72(Pt6xM@zVutDYJ`gxv1$%kTT*US*!zdQwd8W$|=C_wzcx8KPqEN z4J2>tjS%lBstZjB`#trd=4f~Q#EBD7d7=`!^t}R3RUU)V*YJm@FnDjIeGXmuEBKSS=bwU%u=Y{$FG&tuQ)8PAGxg5^NSMs@jp2m}Gl68PiTY}FIv!7Q@ zGke8WqLdob{;H$U;gTuvz2U>))+>6N{#gev+tSzk*4+lfk8Ya+L$5y1V4q<2dn1ut zx#S4kIef-}6@OrTK-f?Roq}bE=Hxk^7MaLv_)M_RO;MgWRz43nA#RFgqK*XpL?iLL zWu`XlD5`k!?2Uu;5M&5SXkz<=>Qr}5uYWsv>LjjVr=b10BY80^U5mkcwg&Ij;ih$p zKyn~J&JO<)+~-C1`E6m?Z9juyGEfq<%DImQ&68~-^GzlLfxAxm+PT>#_W6KxE zWK-Fnb~fDnlgGWA1=5F>W&z!vQh{BunYSga`5C8Y!?efE`f6?%6U%LJmb!n zm0Vgval9oRudwv_V0KfjaXJhj7ybRqM_Z1Ys+=M_1~#xjXwlAU{^{=BJ;a3qAr9V}NyU_;n*AA` zi;oIYzaJA{Cp1nzuUg+neNiHce=X7HtNk)5%}|FNb^Hw+5$1mgPDyr>9D%zyV;X#R z)ZzqjwAmTjc+%#@N%CBvHj=0k~2mSUcrvZcmB6AQM8 z!q2KJhx>*rXG(baRbb{4Q$?42MOCsEcP$9{Xq#a z&ZPtD0nKdxxokIhKX!08(5}2DML-cqLIlJ|ksqUc!umYa4fP{GFTP%Aq&0%wGV)q; z*e))U0JUJ=Lvo)KpM#&wiV^7{W|_K z?}bL1FFeHmd#+MJaYIOU93e@DBjAet3 z)$PyCCZl-g-TBge(5~xg`pkPxa8w>eAOZq!zWFBqA}LjU{_~%Q9zE#CaS}gyZo&A4 z=sr2rmPYq^m3H1~w93}~0#fvCGXdoER1v5- z0v~t z59<#LL?@|Z*;I&0?MD#o7!V$Ut&9hi`IBI2;nVR)AQ%%12l#;DdW=slIIHr8l$U6oV4i(q5_Oq>o zFWGEVmLK+Lcg~j>cY185RrdY-o_E2>Gfs1gfFjVc5FkfDH}j!>(D*%{pFqdUWi}FF zpUGv`kx)516>@+i(mFn-$DFSzLUYzjyJ6UxSDl1RG zv13Oe6AR_h3i5;2M;Tt@uZ-uV61sNH#;&QCynWP4K8|H-IE=hNUQ{8EE^U}4AYb@| zM&>7{Vmc5Tg;J?M?-^pSkj-hP#N2}IZ{Pn)LIV!3{W&=LBTY2gniPQ+jR5&R_J8cx zJU_{fj(5+E@2jU$aXwizoaaX!?32NyE&`wsmUM)?7NiUZ)#&$XT?g_M)W!8wV) z+3Mh1xl&pK(5KRA?Eu+ACt5$SY75`defRBMaZg!nIT8`fdI|?U{VSrz87~j3y2dcUZ0;kty(7O9T z_q#}mW6y$ad zMQ&<`PT6yqCZ9r%8WWXM-mAwl6d^k!B#ZlGY8z1AHsA}6IgC8RO%C5A?w;$-x!C49 zc-~SWdr5CAHbftTg*bTDMQr8rQ7vo6cuI+Ue&*>yB4KYWwcE zq24xZIpWTJ2ftp<$x5g^_z9dS{us*N{|%gK)fOtx8w;&_eSrgb;icx1Cu&kCjfGcT z?YEsVHk41c@lO`}#m;1-NfEevPQy47a>NqQiUTzc#?;d!kf2)vapT!c@NVkOR1t^j zFZ1W4)qBs~n7}$rK09LLl5J1ue8}mU&|UI|3WqpFb1t^gnXv=q*&7S5%sZ)-ohK2>& z5!#EKNR)NK3Zb4)xBalbn78!giE?m0`xms{^G|4fun?baITO-*d=4_s`4+U!JRMre zr;OB`Y-BNnl|AKi(jX7Q%@V+!z_~Eezi2 zlT0J1G3{rqts$XfLNzAI*crW0OXQa_=3pE#N7yi-6}EYdKk3VM+qppJ!6ddecl)D_ zZ_wVBdfws#ck4cQ{LEnEg@}dpX9`W=%-1Ph#8Qm)lovj8`BIr_NJp->{NW_XA>5`1 zbLQ(0cl1q|F8WYUri^48=V2(FhqpJfh|i?Z{zbMdKqSfIGxEq-qn)vv`><>+$^Q1| z3`f(=k2C>2^WLS8{z#LZ4N4S&mVp3yFZ5dE({T61Tj))Y>iIvZGoFw6Z9Bn1JSCmW zdLUF}`vW2A*YH#S)XsIJ_{l5gO?RvY*Vt-d6B?8+IteAYB=6+0eb8$AE0DT(1MZ~v z;$M)?xM(z_;Nvfps5^f@Txk2`eBfZ#3u{jR}R@0!OD8P5i`pLMl>N#4~P%LPMYV#^zUv6<|Xe9O0AE+8|i$#!h;d4*UlefUT^6FbC%6b@S3Y6-eGt-nl1nt~`^s z>azrCK_{|U2tUa)O)VD@){&cXZ@yEv%61pC-bnEsSr{p7<+B4NxcyrC_&MvZlf0qv zw!eKp3f}3N_n+g=`~A?a%W3X&np~j=ML-c~#t4w#FuvZSl(T^xdo(gH;AhZbF^@Pd z;^(~hA>$=w{Rql4wc=~Z4_D701X%pFP@}u2Cuw0MVh#FIr~DAWhkt-J``?6&(=UW} zU%VYsF>t5$1oy|l5S*68J$a8q9##Bw88JMW@JK_OUA*e4Nt<+ zu0x>#>?1X+R?GT@(e1CGyd_=xB1#{<~3oEjH$&3?linI z#KtAlFiR=9U8(i>cp?NmmG~>6;^d+Rq9Rr(_3F=aCtjc8B#omZUcC6NGi6q3YbiSj z%)9y^6PJlx%O`zGT?jw&ShkQZ_#_GJZ{Kei4@xP%ZAOE4J8_gK0xbXm@@w=bK3@V@ zQ|LLqfSY5>#YfVKk>UBtFnQd9xFm&oS`O@8EyPEAft~U5=t>09^}Cr-h;yQ*NnL}0p z5-Fn+$(wErkN+rbt-tYCaX#2S$RDf^3JCHC>x1oM@yBRS;xgN&z`)+E;hY|+VP8@T zHV@)0xC`Z*7W-?c)HcZPt~#=^gm-{rfLht{6qtp|O_6zQ26x^mc>gl)ym!@|_aKyd zp1cTTWMn|ww)o{mOwH*nOtiGi@=m@)Ski6+n^bkt&{ck5TL9Pj2xIH$j!q+yoerFzJRFy#IFHVBA z%s%4gTx#@2QLdT}9LFH=KcAg|-3L#=zS5IWj_VuMcg$!7Ju_3`%0tY>ZI`C(P4#h|1Ug5=xI0;}>Cj;+(+{!v^n$wSM*+^hE|K!C+g8ghnT%KPvN* z;2|^-#-HX~75=u&wxWaZ{`UQ_YhM{uo;+o*HyOZF%+o%j4fN~t|Ji#FK+B3EeY{?{ zFXuNoFf%ZuA*0}sk&Foyca8i6#g(8g2rS}a+I4Zw?)q2MRbkC(Q4kab6(lpjz>t|d z!{qpKy6?UJ_f>VBbMCqK@g|Yx-G1-fs?%LvU0vN>r%s;S%0sv(at3@jY;XvC({dCK6BbV+*0<9+y%?mR4O zA~PA`@XF{ZBJ_V)YGxg+F~wywP(v{kHIUH2*=L`fID0jr9Pn!8lfypG4)_hcIq$JK zaz}z3ltv8*O^EX(e-BCAjSQhl5IY+h0VZmAG(BCumKT-cCX0@$ZRt3Or7`6Xe*qto znjP?%Qo8X6pyXIAydozHa1(hURZ4E7I}%D@opGV`giAB3&W zN?A#lmVD5(N;zRptxRd=wvbkjhj0f_3!QOP}>71F27@haY*7`nn|ho6YUwv zMlo@{8p8Vt^PgKS_dT^5zJ^H8K$)O5K!)}iLMforCek8?rKfvP+LpG<(YCt^a!^O|yPBHsP3X+~a1qZV@##dVo8A-hr65v134f zc;BFOW~*dy7>3b$k7ZL&k{NBV@;906du9QWadgPb$`sIDEtDyPO@~kW`3}-=q5hQ`f zDRJt@(;TMg*^VkZZw3(Z>Y?08GYT(tW0or`C z`)>2yz=is_J){8zPO8EocGZ|}bRLHYcj|(@*L05y40CuZ9m_NUMx*yS48p<7_H|3< zkuTyKNP9^AoHt`mgpQrbowj#__;}LFjs5b=`?t#Gt|93j$VgUd@dd(Aamq#+m>&>Z zwX5Hx{M( z^wk)U2QyfijN8`wn;w`UtJudKyXn7v$KLq(VZ0VGn?s-l+dIq(9aEk*1>euB(El3Yqg-jhTlYSm(&+fLY z+;sOsnLc5yyy2+XQjOVo+&UpUOEh_`X3j6L2JmE)-KZFyAH1%{&+6iFp!T|Iy#gPP z3(N}D-}Ep#z11U8pV0pCu7LRxkA)ceRbwDAJ0Xr>18i(?aNsdyHY>M3)sLky z<#5`0UjrgF5`hPS#u?D=Og=bO90t~*?iJ6xh*KWK(+3?*r2!01X9Tq<|uW^9HmEIxprs@v=Dlj-{_4DUZ&?l5 z?IwWSilL~1q6S_~8gTnUV?dmc1D$mUFo}_!KDGv{_&Bm^6lF<%c{0=X&$>IoKP-@qTf4QLUg`BWRRjfTo$(V+HBIA;Dp!d}fGRXjq0@{# zI3{M?qX9%jI|v~WgaFwOhOn+|U|>L&Y%Z04IPOMFqC_AevwFvIJg`&B1YUy2geRRk zU0&Bzs!39Ni7oI$5 zeSv$uDYx_RYIqquSKMiwAOSsZ?aZT;wm9=nbxy~`Uxu?aV~esF8iz0cQ2L3<39$coi8WC^1)#z zWN^FM-#;Kfx@)T}TF)oO)rjW|td*%Y7#9zx)oTmwvR_+_Ec}*8&tN5-_%iv~eI2p^ zPoZCbKtuGNL;AX;dLVE7@uN<}hC0{CVC9~Y!M^XY{&EVz$qU!t^lN;XGf!r=CFRd) z)1XT&%5jG*GJz}23;g7@AM@^b`_pNEYRNje?U5CjWv{?$nBcJMY?+Cctd5q1B103# zb@etw8rvz`(=R{2?Fo6;$#Z0Ub6uiMy{-;l-umZj8gjXgndrDeoZ`=}@+QBxe?lYq zEXP31*l^sVZqLS#h5$_l#<1SfYEuv5P5s`!Ug^(P>Hx7pV}>m5!XE}7Wnmzt zjUyjj9lUpw%nDB%EmQov&#YEC^Q3;c=eEW2zt8lkQH?vNbzD-j1AA~;{QQNW^CGmM z(gQVqX}g!EPuGX_Rt}bZ#QG0n;B|E43wwA!fP4C?npF2u#vp^ql{Y{^>v5EuGdVVZ z<9&3_=D?SIeZQX>Mu{OWdex2ITN-M3mq7FxjM`b7H}49@Fs$KAcSI@7y4@ z%YP${*lw}39M>iUmoN$WwD?DNZIOlRIg48roVS>_T>^ML3uS>nm$3Ww#Knx#wVX2L zb=woYSgS5_{QlUdo$oF2Sv{)eU!WyNmxXv)-6R?ATqW72>4=9cBt|g(8RR)dVZI%ToM=bUkn)Ynu; z?eAJv`Bc-clb5dkrsr5!aeM7(`xo{Dtd6_??-+61ts%X$5IP689}CDfVy=guImT!%E=#p_PE@zxkRdIBr3|~kbTC>? z9jrel5(cqMW)K$>8OB`;__Kr{1)yOLEiK<5dAiCL%j@*{>CEG)`T5f^ zwKL~x8Aa!vnlFZ;240;SP;&>--R{I#923V`9dlE1EmOW-q1{PXmSA^Bw9g4kWu@hF zR7=I{4|oj>^h^1oUt-<(9I0sDBOz*BGt$w)zjc0_EZQ)nOBIt7Fz)oFThK2f!St)h z9!FZo$*T*KS+_pkBNOV&WWQ;(`ogL5ODZc-Kv`{zlytAdHNcjJ2zHk-XEe6dMqar7 zUQxH$4lPe^p}A)XA%MnO@_46q$oM3ldlp|+C};XALE3F=e_ql%I(y^~4=lxI7a82} zKp!j_ZRc&J1*H0qwE<}~HPBl=#w61R1?%4N#O0QI7s@+MoP!B7xOd2>L>< zVHzFof(H#M)6}p6@{hRxIqvB`5HVxi)t!~${wxL;j;=nrZ~iLz^tTttBW=7fD3jxk zX#%gnPa$w!BOV}e{y7A$Ysdri_2drK`40TFa>VgRJoZE(gUhoRBDg3>ppzHBKY-UW4?!=g|9(fSbPJyvCP&oUh~L=Vit?P?mK z2JEf{%cU<{f~_9WW>_-AgZgo(*Im87B0l&J`i=l_Ld8k1*X58m;GW(+P$FwKcgxeu z)?&gClV%7mr=FyKl66kvG1<=7-}OLAEI5p1GtK7q|HAUbl8t3-OgQfGAYxPdz_MoN zha`At3ZQm!J*5%j02<}M;7}Su9774HIPyn`Q(ocxg>WyM5Qo>_j0JvC zcA){A8|MSrAzCRk6QRy3pk*9JywwmekGG*gYftVn!!(x9gTWzrT3Lii-00%pni+G= zdB+!4i!ZFkdMRQ>4ZN~7Km+2o-eJDPW=qV0Fx+R)3+GQd6G=Sg>^98olEr&mOci4; zo0Ufy=;-ljWq*I4)U3P_J6;`wZ5FX^InESebi4-!vU0=2-PlU79J#sveT15mF#{nE zPQW-+oJov}g3C>anR9F*Nyqa~3p((!EUpbo7fMru4LlfZ%3D6fms%}v#4-=@S{a{Y z8yL=#T@KdE=m`t`8HMp0K?^w&IB!D3cYXWwdcL`>Ll&-T$E2ORTuHse_0tU{C?Til zdE>3FKgm^5f{$I!=I7+Cfg(BhLN@J`qR%NoMo ztTdL&lL^2pJOj4Qs^lO|q|@=WM`nn|vn=PKe6_Ag z`F61W;A=)3`PF%6y(zGk660R?5bYp{r>aWF`MEK%Gb;-*o)OVWJX#(U9QH^gS1ZLM zUO>K@J&q)|XsgKTrpeJuT2^}Tx2S=l23}bjpkZ<2;xOM{Mm8ys&{?2ENEnA0_jre- zN#b!H!<-i(18F_ZD4e%L(7+J!CB}g*i>0A!vD6&-wvabzpbQ@z9FV7;@02ZF!`M)P zWSFqQpNAMj3~f8EFAAL?L5qu1pdoL{N|{Y!jR4-+S0)dyz>BJLdZneI3U<(42$p2@ zq%A>w*>E2wa%`r!oJ(c!w=MN&nIi?So5MnM`MKj5zt;KnQj3bkE+8LEi1T~74lOlK z&yxp9eD`4fbL+5oKAxgL$9{>;qeX2W-dbsN_)jx&nnuj}1mGHop3KI(S!I(x@u=Wsk!*&RJX z9yDwW^y2V>rRG3ZOyLQaRJooHM`OY8K%NFu$ohV;CXQpHqEs}rz{l-|{AbSb+ zSO3Tbhsg9*3+2k+Z;tt6#G7MZE0WG%*dXgUiEd3dh%32}PP9|VfWkcP=P;5B=^PHBA2d(nBP zf{I~x*T5Zj+#zjkZCcGEk33Q)PoDfLsWOd$8w7_m12)hdYGr)v>xo-z=3!uBoZp=e z!u-_y7{78j%+H_e$hcFM#qMS`t8NuO@hi>X%e8LHg8a0FwA)#HwdbCNebQBMCRas_ ze|T-9*nW-kYFSl~07MORh#;P0v{5F?AHsF%$`18+lYJqGkCon zTBQDlpIuvjZ+C~=(*G&j=h)K$tp2hEnR?K@7s_uVJt-mN{7#kPx6=1i5N{p)Kqc)b8=Ce#IiT-+q_oQ5HXK2YRTJc=WO zAzVarXWna?r{sQIgp7-!sDa%`1FKf8l8qZTYK;y&@W5A3&1e{sHYFGiAM=ETgU350 zRW5F4$={e_G{=N?%{az9J&y65Y3DMO{z~sNJ^XB>Fn;46_~l{ z=hyG>G|`{9GCf^awRt*$Vn);^Q|KhdvM^s1NJjG+Y_4y~+uc`!4>S$Q@%v>Zli^FJ z*zPy7)<(=Y%B`F^0Tznzx;kW8Id~vUwvXo}ZQ5?<)iH3Rt=FK}0N^iE`w!xY!-maW zXjg3MiJPoZ<%gW!E|lN#sEtzf(K?2z66XVDo@TSdvUYQizO%=t9i^r8F^~pjx(phT z_v6eQYOVy8x0a7C3H+U?oqp=(-LQ=@b2dpMF$4s zgAv2Lf(?Do?2x@92WJODgx*Dm#N-_{cv{HFKU_3d-hf9~4CQ5IQVB=fLRKEWZNB`^ zLU`L`7mQ`gzOy5zef?95bsPKT#!Je21P% zsqy1=&8xo=usS-#C;Xlj!7{^UIZ4PIo+03&95u|qBLl7c@e|0EN2GhQ zApv-#oFJzfd|qwp6j|HmRC&7-rk?wNn=0 z9$8rG$2hK?UV?9BaGOS*p3vD^kh>#Jb4T1n$4QVX$gXmG9E-rAMOjtsgt!u-3`|~7 z-hMcRo4PQoi@n1uEAVyf;G|o&K&<+d+onOlKr;J?EvU*dyKBb@{U2cbY@&`FbN@p7;>tiSJuO_1*alfDsRyJ+zl*xEf zUl#Jcl=Y`n-j;+6te3~}-0_jpNs|n&7l+BTv4p$lxLenmox(eYL$5F$M|C1~P;r@x zTPei*o8JNI+uw;eOXo|Od^6l$%pAgcHpK}8?qzW_@vb|IQevlFWcgnnly5xHsfJj2 z!5|Og15oFlHbt|pe{6wVcEbwU=w$$IT=KkJ`Hcm#n(G>|XYP5gtCqj~ZKc4T${vUA zA(H~FzO`P~EJshbJ8m+Z{hA5-f+}B99l}LoaMBbxsuc-6>*ROO4QL$YuzJOCd+N_8 z2WqHT7ar8JLtX$9`c@oi4oOD?XC(-J0e&+85bVG{Rr+AIx#w6C0T?`_{Kbz2{jnptxrG6)&y0Mz8+Wl>y9*5I@E z*rd#C6y0~VYqV_%JlHQ?egT|N@dTv3rX6%>_S)hidTJ`zb-a#+-AnAts3utM@9;3L zEhV}*R1>zP{z+Xpx1>&0$4H1^ATpQGPSW&@^*G{eYlq(M4xP@S5{~lYw1i2V+32+M zCTvSbFV>v%8idh3CG<#4I7-eHGCNEp~I-r z0&X46pyf*rF1%=?1J7U*w*kq4!@L0k$Ee{iLcjd#|GP^*@{N1sBVV~w&iUf~@~5S} znlj_$!{zPcSx$m~Q?tyjWAaw{(M?!FgEZxpfWugie*HaM;Vv?B7S_pceqQcg54!5{ z^49&zB7U>xPLT>73D~DUvIlL&WPuQ-C`81S928FDB%LIL$}8kk`xShb|OX9wvi+ zC^Ca$N5hih_X>eHpY5nNDXV%{%ndTXiskNt|fieg1 z+CwLQQ|AzNdSP8rE@syws5s6YQXl+tXlJ>oOXR@4Q*!ED1+)?bH_G1++pW7Ac_;}!P*hmS|;#qNat3}{kXY{Uu31k zFKzu@*CXj4jiE9rOyU-csVKu=s;kK4+_VPnugIFbKJvn zz@Y=#;W(i=iVhKx6K9UN@#AkMLtZwBhx$6?M?bzx{=H z>At0FnrI5mG7RS2biWK@uIuA)a z&>Dx4tVZk9M?$vxc)}UKU%u;iD0tLlXxK5p7v)Jet?kc+<61VHl{odRpt=q zI#0u7yI4C}yOD<-Y&(xj;#L;73QhVil*x5MJTD~coWuj+_SE0o-JzH3kSrZ1OyVxr zXzlNGV-1ce#~q{J+E-z~(P7YIQU}Nw$Jq)D3WtUxN#j7nq=5TCO2!dqUfMKzY@8)|vdG;J7A)G~hw^I%t7M16Y&zLq0qlC!|wvafC_S`Ry$9n=!jlF~iuo z@GsKd-X>2~9EY2*N~x)?mAWbWOaFo&OZVngdf|2BL-C5CsDYveUSZNA%mhh{k- zV7L|l^WD4%lXz5%6hE3KURTsa0ag3ukT#2H0p?zRm@y8dp_4bfaeF^B=8u*jQe029QV@TVo+~M-5G}hzsf0a!M2Bxi5#Q$01u7ZR({Dc0N6k$zZG{m zSWdDm6D8T^N!(JICFBX^t15uIT2!|na~+dM8I61>Fa?;zl}CcsXRJBH?g6G5Mh;FOABzF% zF}zW}a?=_mnL76nIUC+L2Xg27ZgsG0XH1Y;ltbkWvQP@d@o3(+RT`1nwQ-BAB;9bA z-0~nl=u{ym9@{L(9yM7iakl)iWwL>J)~=LCm|QbXPMumSrykURlYaU8U)KiNOfKiY zqxGkzH0aHS)d_bwS~oXT4z+HSoq4nlq=VjokMsyp*Q>J~IKza4ayle#yaIH-4z&y~ zi|~c%`5~;XMSoMLePP{6Qr|QVPJ5M9Rg_CrU5iw;Op~trzpWok@jhmiFa8!aP}IQ7 zsR194YV&C$_~(%JfR06g#C3)*K%FUs@D6Du)XoHGcEjQrRVd__PZnAtOAY1Q^(G5D zR=@F1>|Vv$c7`CU%j;0(Wu76aEQj5MW2n~@oI6z8s`iuDgu;hjn?fGThZ$R5o5JBZ zq2ipGXBr^)k%9Sx%Lp4yd4g=J6X1jW-7-{GW7S|yn65Md*V^Gn^>5JdxJ}5HZk$7P z>A=IGl}Q4XfKq61$di2iyBy3Dm&LMq?CSRCgjP)jp0?u+V&ZHUhs1TF&LOUqHgOk2 zrA0hp;UhWAp2YbgF;>lHR9mVC(?58tzF7MgzyA69--YdO%{?Z6cYnuSM=?7u4#x=< zC!pb>VPiLTh;$ey8Yp{QXiy&$@`X-5HViL+Mz}so17)Fn#6R-4 zC9(-wC(b=WE;7BjC;GW`)Bjmw64h!Yaje}lL zrp6JX@P(>`TVw#6B3LaEhdT1GIve!*J9J&dP#Id@P!iR}+=PlJ zpq4?pKts9+XB-a1M36Y^wm+M4tcKm+ap%~jp~0auCjsi6HCmm6RtGMe zI*05GW~5*MnU{kj@Q4f-Mup;e&{-f$_OVbSb z*VosAX0!b8FY76a?0f3*@`>XbLq6j*?d|SRYD=u&9Be23|4^MB^Mz zByrk%v;VLQ8mHz4cnlSXgbq1^m5;-O4T&p4tHD5svI4Go8AJX7T3M7O@C84zhiL*5RPOKk>EutMVdl*Br)65v@BH;R>zT$krpXp{p z(~!K(7Lqq{dkR`oV%N&nI=T;`&g4Up0C@~`Z<1`)IDA=^bMsn-sNU)Nn|zj)^&$&G z!_7uVbwDKXpcX*3EBSl-T3g5TFJJppu(I+p8Q)MD`UN!++C^iiD@5_flJcM~*Fl=C z0I+DgQb z35QV_B<{Qj6$cN8#?yHd8W)G;NE(NV}j$bkZf z8i#*$g3t)@3p?ih)jE+G2d49mlC!yl8ZbL-N*o_4JK=*Z2o5FbdA@=BStXNzEmUxYrn_4@nd zVHHZ5L5N|w;m^;@J1*E;%U*Eba;*!gwLX1on)gh%miUO)igww7CA?nUhhLefc7~<2yjrR!?JwOA ze_twEW=c)tIQ=*i4Q4SEHBi*R%cB9e9}X!+4j!yTY)Q-lMwTPbaO7GR<%J=h!!u#C zjIZ-V;iRPyZWcg0h#UQZuFcZaHeYHFx;WPsM!b+qGwKi4j^pbaI3_lff)z@E56zb# zlYvDfE>#ikROxh61^5Tq(si?vhp{{-Bjc+3Fkw-mA7HYta?*ApBjg<(9FXDu4#{S) zDFOvUFg+lhB#a1UsJ(U_laBEq@XJkvTj7CELZ8YlWFlV#rh4#paj5B$PDyKd4Qgr> zr#xIXLJi~{mzUNNNs?ui;LEaARTa{Jd$IP4GU?&l#7bm9Ie}$i?iQbHL+u=tKgeU{ z$Nq=ad|2vgs$~*(rDL0vhW89yE{7(D)#b&if1Z9(oXQwgIMgMy$@aIu>2a+6+Nj!{ zYt3H5PU0QKVW_jt?sA4Sarkg1DQ6rU>WqZWrpEM|hB!i}b7->_1Jc8{LGcP{7q$@e z84TqG-YiSw02kdpPww2vxLP^$)X5tEi|cNa|GZ_XYyywUrWQGHpUHAyaN66}u8^-^ z^GEr?A{Yb6z(aWvE?+1QcCg%y@&`EPYTP+wv_8Y~`#V=NQC8i*P!{Q3y5)7*N2-s_ zhJ%)6;86MilX-3}vu#Q= zmfvB-gZT>_T#CfyG48Z;rk%@Unwy(pgsV~?T;gpG{!sbyhKZ7?Yn9FizN0f}NdpNX zilL~1q6S_n4Y&bgdBi=q14hWhoghLSv=|PPxY9A5kpU`hr_5zxH2%z_`N<1YJg{Hc z0eo<2_3ctQejh2VYJyF&vNl;fOKPhzErfIi!a|3qDG`2j zHa9?>#pM%Q=G7;zG0r%nHP~2JA+s8~;I!LpuE~jVRc>7Aa1%0=sh6^f8gm@+u|aKu zAQyy-T2Lw*DaoN-=`LoLV1 zNogU`{(LdF0yjC8Rn;=JVOSdL;Ju^0c`YX7q1UVci8IY%5@(zqe$Dgi7UMlGSGldCTOKEGE8ds_?|f>3eD3>8W$ofNHOgK;udA=qX1vES^S0JM17lHD z20daOzFlan&XL|Ikp7!s)Q1_YxE>Z~=~lLhZ@;8tTA)W8X}t*)9!jnX7_$YNvbFAE1k<8kgbp3BrS)$RVwgh zLDy|EPZ2L_;MJgk*49=mci{e3m^i59YwoX>z|p9#4}J|&a$On{AkWo&sLj+fvsK8d zHY=nnZe~V#(e99^1;>M(a$fNTfkxBHK?djL{Ii%P&-S!Q3%7tg^4!Q)r0pzioAI%7 zX;Vu1_3g?E*}t_-R&>-!TSt#ByL8d~y5Kf92%G?}7vt%6%&%1x+T)mscmqq+v2H&gHq&{3!Fw z)&7);PBX{Nl$J_ag4wsq%8c~li?d$cbUQPTYk)N+A#Rrx>gg!@zye%pZLXK`wL{X> zfSV_@R~c^b*ngBS+QOmo(*h9_6qwgOi1#?sTfg@{<zam8XFhA;W0=l_P0`045#I zL;gHSrk+N8Y=*JnxQ9+e+sBTjFxgKu+MHWH_M2LT59`!5$=dFCop{_mypvhyaid@z z8tk=<0Ij&l=8{K99s^M>ravK6YWJ*ciMR&%5&U9 zJ4ISY-D{>CBAtu=Ae|HENaNJ~^#WFO-nChZKd&+koN>k(uQG{)xHL96)Lb}Z5~3~6 zs}+r0&j{#oMz0BOha!U%MI?xr6?s{<7DALVz-eL}lW}M=AZ-c1u3fhBR;j=Tm&)rW zAH)eSUxmOL-uuvrN`s z={><=N`EF(&?z!dnw-mm$0;+O1bGA0W&o~`G)T5M&!7_Fgpff?2rve*8Up_~B0&T+ zAwS!z6xRdJxYyR!%dDmj=^w0-b$xwk`vf1q79zXJ;WF?PxSkq4tK~%$pq5H(mQYn$ zE_+YFdsiqogYUs`NfYyE2>E02U%dLWycg>KCXdxCYB#T(3!JJjsa++Se~kVlo}uif z8eqQ+hmK|73PXRfQDMfFeWeUW=744xrm8Y&Y~U(#7)mv!G?+Y?(IMPDO*2Sbjj=<{ z{svrm09VH}Kywx?P9K8>TH?w_XO9iHGJzwFxRym6mC-m>N-Jh@5q*LOd9qB0Rz6L+ zIjwqe0Z)ffy+fICy(t6r5Xugaa?n^!p4cq2DpyHEH4PVa>&j+64<2WHoS#D*w`OpY zxTohs@^zj>Ls^78bhHPk@_^PM@=@pRv@dBmRmM-6rZeqLO-(w}?z4_AeFUL%y?G%! z7K`(;$~vi@u#fc3`;PQ>wX41(`XW-)Kv4rPz6J`|d^L|K*`W&ICxkp`Y#b6-J`Ry1 zCyt3B&M^FfhXz901_pzQ2Ubj=z3j*CD~+4)lDfT5RMtwqjX#{}<_vf()-%6mV!PB- zWpwuv-F6sT57X`uSNlgiyj)7iLwB7C&~RQGm_}O58$s}Cz>PtDMZfHauYA|n)?&uJ zQrAz@<|mK-tkhTZg#V9{d09o#e^H20QEXV?RgskXE_XB9`k4%y4KlkqD@~0JQdM28H&1-(!7A}!aF}!A1iQHY zrkNL~{Z;-k`m_7bxU<8ek89wVbUZ|H7Ze5?qeECTh%L6N%jD>~1u{?egY!O7`uqCv z0#WGoAzu;XYJdYYgDtPhFv~KnWVsyNumOA5wn{k^%g57rV+jX{f{mlWp`lU3qtUsu zL9XqQAlW3?;jBRL%uI4TK+j9ai&OWq`!LhqDN7npl?jul%lPr*WmKm<6vmEvE8`lwWp>3P$u{pL>o>OPhx?3?{^}1!U=c0~CmblS zLF1XU$;rBKX1@kAxraBe#InH&n9QhDdr!tLqvCimb+B_S=Xy5%BF*)yd7@MZ2wXFTI0Ye&v>`fzYS(8SCIQvfMV00H5*l)`4*slVQnriWJ zokU2-(RP9JlPD^C5nubj`FKKjham@qO+Z>4%L$n zk`AtyUwDT!?{~7k%oWKKNnS*X8YpUDmurBw!tI4a6yo;NA;C;~zbZqG`^s@|{b<5@k7czw$JdakN{&EJ<}?Ki3DwC9$n0z# zdRu$JsrK-8cSzYhtIDbMK#YrL!Y6)_u$}GC37fTRo|j86dXFq!{G2@T#G~?$|NJdH z(cKg6gco`F1>8`c>tA6k5U@Z1kmHD>ei^)LFs`Lh9=-Eda`%njmggRrEx-8Pol?&C ztC-Jq0xn)}hb-d7u0P9&OJO}&VyI7qn-tsK{$xD{!`N`#y=&@V<{$~3SsOfbm@vRA z@P)HFeClinkDTDgW64a93}i95_-G2js1@C=pLBOkXC_~#NvHAL(+C;T^rNL)uJ|`i zAEy+?(&eS8c*oE6j(mLDfEl3XMl2V>3{4sIsTUn&a2}f1jfu7i(x88Lwl$r^3<2tZ z=EW-vcp4rb0Q!+5uT zJWqvDk_*l_CfO;@@&eyHez07aQ5@*&k@}@SmfF3Jm+~sS2Ly7nXCa@@QW7$t9`4lX zxn@8ibll*)f8*NgICv&t~UT=y8-1RwXdpfP>*8-5=88AM?x#&qaO7ix$KbN@7d+=_#@B?Sd z)-C+-A$HyA@0BNR|DL?*+|Nk)UUm&t8LVE=JWCb0oqpu7Cb;3+=MY+CL%t7(&N}+(JNI3!5x)7)mr2}NWd9(Wm^m8JbIPcxKdueEWwXWRN&wY4G(TfMTg!)j3HkzPJ|4~KVh)a z^*0`-t6V(9DZoz-NzRFu;skW;lTN#D`M6z9I|ZS>!d}I8${UJMF6y4CZIQ}xdrH>> z*Gl=xpVmjFBd(VxgJLLZps0bJ*8qp04#O}OqcO>N#JMqIvfDbA>Uc=cw|l`(IzQ%l zfuJ@Ow0^VBb?sTakXpL*MyV{rS5IfoMJjEEi5rSl?#X#($79=Kh4THO8~^ZUx%RrB zMN%{jJ}>)MJ|m09?IS(?HL_vjRxMM@NK{=5VJ=ruLJ01JGNH9e>dO0MhCC*BHk62c z4p8LRH{B|A^>y;CFMn1YbJmS=4Px*1-nCCiTl4>u7OsEit}?iuGI;&-?K%q0nZXVX zqH#esmrFUhumPqk6|zTlijZxj_p%5PcBcI~*>}UQe!8cX3e+B^@CuBf0Z-#1WtEIWAL8{9KWm3gPsn~O( zoOH??<*&EjqGUh+@%QB9Q{N;fp8ST;266tEv;Xh1_IG_~ZO7ivZtu8jcMZ;?4+zGE zZj4z2TjS0Fp*|!azdBZm^?8}HD$KZJ_FbKJz5qf4WSzwE9qvEmLm>jch`BRlL0$-p zbXpfYFMr_W>CBT1qtTC&o+l2<@i;Gr28EAL`9cn7+Vx`N`gJ0Qp?Y5bsBIGU_c&(J zXwSn4$0;{LUOMA6CH#<%5t@%q`#fpy>eiX|^p=m?=UB&#mA!Tl#3{70xj*w@7nb8Z z@^dLW^8Lm2?x9JFps0ZtO#>W%IA}xEc@B0Z9<8Bd9jC#VAu3Hjnw@dpkz<-FkB$lh zA$jvFso8apNYjSDNz+m1gS{=mG&;4IO7F6}9Z~Tie5i)nG0x^b`OHGO@bb@C9{f(5 zI!P}1@Oe^GGc4n?Ps@_-37CMIpnJt{ZtK+9_?!QWWWMm^N}ZqJvZ{aMHCKvur?pI6Iwzx5@a zYT$L{-1GlgUVrKvRnKhOrv5%@>FkgV*nq7%(?Pn+TRtG;E8C@|7Q4t)@k3W7^4@bVlxLoPTvo4Iq2*n5`9H}`w>^TTpA(VHn#Y>6 zGe!iBg!Ht&c4hxp0r0a++CQp*a~)f7w|CsFea$*q=QTs_9og>owKv7R8mFanJmHA) zbJFb0DSHvqj?jS$-l=00Bs*+L!Tq*ll1 z#Hiv^7RV5w>)GKp%40lO6m-v79JFQk3Uk_*HJ+|tMCFU9$t@o}&$dAm#Oq0Mt**!o zT`-kT9crc@E*%U1CY>t|l(Jbz7T3FPn{ta(MGfqr1~~G}*2j>;Ki83GngxUOUyR0M zNl&Uoy7CR_%FpsJCY|4??3XjJ1MRD&aq$nN@u0J$tfnPcNAfodLdd{hzR+oB#63=v zG08L}Tk%!r_x#hP(uoNO2EH)*>B~PV)2B{`5!S-8)Z#*gRHN@$KeK1 zCT6IHhO)Scu)1+poG(tYtful#nUq;66RNtfi%bpH&SQfCEXC>VACP~)>Qec{zkgYt zTeMWm`_}h=qR#ujU3H0M+g3==+9zdA^SRPoQ>#v~(%96j+e1>8emL$O)ooJS-6fN;e5w;|*3n-sJvHSrFkFpyG3|*(8NS_8A^lQ= z_NlKN#PX#o>>AUkU-IUQue`>zwbjT+F8Npa^i>zD)4yT;TKV*q7s@xk_e->4MrYn% z?E1e@|ED5$b^Duk$KLO5@3?dL`XHy1PMk)Ve;T4SggEo%WK4jiWGAH6(82Ep$RYD9 zf$>RqNK0}a34UR1nsxAm!Es&+S#BX#Fqh(hAV+W<|3aQ zeqiQCl8oc11}(>{LkG~owk{EMjDH`DHgm_oEavClw{8pC$9Bqc1(R7=e?taqP(opd$L zkowli#r5v4!XhkcU>9qEX2VPf5ipM&L7paz1jneRy8s-wi3l=rV@=XPAI!LBN^rw@ zE<<;exIAW{bCc9R^);!T{#vQPG8PJ`GKQj)eDcacVibo=S$y63f=jQCoM^JR{NnTF zkOTL{mVnLr!fFU>Zp9CWC#vnpVVcB z@WD9l173lhYQ{@Fmc!K5*1#z&)h|KUVEN1!KYfK<_Ngz(lI2+Jj_~DgejiR`g?!-f z0eQOYXsK$PEY;ODST>0d?;)SpU#|(EcUAgj&eExhxc0g21AQ{hGkDvC_S9^FLs+IELbI+2epM1=dWy;h&OogR|WT9bx7iH%$Hg{cz%IrK_t8A7<*7`raOC#XAavcv8(Tc#}78(zxuX ztdz@-@L|KM8te^^)d1LHvIb5&aXx*?V0qGyz4yq#ZUf6%fIc`8|BwOzfS(~ zy&pt*CiII{f2%OdP=)KMwJ#>Hg8p`0`&(&a^>?>-+>;IKPAZ|}*2zGn9Sfb#2PmSv z3n~xqi(!<~nfEvSnXYrIAfYE}PH3*=R1{sWfNZd{9Ph-`y@?q(~BS?4JcW$OV*0{I2UR$6>)nw>AYN$vG zy=EFFOJ&nW>7Msp$(;BJ7+w1$c+9DY6*ch6(!k9(-z;0UY*C?3Jn=-CK7INtOTba2 zp`GX0cYE)6~S2Cz>;Z>19G{|*S>w7-1x`8M8xlS%b9Y<$;V-j@MgSF+JNPFaLluC z$gx9O55Azummj%)x({1W_74o$G7P>`1b$r7Qd&}{Gxk+*&a07!`G}JbXU>_A?=Z1` z|M?kn-dC<#qve0@%iosv_ns!lzxP8jr2$`K9 zhh}Tz>ttSgCNl%iYp>83c6Ix+O&**7AYMX!yKLC7Hj?7NgI}XF?US%uSyOY1G+?&9 z8jd%0$-tTLN-Qg?s;<=~GMs_WqJ8MN55TGSO))sBRE8%X8Ekk^j$LHPBZIA%OTi}t z9;By=xt9K|?>I*ud*nfR`YF44_`<(kAxF+VUiO;3pE~cZ6BQtI(-*7%r~Qmw*8U^- zJzA$R@H3_ycOR^Tj@!a^Z)xwvejY>GO&Sr1MC_79zLhqhGVb{nT{pUWS(lqB+ghdU! zax~D^)+U=bZ&ragJNRk~Zf_mZ#3jGTtT)yiw>n5sJ4NCMbu>7dF zQ>lC$C)AMAi}P`qZqKwHoz7vJK3)Efv_ISA<{N)0S6z0#^kT-`!zrhqCGY>>N2IxB z9Gv!6sSh_bblQ2nVBmWNe4;@}y$)tE(~fVRWC1zT&X<{a<5-SuswrDJ+Q(n|^<8^7 zlMc5YPg$xko8N%zTDM$;B{TnW$$8R)B{?>g4N9L(uQ8sDGqeuXqA_ zn#0lbzuoPhC~GXCW5jW{K@wX-jsde6gMA$%w0Fv4r~&f00$I`{Ag>3e^EfX;gw$zd z%|^0dqse1jcj7lspoEYG*uG5l=a!2++%PzF3C(GrFCE>zvZD2kGGP*CKJYcv?cP1c z8KxoYjX*nSD!^tFI&N$d@E{_>{ z{_oPb?59#U`$W9was-lH4rTxz_{O-WD<`W$KpcEj$u{U9SySCWkn@d?sVQ+uNdjZFMyPR2QJ9-0G0?ZVPqG;Ly4vo-}msJ@`&NdJZ;`~3CEoSfWbQpohLkF(vK=_U3k}6ZTnYTz1dEKxFD2= ziwF&X8|<{OY7mA7q?Z#4<_kH+sd~wsBfbfR@(^q^%X6!Kqj(a2Q zS~X_NISGIaDiVJ1PYHFY1!mzXrq*54108_t2dIUJM>qvj@~DR0B)`7(&&t%&=jGk+ zJ72E<@sFglqg|c(PhS2ZECH;Q)6RTbbOYf!HBPQO3P4#{HrvWlQ`IH~MOH|;dHAfq z>KZpQGT196<<($>edD?82iFhgVx4#}kJry}LWXpCjP^JqKmbm&e5QVIC_luD?z8zzL82bZ0#>GzDoL`Lu&; z-?_}@+;`7kHSe|9d;jQTPm$Lgc}#S@@wyT+Y-{}kIdZT!sDeU%)AdOD z9j*RpdD8SIC(F>{#_Z3SaNNC-)!?$hag^4KlliR|OJ`?0K0V#1gIg04osgX-j$mry+Teov{`4w`N%eBb|J8%Q5e8F zz&(2(kg03~oYUYi#1KyVDs$RbPdrP;wN8+hmKJ>rpUYEriPLT+LITRbi-bm4w_qG1 z7)R06yg@jaq{Y|NY?sET?HAI|R6ns()?)UeMTqW5RLw=45}M;i7JlWjZWyBq7}6|kH>N0pZrGMRs~*Eld%w_W9}k1d z?H(I&f!p~t?P(qIGLCj((=Z8T>l2-DWk@!cUo12Z;&PXnj%RO^wx@5C<^xXG3$M=% zrAsNE7d24S!1il^CY!-w8mB|bG-C_i?{xNL8yly1hCV7EL2XZ=|z?^>Zx6H2kDrDELPQaR%o$u#XDWwqmU zsU*JsB-!pQlHIyode%KD{que+1ND1L{~m9V;nw}58D=sMqNoJ5PJ!A@^q|eZ-L|7gpVY_X~%kYJ_SIHmDco}Uw_t_LEByRW9Ml{ zUF8Kap#}mvz2TJfERW7Rx1F3kFikG{=TFLieCFe-^X1DHYdih>7q8XHE8X@QEJo&s z6!3OO-f=<(%Iz|T7?UT`vh{lxaF^9z`>vHZ7JtWx<8GtEI*mO}<6GS2aNxNPG5&g| zb+0gnwxK_el0DdSIBvZ<_spm~j?5i5YNp$Oj7>|Lg1JTxK7isul`_6aa6_P6QsSph zM=mHzN$1FwH}`Dx`j=IAN<(vpOqw)7m$cByF66lLliu9vBD1s?`y#KAavE?v#^u3B z$~~(em(HHpY3GZm+O>d;duyvJY~*5=pFJW4vA z`9JB#MZSL0UMinoESbL;iW(?tU`I5-kr$6ej!6wZAPEstrxxJ}N`a_AtI>KA+fh2z zL%4>NtoWnUJ%5u_Pd-R$555R1F)I?>od@#;rYPBGH$5f^{f1!(Gx(1_@w8m>uh&G$ zd(GNIK7QeOaN3*UwAbtM74A-A!3#)RU#12fD52v9O|GPTE_2Maf74=VTX2IsJbb)V zO+8#DG&Es08T*Z6k9XSEE3UdmzWDhou>%cuufa>27k=<<^8KIOEQh`JXnpxo?V5`R zsBMBLmAWGe#29X0A!QrqNoMn7QrW&thHEBaLIz88vKytWalF**d90ROT3$(Ws=Cx> zC0X`&(zDn5q`YyOZl|c0I>V7PXzOIws~pemgg$uU0q=vxyKmGdQ@trY$kO!r>ACWxOU$Q_DN;oj zJk8IaTi#1u{|dZ3mBE|wya;fs#E~3#I_Uh!1tz1UB-^V?VydPb!p4<8d<29}J6}ZQ z@)*vvlix_Z3MyuMB12;*+foe~`;@M=G-i=;Gb^mJYX7UlO~s@Gr2D~ZC3DJWr5eW8 zXtzg*B5hFvMGYi1Ks)bt-yzdEEOo5X_-Qbk=T?1mn}>|#jXyf=B^wt=&Eg;Hi-8TV zd9Re$Pj<#yz7O|MD$;_%P)EnD23UraNGz1W>(a(eTjhNhT#nsJ`jl^dU9DVw`Ng_d zdtF_PZrMnCXyqjMfss4^%t)K!*FI&GSEA=FEMMP-MY8qr-^#q~(U`%;OZ9kCtuI`{ zLF|R2&MgcNz}APCT=6;i+7~~iv+iBk&E&)Hf2&;o^FPS}hrC8*4$PqGS%5*K!Uw-wn2^qX(m)FuM zM>0LWkzgK6A!hYv-d$aP)_E+4G2ys-1GBBQ17bjNP-XOP7?EMHJKeU!{B9k0^?WW~ zzW2Fw?sSgrrH{+jT7q3JhupJ#>U8;|pQme(E+LmgIyIe}#;z2`hHEwbC^SQ^yEVX& z^08&@oKx_k)N|5{_3VR~%I?n&>z0obVUYM?B`%M7v1Zz}cIZ3yA}aKW3h4!edS@R~ zljuemrY^Y-KJ-D-S5Dj?>)qE$`y)S*^10_1*SqI-o8nnf13RpN(QFQFq*+T+noW%3 ztXNC8vCT^bpU@6=tdXk4SQgW@MC$iGNh&eZp1$FxJ&1!ZODXKa&|PEc>bB|is6<~d zJpYnU%CZ%!lnwb`@zL{TAI!Mx){k)9`N1TvYqnr@Af7*S>v4fn?UvQa=~-+%(@Wt^ z=DrQj!D+uy9vq<4elWJuoq%_cTGc6J9qAx?RS!Sn7_52!wEWjsJ`4MtmG<_n^1*Z7 zC_nnu?J|2GtUU+U;qG;kS$mIEY`9lS2fL(d;vA{i|1>-~oh}1iTcqpRTV=Rwz0}Wn zqf}uzq1spEV;OM`DJ&v9#_=GCI2r4Gw)tKcYa&n{tB zp3A`6SbrSJF;3Gnr}rI`jPN&#PXqtXb-&YV zEo!R$*ZBLGFpgd(&sZ9zG{_jo?}vlKimZ_Lm0#gK9^ z-Q)?E5N?c6=4 zkn>L-yDF%W)pj@idjsgjBlI++Xtb7@xaL^45~-bmi{jcxqTVjC@9Cy4L=xLXd=YB2?^FIZLtxm=rbD>mCIVfbIC9pqp z!I;|*@+59$x@{m#;!H;r?Aw6+$Jf6jx7~5C34q^o{_!nx!qG>-X~z3HT-UDOpx`Ck zc^FLdvogJGlZQ5z`P|lOD&j2FiX_h6W9YO$kUdt0S`U(OAH&pqZOx#(k`jO^}~%^T&scb_gx?z&DY=6zb~AHG@|29`?m{-;RmaUYY~ zeNM$vnCa57;t}bX_XDY@pCGNTzXDGCYasz&JXKL6s6tXi#(Qu)(1z-{DaOa|Cdv5Y zFP6H>tkli_H~k;cZlYZMZ{Lukj(t5QhwwfY zzJ|)#_DXydiO#a;H%`V2)?fdUal}(%YspEejGn(Eg)|4!S)4AJ!6aBUcKK<9eEaTq zpuGE>3nE5$-SH>+`H#PgzJ?d7HH*obls}(4`?r`o-0ES{RlhF+%K7$ zgLL;8&f4=a@{1e1?VxLI7WOG?A6_-I4^aRbFF)ZLcWoHb;^K$o66`!tJL?$fdhAEi zH*p^+X`J{H$o}GLrpUdhfmfOacEskJs*p(*!ARw9cuYX3iK5?_$+mA_wrhQ}_9QL|8d>NJN zw>itM6F#)VczTU(C~Gjg%Nq`UDRwTL+Md21`N6ecQQd9auulHzk}KqvAN-r_^V)Y{ zA{-x{)AF%HSZ}vS9NdMlQfEMWl}_|$+;6edlY zrp`Lov)AIIN6}7YU=SozdTQ8Ycga1~YG&68(}b1axTR~o5*&?}#TtF?Z<0H&H+0@B zu$+m`J71Q4(^>D7`48PIPb_#?Gk)cot7Pucuh*v`b`wY4@ml55#89ufki{K!>BF|9 zJ@^0@mN;>llMHcpKK3E-D67QRrLaY3MWYN?j~A(I!F8HTx_GnawP3$9+#t)^x%!*5 zyZ`T)aNOe|!|p+2P*IAZ zQxS9W+-Z8F&|zj(bG5uo^>;ZO}!9QqTfx6->F!Ky|=0=va@t_;54fJ4US}l2m6qggXatm9cOuq zmadQwVlQx;m=KvfalBmlvGb*=u|Zn!@wj?8VY*uhoFETc9=N$oBgqq`2imB7nkLE~ zd}^kDsHCJy~Bu)<#X%Y95*Nr$pbOD}*TzSd4!tFcxrPnJjJx_k}o4=v$YIP-m>YB_D z^rjX1z@uO(-TItlI+sX!=L#w5T@PKu5wC8M(ux`!wde!v7dBBgl%>aO^XIN-E3)VXWw0b)9p+R`IJ6fZ4g^E{HvL zzm}HQyj#~FlDZg*8hAN0u*V*I=({2;n;$NFwFS5D4k>>UAE`ccSop$a`KsHbVZ{wn zjSr^P9(q28I=<2j=X&eP)v|f(o}xjd(` z3hx5Zw&*IEU~7~*#=VXXV>@J}G2Y{HvVukb=B1cvze(=t@8_z3002M$NklgIbY_b+3{b3QaxiXmdl(Bd1xcU69Cjtp9+MD3zb*nLLJ>? z2B_DT;GDA{o%tm_W`G%Z4e&Ug*Wm_iIEyx0@FnRyD(c zO`nPV-9Ikh{OaeF^vTB`k#GOkf5@fSgWmK6&0|-d(hZMD<;DlF9p_VUva6(`bx&-? zIYr9%I#kNAsX|#rUEB{SmUSpAp6#aBMr)wqeryxkzFPV>EtB4rzm~ygzAb&@j*@}t zXGm%N)ab@3DSIK@>*}x&%~*&V6OOyNPiC-vAQfs{PaBEeJBvf~S{P#f7&q_qK8Q)@ zJW1jC946z8!wdg9uaV?S=ib%yF1N#%y8g-fafTyX#u;@;L+97+1mc?xe~+|1@-x}; z_peFI374t!p3J*zakp0Hu3Ot9jsiu&aBUCDirFfq)98>d&~k>PZuargjz`qplMj~q z>3FFM7lvXeYT)J2z;VYN_i_}w(`9n3>3B=o;56eT4b#-N!JcP3*GuiQ-;s*!R%w7e zDZ@5{?W@+v^AF6I^^ZRTLO4Wt;jaWQF2ZY-44hZyW;~Yz3}I{!Y}zQBm#vV^*v>Hv zr<9KTlp_z7sfQmV71$ky!#`X)kPSBqaDM;!lh?=-&n%=U8ZP<)_!;y z)pjc}M-fFS4-%S%iJ|4r{dJlbqG!F^9=k#A?>#~W>9k|bI==u~g`K&|;r#L4EY}eo zcIn$HrK>PoyY3F84NDblR^^Hl8qRJzRMLl<5lM;&*PI_u4RDHk`eoWbTbs2od%^!np+ zqXh3$Wa2VAOvXipya3|72tBXI70TvmNqV*!x}Vl%xE9~8!7f0@op2hys(Qb0ojb!1 zzx!1={>0Pe@BptsP`~}AO3j+vB(w2BDQlRD?M)Aq$^%b|mxYBw)A|qMMIBIeqBl#X z@e<6+SG3NC<3C&K_B{pTYoqk6cu;!ge@1#I947;Ny%RSA)plJBWe^mqkJl|hHs;VV z;kb`9_}SWp(za`mMV27uRfy*{^{mMI3?^1Bvsxg{?s7R)`D0U*SA^3!+|~3!sHj}8 zx5LrOi!ug;`3fCrI-b*Lmen+C5?x=;bg3}`UwF1}msoJr4{v3gKf;Z9>5ln&DTg(U31ZS zcLfz;Q3Knc0ko_d1Lv^R&Zqw10j+70AkK`yrzs`XUM6Y+?V zNXQUHI*+re3=ZJ?7?1u&9_u|o`Wp|B@#7|8c?{OIV~g4hmcghKg>rS@`i=9YV$I(q zgW1>&mJU_#e>%PfJp(*VwkkOF!}Z*D8ICRd`~UaOv#{p9Pkw#lty=yQ&n=Vlzg;bV zICZ8}K@QG;AuB{8!T_kKou?f|)>|)^Mx^vc8R&y<+%EfUyU)P01QyqL*t;Zq=Qm`@ zydTQHeB)Pg-`#&@k_Ik?`RB_&EBhaCsBZVjmvTANp27NjzJ$sN6>A5IZRIj!+$P57 z1%F?nYd-bBn+(iiX9MxWWzn1r5bD8X!?rbm7#&@F!t}M&Pw@1rP2n+ zd?ZJk?Noc(c=eD# zqn9MZdUuva$Egf!exx>g*GG^fKoA;=E^7CJJZZhZpXYWe@ zD9&!kq-jk}TPfO#%rU5(vbB z4TLygE&~oY*x1I#hipr>eCf1qX{5RDo}QWgzN%M$|J^fVNw($9YUa)KU+;gfs$RW% z$6x>Z-?4U$yyrK6O`0)1rU@hUxojF!RJhpCBsP1`j8`S1EE;8k@EYZfe%!F%tTd!7 z%;-gicithL#YX9_zW@i|VQNlGE2c*^=t#RhGy8T)^3d&)J#se&7jU2*cGJn8jl-re znySgi(x|OXVAq>E7>ksGi+d;KUDy7K4E7$8n{Km(k@szWOy2O;cglCa{$(t>#b_)0 zKG&%UY=VOW(`SwhG64V88Pq`=2F`#V(OSGB;gY~2%A;*p%G=)ZcG+>T--Bi4s?%{2 z>L1Ip<*RTKs_7w*?H*~gvmc(dFLs$cnd*XwQZSE}3A~;!C1*J~UX>nHKOW{d#|ioD zdHAoeVEE&b#6I|(3Z~BU>i1;me#4lP*STf5qriEP z0vBr($A%?8)+NmsU8@COS?y-13)KkP!hq-X3tmSdwbw7NxYO!iASI1DCk_F}#s{J! zfdLz(b^;88bAP$9R0`~*0X>pEl1I73E>-erzC8?RmE51Z{MV)nnwaG$_FzJ%!Utpl zhj;XCx?Xzk_$(Gbec+_bdFMk}RS}dB_p4k~BPxQvG=z$w<>gxQLQq&%YIgZ0(*MZ! zrT@{Jr1=~ys0v1XtW@T`^BS1fz*A|!ItJ@2ftT?17dmFPHnzc@BT~Tj zvOW8|<&o>J6QRLAG@vaNh#{rQ-mWSMvJ3rjKo8W`OXA%e&!8HNzE6w{$+jQzF!GQ{tZM>m_sE8zqVD6sw_nkL^ENJ#pWJpg(d70!?~z~q&EJ-9VtNXL>)!Pl0dy`Q1qWJ<{D%Hrg+|_;TH&y@d;MRTtdqF{7uE@{5PM zptxzzS!GP#j&`i)a)fFGRafOtc2q7LleIMU)^x66c3+*(K%j)wE$dz_*I(cZ^h9yD zT$;X1UjNeb<%XX=pbY-yTi=(LU-c4s(`#R;U$rCY$XF{;EFFYtu*UEyNDL1#yxFhJ zPoRKY4OlQ-y7#Lxw)0z3Ivs1fxBwdAv*rJ|lbKzB|F2`_gyHUOiM3q!;w=_s5FKX= zhDVdw$%D=sghAe0czWnenZbECuYq|D%%%Z0 zDgLxk#^DV9;^=_nAO2sGP1Q;RzS8a8_=xQM_gkbE^RsKQy`dQpXpii3;jC{gPm^Gy z4lUZ^tyPCGqx1gM=oy9|_RpR;dPF{Pka(p`Da>Qf@Be{}U-)Wy?}ofQy!#Ji>-JsB@pnJ^NqPQ-=gW%avo9L8 zMZ_$ppl>jq_63A#hBl+=)9>=AAG9H!^Y9zb!O6CF{H=6%Tp_j1OQTVD${abraTrC+ zEzg`V+$%d=9J!3eL`5vnu(G#`r&G09uVU?<^QOIDu+a<{XraPr#{$211XgWAq~6e9 zZi3VJBR1an$;kwA9$v}GgU-VP^acR?!^D} zLk@;4w;JR@c`5lpsvlb@Eq{`>2qD9%}b-`id^<81CMQ zuE`ISHEGh&!ZKuklx_;BcmDaVXRr&ufaI>`U@XIl?kKa z4gSDTUkn%=1T<_Kf99N$kZzix9xQQRJYwwQ@>-|v4MJt{Jd>gOPm%niTad!JZ_;fR zdv7mKn|a2@I|_h0WI1S=XaT!Hu8~?cRH>GERDi(Y)E5*?MH#3eZ(%Gmvg68r&Uf#< z@1LaUl6R;PP^K}?^VfL|%xj>m0rvU)vHdt?thXr|-ZWy;kN&-6VdvGK@k-hKi%n{@ zC;NKPwX^Y+`5CSC#sk>*d;5~-%fa?D6#;w7$aoD7oFPG=tDP)L-4yn+MjLPG-6L!E z-z^aY*^4H=V2te4ktnvFw&mI$*4Zv#N{S*7xUV40s;QK zU}vQMfK=6;hxy6i#J2}`7~n{EoTYcBx3~(Z4C0znOsC}+1C3Kw#*c$ulQi1L#-+IR zM^adQiDZ|Z3&RkH^x&n1diunN-X|ab)4!7YH$ALrUVp=llE>jrfBTnzVq0pWFC>p~ zEZn_+*Qoq_>xdeOHTV)aR;rN+WQs3U>Bm=;Yv27r`O!~q)iinG)TEDn=)H2$`R7XQ{@Wxud=K`-d=XxlI&_bA;#jWetMrEo z2DOCBGE+ernbC#G8ZeBv#ySRCDiI`$@P{E!#B8`8K~Fi!BcIC*;gk9Z?n;$Guud`i@BJ z&_OD@6#!a8=>!?7Ui4<9~5 zhqpvCBX@PJE(+lVPu_QS|Qkp1ss-~)}z>dz=~t^Eb6+(B{I6} zHYp`>N?)^mvYt2IO@;HvYlpaO$5Xp+minH1q!BOl zM|bX%ZP@W7InalBz-Vxx{jpc&k3l2YxB9`gua?JF|B`g$FqzRrUWy5v7Me)GY{Wq~ zSZF?(NCFo|B@FpuO;*MwCBxV$<8a&QvJZxSp?|-m#|C9*U%&kE-S@-ju|Bv1i!{dRW zW?;$W?fFPPK+yCJ5}hv%G}?!E|3Z?9qBNZS8XdK+g*C@72;2+4wysv5ciuVj@FR~) z=aDXw$%79+BE5Zma@7?tPy_A3UBmLFTl?kFgA>v4aTxkT7?Cf@@>c9U4;lCD+b^$t({IR~cW>0Rcv-o8sr=a|KO!5>Is*%G zTcu;kDoOW0fxXMO$au#KC6%@f4XD4k4kHG~@K7~$g!;oJg7h#vK)ZYDYg`8US=

XD`()GJaXCDQ7pJL=mP2U@^}V_rrch23DH)xtm4h(w@7_Kn-Gk$@vb6?l zHR8{>=@QqZ>t$sBCK;+-Ac_1!dufQFdHxx6-(&ma;?wK3o$%z^XFLcOOXokkcWxQ( zsO5ZvFQGAy4jh%%K^X2UFV!;gwnNAtJ04VPg+wTa@y_E7@@meZVU)udUiow0goYhY zuk3%i@yANfWGi1y3d?6Xucu)CX}n{;xEt?vOP+_$;rNaB>4rNx81#PHd4?WXM;6pi zO`-(pXD1n;hJ^>RhiN%04Pp_e_ezbPm9&sCS>G-rk9=1W9p~eu!GiKr`f@SO`{p$; zCp9pu;m&T?IyBP@{2UD~_3x9$t$!y?=e`av;x)4QE8mowei-dKJ*G^gMm{JHw5*Z4 zH@rpqatmd&rXC9{IUR$HkL5TPTr1-V81i*FIk@mF8ODhEU)=aj>Bh7Kx->>E-+eAl z6UC{dO)zXzn3^%t+b;{wJk5-&8Iz!lbpBj%zXn=ggE;mN>{gf$`Uy$nevhXWPCubK z_Y-P15l+9RXDG8>nZ^X>?++Z^BZ-5*kR~kljQoS*PA=Ed$2AO}}N04zL%;_1YVSseXj(7pV_ys;hhO3YVc#flSsOz+OctUOS z6Q7aBGp~_!+Zv_gMfZ3oL|O1F)ETF*k$?Z`Z8~L!U!?Bdh(onbyGm|;v?%?>l#EPp z^G+_%b&1JkHkS|@_C=J%(SefOvtvY-H`hp04)n-SMLxm8^FBG!C&RsmWfCW>+PVU( z!%W5VHUU{TK0bF1ch<-`<3c^RhU2+LrRL`yol+7Y{rptwFz#oVhdX3^BG4G*?8gvB z$P|eaApK0U4C8*ot&TU^z!!Wp?EvW~FNgGdT7>Gy+y*Q+4Y;-tGq4G*`VvsL+tXu4e{DJ5kymPV#Chh_vfjg}Z;l?|8yCpRo z5#Ul-NX=<8L-+o@jEK$k{BMA8icbwX?w&)r_%uciG>%+ruDCv;Y2~kFv>o~)0Kkr z_GfgKnuj@#K^QTTmSHK#-jy$u3x4erYAiAC&FfanDs1^!&(keo(AEtN$^fR&9DZV_ z_DjmoEjPlrKO~|?H~fU!hak5Ot#@MnT^sCD%Cg4&ft~hc^fRZPTZbARWbpKeXH7aH(`<5@W}l$2ZcW7-Q26ms{PYp#<2@v#reGNiqry$yzY zGuAn9_ntb*uYRFqlarFf$)P2TQgg6|+H=*%A^8PWDC@XP^fS((<2h6UzafnKHFEmJ z%YZQMpN?rEz8Ob$=hr0LiaqmjK$g{S%4I(eTxBz9X)H9zDwM&8f9D-a%&!gSyz+zc z(0;+o6TU3ri-x=Ed@|g_9H$IxWN17iy`w4l(rp9sz|J8%X%`5d56VJ6VXAeN{{Ug+$BEU~b{1{X|z;oz4Dj*Wl zKSCzTy8z>91+0h{c{*Q*5ud<=d}xF_G=tp2Tz*fBQ2jB80F})l@8U8k59XKQv3$E8 zTrP*kt?C-mh|&-8M)JkdB&G8rjQb<5Rs5c5{xL@;81G}x%y?HlnVP6YIzGUtTk5MK zjCTQF;|;mo0aL<5~e3ZVD_($A35nu4rYf2q9oxBgUKymX;l)UiPFFxs;i zeNIA1u*9hEfb96ePh}J*v(gaOdEtxzFXbEh7&n|*Hr8(-4aFfEaY;jlAa+PUokR6n z*sT136EA=F6M8!8$BPc9$Z)zt5)0|m7-(RwffPadnNA9MZomn$+&yR6l7;f;pZut- zUURzq`bVyl6c+prW0c*bGCjJU9lD+!D&LS6ark4VOPsnfnU~kU_itq3iZkWCZ@(6E zx37i4-ilv~G{a!cVY|m9rtmPmx|UU1hN%VXrGD4fk)IK?0UXR_*{Fo77gMdP*P*H$ zHx5Zp{tnGgIT>bpG+gSZOc$poZ!c&i{~WfwU;_KOa0K}I9TbLh_C90 zX@HvN18}~2KAVxILcLu1;>+cltFM$_|L{LZ>yq^-2kfyQQgXdg=kyc0{vA?4#yM0! z?(r(52K+ZS4$G!pgL-a)PFV~1)B$lhEwpar0OH3~@3ZA!MVh0R@~iAAtN<1~9V>HG z2`ng7NXD_TpxO`#&MVZ>!3jCoiv=?J(rs~y;EZNsPK4@sC5m0*!o-3jZ#8{2esz3A z7(CVZ)$!Hz$HG4bJyo1fCuBtnMpLlNzj`%zHF>qZUGEOHEHKhRNLP)2qWblcbEy0= zZH6Q7eGh$GjrT?z(v-pB6el!S4tgukUk`G!(}LXmZ0~a#xuBhnv9O8 z&wGy_t7(uW>z*&`QTgfU-q4k@U;EO>{-c=|mJ_j|n2(!d>o+{vSxrGUBA-hWe2 z^@nsw#c2>6c~1!Ri*Rns_^ZG7zomZ?Up6LTv;*nc()T3#`jX`2&F}xb{LO%8 zf|F43;*h2c(7We|31?70ibwas*u*nv|1C1H>}p#m%<^F#C>*sB@)hY#Ly!NWY|88+ z)%MQ`%g4(kQI|E@otGy<3gnA=$mjFW$*8>O zjUSQ&BLx|mw7vC@l}{_HNG;Rz0eLk!Jimr9_34`%N2RU4MwYhWuqD(3I?qxKtK|GIwg)&? zpwKk*-tK}o>eP522qNe>ap*YlF!x&=A10nAdAgq_p*e@;JIATScfJmt&LP9&e#2u( znP1sIQ~I3EIW;vdSD#TMFInz8|%R(io!gpGQL}0x0 zPy$>$ExoVIiA5m{9eUxM*Gd1w-;q8X(j-sAc-IF$^9pAVD)=LL+@Ovb?M#;F0h-vs zyjN1=6=g?n8KFtwq{2Mr9bO>=o4z8$%_}6|+@a}44@pCtzs_r5&T4>d$J%%qaxi9m zbWmFQw@7N;wQ?Av?PJ}&*g*teu40;+0O1I}Ydw7Sn*g(zs=-Lx8j?QeOTRTrB<771 zzIeR#W1(*%I$qwjzW^MC!J0VZO3b(4g%L*E5|W1l0tr9h4fM-_pWi1P8_s}{i@n6T znLy}gG!gpaiIKlM_7#i^eMaPWe5XBLekPKMrfD#M&LMo%FQyxf9>551(-Ns|SP;dR zpJRs4zKXi3O=8Uv7Wd{$8QC@1D8pEMI#S}4jAvip*aroAOv}Uli)F+5lp5_RobIY? zid2>1pbQ98ZCA87#|3)5)DjX#&% zZ}q|%#Gn^xC^t>U!^qMTayUIek#tH}5oZwW`w!v5&Fqr%tS$sPE00hYLBaB%9@43# z930Ndq0xj4m9SN0tYw5OBaDs$l35Uxcaco=TUxY6+Ryt zT84mhUY=3f8d_F@&fiyE5kDaX?0V4u@ORaCPvDTIG(PkC=$Detda=BzT9iG@JEX(T z5D^v}^uU`iXoKD#xHi&5_F6~$!HyVJn=$~ACKJ*^7(eHxd-u!GeP5E~i$AQgQr^n^ z&MNeL#JmPhZ4I!g@W=KR-~^`q)O6wm(1wLFi5*`K-v1Cf`{4+Q?T>~A=LK(HeT8oE zIH{xDq4`Cap-tmn`z+NDCbQRQ8EB6k^x*1nNP$&_ z6r9fW=a7i>JHfOJDguxKg6{kVf7k&|NpV}ju8gf@)UBRXGukg5T|bcm_UC84R?a~# z)jh*v<<0za>&^doA_2piFHa}09LBNpPv6Ld-1Wqeyl6dln}I1xwS;Wf<(efq(uMcj zpr8;QjtvMH=yl*N%i81Un6f z+7;pwL3-e{?=WxVXBdA<8zB934%M$A2xB;YN)w>-h(XHB7o(Bxy*h|c*R+ht(NFtw z>IO8Ges0e^E_ZChd}io^yM^#Y=Va-dX%7`^<<_lCgL<~-lgWj$3G(q-mBJRD6a9NI zPk)0Hy6?2bl9W&LZCRt2+0-EVSu_E9-HRYm85dyOZ=oR~jzD~6PRJr+0tVgac1SWh z%Ez*(+jq018W%~bWu=D4FZL`ZF7s^p=3>zMc8p4K0t=~8)lPOf435>xZQC%lYIMx% zJ2Ms>zbe;`_hEORQ14X_mHBtvxd`XPh&%gaI_-bx=g;d%XgwBZI}U6)$T!YZH40f6 zog&+9)OmWRN8a>@G&#I_N_rM?52j_a4xN=xK~f*0O&=Ji9?+h3>F2vzktj2l(nVP51+E zQx4^@=;;hRJir<05f>%HgOc!o>)sAS_Z*Wk$Xe!P`QkmrBFhQPLmI_m)(7`a$?$l2 z4u8bBOl-bqLEjX!2OnDY937U`3+weoLv>muZ>nX5BzN8`$)-yswd-~%_3V_1)OySn z$DDZ23iD5Yq{k{hBee{Sz-m?0i))^omEKfcTu;gZak|4>fQm|D*9pC-=Ujk2I@l+R zdTx{Y3x5sChdL7=>8klMjB5;jv1>wTP?s|l&ls_S^E979z2_9Pal9lvvTsx_T$946 zJ2PY2sKt&psi_0l&4w>ld|9$|J^z|#mt_vZIbpcFArn<#6tq`I8bfrfE+wqM=1v_$ z<26L-|LHW<>S8stmxO7#$2Th4LIQ51xAqEypq)%QvAqC5Jy`+-V8vsyoPOaw@$d2K*om z)-x2&el?8u8)3X3&k3t#Mk=r76JZtkb(tU)^MOw)S|>HwB~>wE;dW0I88sp4O=hdp z*b^CZ0rRUam!Sv$QAV5pJLYh<=%O#L^MT7enAgCZ(g5o}f8OVF`pd*l4O z-Q4?ISK}}zYy}B9D3i5yAi}zt!msbZ5y`Jv4#Nz`JaOF+MGk!d`KZ$Xzo~*A=aA8) zuW-`>%CSxt!4L%!>X=^x#;`Aas(2Lpgxldsu76^>-}U5-`LeYAFh*R#r-W%5hIexH z&1WP>l{e!sp%v}eWxv**Jt|XB0dbOrrBXZkL)Znt!9Ms}vEIR##3hm$9E66w^wC&6k1nz1V)m>! z+9?g^gy9}F8WgtrS?HnQqd9{NK<{DOK4HO1D0c7o3F-8Ti`_BA9u5&huVoQFc!AV`GyPe7#HxXJJ_e2f6Bwopfrv*<~kayOpjVMNnK)x7Q+tI9I?vS zASYCxnrY^B{uzou3jt1wARCb*1FXR^hByNk68b^xXbe()^+iN(x7D&phWn z&}fPv$F0kvjh-Zc} zMk>)!jbc&d_?UGT_Vp9(GB&d4VX9ZOKgHB`%XSUtVanaYqJGHO21Gt`f^meY4c*uV zFVlyqCMpvd85I}(4;=24(#j=j+%UMhU#{-ANga33I=*KpKz{Wd8yl8Pu@|RuE>g)g z%^(DIrd?mqBmH~{;bDl|4`YhWxP6gy-JWcH^MI%ASPx7iw+`EI;!{Q_C@&y9qVp{j zQpSdRC4;Fyg?(G38`DX+?|B`*BAd>UK`oDvQR;&%f?gO}<`|0$f!4!Y4J0nHJ{>FbQXP>1$ z(~rRj=g;QU(Qs#HZXGH+RaO~?^s_=okbcgQiy-`p6QKH;Fx{jK4Po>dhV1ZI32UN? z@{3LT<&XaEA$g#W4X9RLcv<6gdJ8FEwGty0LKWUBHiTfyst7Y23$uA_L0;mi+|%Xt zyb{L!nlH@{XdOoRp3aX-Wyj12mm z!}ydTS(AsuxSwJ6Lmt#m2zfF3)o|u3G3R|mjrZ9PX|kS7b;5LTtF|>S7KK1_c3Tc7 zP26Z#0hHKC)iBr0klv}w6q@gVpFi!TQq#9n`nTSSI*fUN)cL%d*T7uWfVO#LK0vmI zn$e?DlWo*Ksn|bgeW2>crXEhhXc@wM^8~)UT4%sE#J(%QxSwJ8seu!q`n69)9NQoL z#0yaSVD}T^M*#`3Y*zYncws=l%ohuOAt?C|^YMo|yYy?Y?n94wwtf1&|8oO}P=BG{ zWgpBR@f^l!ogZoFcY4J~nDh6vX2NG!5eD%1P`6AaaX=ONPUVI02p@XTFhG5_FmKTF zIQpvneH`Uw=SzLy=P>SP7=F)|zB!k^7t0&Qu=sZZa!@vwkITz+=$CY(l!kin!q+Ob z!v}#ss`JM+FTinq%?hi^LG#Wqq-B2SoPMOGKk}qM@TRd$KcV_n2WFF0@B&W`loc^w zJhA_FsZG~Q5<8CRmnQOJeF>0$<~4$RIlbwqkJJ|Gdql2bofG7Qe%f~()8 zohXpvHlS(6*OruckQhTGRlL&qw` zG2>qm=P&abm^&I^v*C{}_-jkOFlh4V@?aRjP~ieT>yQa(Kwxks)d)k$`Ua&9?bY#= zmo-6+Hk`6|n$MfALoy&w7cTh6F>teaEGgIDyGd8WaE(yZ?lFC@_TC4i{yRU^5%>RW zZJJr&SB>$JAxTaQNZps7n5HJraij)#_^hR|x%L%*AQQFCrWP{OPBUP#?CABfgg5FA z>92Nytc)|WzWLmwUH;bh{-Go$v27y%xGRLy2fn3&19++WDGrk=$OOKIjOO2rHCvd* zf%5s!R}RcmNm7H`r4C;UYSQ;b`Eix%BL;bMm1=fb4ygO+Crde5?tp5t+Ub=l9M`?+Dbdx3hRBn*Bvd^H}OwHmJ;N$3_b{ireO z0--Y?4WpE{szD$)k%$@YI)XrC(5UeXfQp)SUUl(%1A58`6u?E)y7k zTGRwUw(nXzF^A!0QdhT>h2RxeEEyt zi}^t?nBBH`j>=(b3fpcM`Q?Fji9Hu5u{Mrh!1PH+IpU8n`eU-4a7Ir7f;nQiTd4N2^ivoOMXxkbMX3y+0j!;QFs?`&p~BD69luZN=P<-0z|l#2`I%;5 z4IS>51Hg&%BW`YSo4n`cD`ee*G!B5Ee6S9&U+&h&_sSP;>XdFECE6FspT8Abif!L6 zAG`6W@}Mk{ah%A?``&e~EZe_LK5}Ds)DAqYsi~a2|J~=vX=6v^Q`hZ~gOPybfuCoh zN+Gl6pMtBSyzC6{=%PAJy~dJq2drwtx}0dhAU~BO@YVS;AP!$ls>DJ#6C+KGo(xPPdq&WE|6@WBcH=18t#z z)=n)>+i2jU$m(Y&M*h}@f2b_pq$6^v4ZBtsMxj80lqoDf6IFr6IG{q%!XKI35n+maw_KQQrKr zwQ|W)PQRFx2k*mzsbJYU-wiJ~Q$FwlERs<-A)TF=n3qe-g8HQ(RWm3jz>(5Yu_Qy-f-pOFK(b3OET`@1 zmC-F7gmaqAC>JHd3K>Fq@#TrmMW+8`>6?kvVtV20uem}pb=b0)hg;2{(_+tF8N*)x zwfT0L#8#KRjTd9_e-?{VZ9rsptx)>s&hf=`Vx&)UefR4rsR>N4s&F&tUlq-Az+fTk zemJr- zFTn^_{ObG#T~y~^xlvO4QEd8Ed;;({Ajm*I zmJ`wj901#0hWkkvK55ixfZ;hl!&P78rxXk!1(9LASCLG0fJYKOGe@J5hv;;5bxULSUD9~o3-Gd0z5ns_^o!V0W&mGvwhWeJ zl-m&MpjxF|E?pctCrbTv^<9InlQ>yLu71(;rO;51J;dWWE$d@s??x<~9G1FFqm1nt2!I{9D37x?6!A4qfl zC73Sxych%bR;TRYoHB&n-zLW8iT+yIJ^*&lVL4C=Y&|c9G}{|9Yro0j5Y9J9>p+w~ zdl?KD$_<`Joff z{l6KLPk-PXSzNzRe&dpT@~NNglQ*4px-36$xh%TnQ8@r)Hta;IT`tv8eWO0@!UZz* zz@ESlnOwX=E^Wo_NT+-kFAs5*J?;EY@$o|)@`1^pE7p>=23R)KH619QkdFH0CsNn5 z`w2BZ;A*2rKsCK0#V*PZyd!?WAEh^qpBdR@L!QhKhEehYrp-(~_^(*OKdgJ+rc-D} z$e!vAWv>s_P?1;_f;WO12K5kMwSr4oE~a6+@=iG-v|!Kyhm^c0a}C!^h^vtl_JvM& zte2sCFyH;9e}GXUEU5BLa+#HRH?M)YpaC=swqgEZmOK0~5D-RVmR?#r7PL!SYqJ#cIXxcUoQL2e*0IrdF-?hhhpxQ7`6u7N2>Yk3ka z_u%J@(%ZB5SNk?&Hxx`S}W}xi=?5@h%gwH{sQPiLgd-Xm-RxO z<5s1k=@FqS-gO?815n~>B|gte8R_qm1$FzS>Fi%Y-L|Q>n%;3-xC_}B4#DchGx0#N zMuvu;!*amauUv8hUtPG1j8_ii30qoDCfm?NIhcjN5T)&l#pD(BIP!NYqMZ7MJ4(%~ z1It?dY#jX2g$kes9Q?Z76eAfP;>yTMpky9KHstB-l*U^dF{2UpqWs(E@01%kU6cfM zIeE*koF^|^m6qi%JzrkG{Vw@-XPH-`xmixDCxbru8m6{H{2AnK?5KR@&K~*07d6Yu z)j+;$pZvUYjl2w<{A^BZ^c}EUzk@3+X8NfkZX^d7R(ecz^ zZ(qfv9L#H#9GZVjW6^8{P)=KuR(jG=B}N{k4*~X2AeD#HOWpF`(*!;J_~ll;P{uLW zec<8mN%H(Rs!T2yxy-wH4VBkSn(U_5HuMi|>1 zZMV8k?8;GCEh8{ysvUV*Phm0ccT=Z2zFG)NE7t#kO)XB*W*^xwcvu?J=u<*p;%znC z*eLnM9kPH%W9_b07=h|o$8_p4t%uA`!4daSjJVhGOT}o5 zw57%*yTHHjMoCWd&|u7C^vXccu0m|xaaX8$^!3)w;z(1y>YQT@6tsw`N%GC zsFPQp-wZe@8-MbIJn^DdS$A$uUUFHz-1K8?po`J?D_8QfX21OKXMMm5hF=&Vf4cd{ z!QJ3VYWrixgMEo`xGFVP-Kx$!jOd&Rl=1^ReK*iAqE(;n~fOE8mFz)=}z!J#Wmmt9-G({+ZcWm ziHuD03qj7 zLOOk~4v_Z@^)TL(EQ>kT={@&jrltld)Yt1G{z?s*_@&x$8Nj-S zzU*qLZ(Ak}jjfUcejUom`t-_dvOHPzgmmr?)j`8O?CHcS)7XUrO~Zrz(lK_wG;er~ zq^9ZJkt_9I&MUWq#6IpgjEtwow`NAAA1{x?BgKQ@b`51tQ$uFuNQxpQ~w8}2B}(1xrL;hu#a6})>qP$4^D%v40pVn-IAv&gqm z>*VZpz&jal#MK==@x2G;aoYH=a9{n0O>)L3o+m3B7RyJk-YXyZ_kQhs^VuX;c*Ah# zUc->X&41>_A4+7zZIg5_f>fsx+#v7Ehlnf!8or+hEBM9$%tvw?&1gRV*G z!l?W^&s{4^&RHrgH}%N!D^>|-$@D$GOYT8B_5ffS)}=qA{JU-`sPd!q#@B(v5bC$U z^M_DWdK%{R$I?5HpONziphGenJw&7Z0ohb@IZlH;D5+!*q=XTGDJaaUb=s^mc4PI# z>Ja7TbRqB1>51pT^ln4(mZs&q98L`(4FBp=M)rWSPrM@ALpmKfLb{1r8|r1odZ8dpNO2| z9E{l=G=f2d+ywH5QTMflUKuX7$e}Kr-WhwqJ=^+bITY%%808(51WZ?oDWRv= zYMr6pOW6Gh1DmCEha^$g*=EXeGdGqW^Ucj`d`$0@)c_kqyg{%MR9A#)nP;5_k*=@~ zFz#1|45uX?VBGKV3M1yq)P&JSL&3V_FTZbl!!EhxTzNyY@f+$K7A_ynuUH_f(UY+Q zo9WlO4xjpfH*~aD_NyENa?`!N2uaBmmp93aE?tb-@*=x8@0Kpa%b|Vp0Hbs5a`lpW zxoSfJH$}Pq)G=SiqP#?tT3{T|QWn|{K$-{Ngb z3w1)>SiD1}W4}at!nj{~k_VkbyQ{o<9N`p<_AN5F_W^k<`x42vuD~bjddX+-S(^`0 zeqYx#Sg+`}pFVUv0x+s`V5#TN zSWg2t_274qg28~)UgdEr)vjEl4Xibbvs7D3_sD9osZ028M7KZu0O_IY1qYeO| z4O15H_3H8%iR@_85p5dm#9^CeAMH^4YfVKHX5!Oa94Ago9{i(oIE{FQfH*&r^Xp(O z<}26b8f3gCC*wW4K~SMU zA^mg?>93|&d=L<-Kj4!`^^a={N~NMSciken6&I6TgyVVe{D2$yQ2_s`R7%18bA3Vv zFEFdyu!gH{OzN}v3V#yIfl@RVGICbaQM@qZ@G{ME;7}!|$zaM@DTM*1BxHchl&?DP zyr0YNlsDYjFrum&{NBh|@La5<3zpOXY}Fg=AQ6zYEh~$QLKI{@^}N$Vtkp-+q}e zx^#H2{L8_zJT4dUnP-P5l7EvI$tVxg%~SeahasO#OL-8+{jUxnQ?u|TQ49iM`8*cAv1$v96Ij&;mL4f3kY+V7ae;a+cm zthnKZv~*UF7{`1VPwTydE(}dO;OGIUoiQSmr{R&0(vxvuUVr*8$yEOy>3j6Y`m$*c z&%kgV%xmBj(|}ifwgWYGVOf@v1(_PhXL*cbr!dN0v!p}P`Fd?{Y%pvSuo0vV#&Lf4 zMyc&TC=K~KgtL!Fe}W(T28ZhBVpoQR=ZGF81iwwSv_8nfh!gk(n8F4H8T=X^_zvc^ z!{|lYd?0Jv4D!^$Sj?_kiaFIV@Hx#5IBef^4(WHJoA7x4NJ~6I^BccNZyIl5aopK> zbLvhC(?@a`!S0!Cma#*Rkr+Q*@FQP`^ivmvoZnC5xlo^%4b2Cor3q^f5OFf}O`dJ7 z^|CyBR2DU*B!@2<)G701a+-`PC&cD-4Nm#&Y`6?tC1OMybtUOjWHfJ~dxf8Q;g5NC zNI$dhFz#oV{sIkwx-dmm=aupct>PIpKlle)-RAV@7MxEoAejYcYGRDDc<5a5a6!WQ zEDX^64CZqgsp6+o#4z27Umz}QKZ^3mXxW^sa$q^Pr}||DcDl~z@B#(BsvwZ1gmyQ8 zMb1eW?z+!f(1YjOp($)`Ud`$_#q-X(Nyi37f7Hl=A3%*$jb~-1V=H4t4-^I&uCZmP zxFJkI)u+&i0FlZYsgSoDTtT3(|Kme)*(WZL)~4n1v6t+TkKNKO*WJ=7e|psd97X&B z`Ka72pS~SC%Ye=nVK&O|zULA(yk(+CzWzPi4MF*0FWsAWt(RY2l~IYt4(yR14wMDo zarZ7cd{wJ1SS7-PckZ-Ctk{+EKkfW$a-c?@wSzztB(%4FB&0eBP(gT*FkD^GlXZhu zc8Fpb(edHN4%ZD(9bt&`2ae-L_{!4w>%)OO0+P#SvXa(#^8qxt??Iu_-UOq)8HQX(_(k(6QPXi zU;yM6ERY4~oGbMU7dlO@mb@?~?-CJ`Xlc1$GkZ4y%- z(0ny7*|jTVV$VTI^!1|ur_qkN&-i6@t~pL~yyMF<^~#S)5p#Y=MzH|Uopm94$0o<> zPDrP14=EdD9Q&AZKjn`D8jQ^iS;^e~CCN^Vz;NN#ci!;%Ji-n)xx6GWK#*&1#&jA^ z=dd~W#A3f}{ecPN&nJf-7s*kDPf0N(qY_i@YUh$TB(Cuc&&MhPesxS@#GU)h_h*;L zXy*esPvYhL)S;Nw@;EvBQcgW-fZ@KTZBPym9hJ$p7CG1nquKTB5JXQ`-^i@AHrGod zre>VmHh=}D?P{cQj=k&5Wg?`!rQt3an9NIk(*n$G$AVO>HQ-koEzdY|l|LqKRgBdY zurBA#{Xx+#i$h~j=14T&(FWk6)WYGLB)9w`THY1;@H$6Gm*5Kl|HF#``Foi#o_gS@ zdjsm*;%4l&v->fbY*{CVj$ptKX{jrV@r=r$0aKM4G6{P@XlO$@-~|e4yR2i_JHI=z zNY`lD4sDia%Ha98T$xyN@j5pQcdu|LZFUA)@uSkVNUPuMWGzBROt5HUT1_Fk*Pe*S z>>j?cnv$pRW{iUlFtF|{NEJ1p(q8$iTb9fJd}*7kdCB?m+6R6r-@f&3xnaeX@`kl} zIq#~=&?bNenvc zOn8eNIUqmZGbV3Xj!87dgYupGNARFf^63}PYtTz1{A~DvgBrlVp+5Pu2be*kC|kw{ zeAT-?B*wtZdY)pnY`bEd|_?SJIaN|xy)qA2cq$`j@dd} zM3Bxi1sw9!TX>Y7qZG=(a2c7b?~wfJ%dth|-%IN1Kb8zWYkOx&)Dv-(sek`I+49xv zr2omCQp>LyBREoC^)U-43T6tK1>ppPotI`5w8!HAT;o?~$K|6E@OgOaH~&=%D_6_e z*Z!symMneRWl^2Xvw45R4L8WqqeqqHt6udgS+i!%v&sIX@nSXP&l>@~<4K(S3d3EE za~R_dtt~Rt)`Tq}`>~xK))oaKjfWwVD-Ou`fBcO!U;P0!l(<)UxdL0Iv({E=G*x5@ z?N0Z_X5*wo8cB@!w>Rab_P%dPzVivmVx+ScZCI-(;wdZfa!t*WT(=5Fds@E|*p~z6 z1E>VyzN+#EE(ld_KrQzl${yh!X7{{W6Ea`*)$=%Wdiv6XsSbR>sLi%utK5`~b#0eS z`#NJpaf9)w0ikRt1E&DxFu%I7u^{KQACcihDVb_-l7l!&oJC?%S-MY0-|YKZun4s= zS0m4F+bzvF0l0vd=QO@z@JoSd-o%VwC-4$+v;^G77Btre7?&IsrlTyTSG_%vi>uEe zY2*GXUd%tMj_JX0=`X=Hne5#m+0i|cTl$)5d5E4Ffl$jDe$Q()WU_VmdYr}dp~1l+ z`RhOXeL3^RZGzT12+^vE(!)lMcp_fI z9IvW2B@)*I5%?)CL>pQeZuKTA4@ad{!B}u5?5WSc^-W5e%buR&r2QCgBTU;9l##C%6PVps;bVhj?e{|#X`&$ zY;<08_rFTRp8IeJHl`=QaOKN_8!x2DVy|#mm(qxXmw4zB=K$MoQ-L4qJnJijA$P>_ z-XQO*>#|choQ@x|I1PzL_|dLb$sgD(>GrjX%;^ovAtDI@ zY`YrFx(5!X&O^OU@Sr~{#ExS6mzWau#D7Uu%n@%^+oZ64pjBPQE`Cm;xOsB zs}}E=_#lpJS@9VbvTM9C9EgFq!oeTvVM~cxOy|-(n2w2KJeV(c?zaMn=h^Uq30?Hz z1@^!F-pAxfSC8EG?a#^h*oa(o)mx+!Yv{Q1%+oCg*4?(2f)ui~IKh3JEN;z8GhWo{ z^BFu-`4xk*Psuo@$>{P1Y^uTa57ap&j^|(fmJQF`oP%@1aQ9O6N=#^emcDhs>{#gM ze!5BQy>6c^QI&A-4{Tz1gLTL_-F+t1tZIDhMIwLx=igQyMz7US`SRX*6QZk-|NWQW zmA{4GI%P|5;~ftQf6Ep%Nk^U|y%W-RctExdaGOZrvTDaZfzglUl=8(d+#)xpN918x ztHvLsmb?r<`mp@*mv_m*?OdrEc+eNpcKqu6c)lGHZ>Icn1gWkD@=f6r7?CLh(+PwW zL&YY9^Eg5}O*e*bVHy=K#0#0Bk1rpTJ-mciKFaylj!N~SDaRS9pQAyJGkW+j8QKS< zy#aIDVYDw?xDeZhHDM7=w*1*J;$Skk+?J=%Nm3r*S&k^J)ul2H;!*J;THu;~K%qos zK1@!_89dbwJ!%)GzwmiU_IFDycRj+xk@?hoAvwF5&dlVGeVQgNokPt8IeCElmAj?GOoY)H)fFBH zrZeSKCRM&1#Q{EW%(*}C({$*!^z7>f(e2*#MM$S0l&J;~kAVT7er<*UUvXlWnev~+ zgE4{a@HT(pZ>8?Te<*FIoo?kCqnr0WeGOKN!$?GUe%S$cs9{hI92b6^KQj=7*I@cR&7dZrP+fFPi$)gbt7jo^cXpW+ zkKTpdK`=Eb&2@sWPNA;P<@8y?sTt3kIx1Vcuytq0G8r5imY&`L?R!-s z@*)#Dib>~C5pWNf$HLy0=7Mz8b;jGD>Rew+hYKMc_>B*A%0%{Z z$c3*XC@1Z2Ew-F*+=nWoG0KKvRd}u^2FLvYEdmi;6jJ8JAZC*TN2T6fk{!a=n)80U zoc9=w3P9w*kmX}KFDuHP6^20n0A5Pp^qapYJD=PaaavnADyP+Ll^VQ&4fAx|?tV?5 zRL4{AnLeMJZ7q$`TsI)+)bEuAtp%J((4q%lWm0&7!j~GW9|;-57pOytm6FbI+fy4< z3RzH@JWHO7NYFXvnNx=QjAa{YKzwRPdwM`RF(Si5C0Oz7Q)i&A>VdohIb^GJBa#$O z>m&Qs{2bQ}P9|6}_fT617y<4%&?9@Q)7M@D&CYuR2ts3!Eqw4uk38gJYm--rj2Vpo zhqiWuuV&C7VvZ$mbr00Ie0;41RTTSFQiY=RWJo1W1-{pJR3AL&j-qQA5bzqBNctyDf*4MyJj=2IUV%sp4v0jjzk;Mm}?)opHd|$FXXnp932&iZ8;*!+KFq-~AlS{7^GT#VTu)8jOuE&upMsU3jvL!(_24jCn2;^DTs zu2@7Fp{fW1I_ke?AYv34YB>BM-t(mSWYP$Cp12Lk0Y8!_`2ArW32O{zHhz>-xDb!b zf&f;ba2pcvsBk-rK!HbD)+9!Hr|{*1dcFKl{w#W6g+%AUv!DUaP3DduLy0Dt#3`7m zMvP=*WSIMn_n&o|45P2i4i{v&A7@*j{Ay6dbyzrw<%mcf?P`2ThX4I9WNhU{()PSJ z$Ry_J_G7Q>5v&2AAyx6wGE@(pzm5631?)amhv@+F$jy@6@?*h_4QYUZo5#N8DHyvt zumP*w`j&(=Em|P5W~nqSY=hyB?FYewi#WA%hq`BV-;6P;$LfB(?Nrv)>G(#=Hf^r5 z@=kACRru9)pE|-HjdzZo=W$5Uz!;`;;;UtD)n%GQlvGGD4%lw_RiOd-Rej^WwzgN= z`W}^SNxo!t;B?jz>BCD?u{buPzR8ax{(11=<^lPdQ8_KMRTdQ{@s*=Rnp-g1j>DF8 zIu8%MVnN%8-Sno)MaQFEGKnn~xo7)q=U-53bPlWim8D>K9!A{iKFj+rLUZ)QC@=uV zgR2_%e^;{0F2YwNY;hXugqI}gT-Oej3vSA6fvs#ojd0pJ|K*(@k_R4o)RM*Z&9A*i zUiZo?K~t0k#VxWUxd?S}ffVr!=O=g&5f7}Q#&DR(mQ2w!gyHV>$(ulB z5nC)fDpnv?q_Z23$e!V~GK6EC$Hu~{B zjQqE?Hc4x;U)pjwQ4~)=&r5v7#`QBB)CULBBBWCt!&}yrYs{f)8JHoXRG4PILUB>F zGW9~x8}!p0X{XVSg{Yj<&gn56X?LSNpiKj+e}t;2R8P#WB9H=QqIt2xRlk)YutOdr z1|biI>iHrQc3aBbcrl3gQ6lAdjnOD7jZ}sgl9zPLDk&|!Kn5}2J$dEFu~;vIvhtB> zWjyUd!=m$%hh?n0OH$Z|jA?5*ktv%PJW2)D~TON|d=bk^S zp->iX-uF}*V2$8UOB3bHGUI_{!!@l~K;0`Dyx`IxD{xe{sZqu*STDI>6frfR2*ZQ} z4!{R7N8a;O9PPG+tF_o|rugJWDJ?k*U#ngSBb!q}aJUnUNp8=lbrgiUkdbquyZ6e} z<~t<;n#^Po2P|PeH;s10(P$@Mtykm%f75~vseyr6zkZEvIg1@PXaw{^p=SlvaR^Z1VhAAxj*@vX@C4nzRU9u1c zCJvTbb*afg(mP^^}+?b8SFgLh%-pKhW5$9x)$lm;|s}T8=jvy zzXT_K(r{$ArMr7!-}q!=LRw((FG}r`wro-AF^9UT&>&4P*c)(g62At}2-keaTkroP`RVOYwQ zQE9-O|GuGva-gnRj%3?qcoNf@@Pfm^96IWi_LHC-YVl>MskUDhCbwgj0JfLJ*ObOW ztB$g>JoqxIyCgvfuLEOUPhwisq!e4%VHc$|2Ek%qXH83*4uyBEgn&5*=7izyrL2XG z3oD9OmdGMT(bvF`OQO^5sGpMavX9B81?y!56?Tw^5zJCkXf$ZYI}1W}!wtPJ8jZEo zW6}f9aacRuPe^y7{9RT;x)bH6M6rw0gOfH~zF{}ij>yGWe3D9HPBDg!e7;g-)zP^%b6oRhwi^m61oT#Jefb0-S}Hw zC{JB+L&A*>%EbX1m(OGmJRqCZO@p}Aoi%8%_$id6WEzBY)%eN}QHrAR^dmJxkNhr= zs*AkHQl0a*0R4IaW`uN>bXi_9_ArY#T$9n#CUZ~knE8iZ4{f|p7MyjCzC?K&)x_s7 zC#C@wA%0eptSc~*F@jgbp)RGN2eb-tZfzQiRC(n6NJpz2*|1g`H&01#XD1d{!gxpd zvgCDC9DW{*lmd@2%r720_^1^3J*-n{Ci1PaKarEihQ`&{$WDyQWd#^9{fBXGH8v%H zF~jX1xyUq&dGd95#_(Q8+QHxJccHCal9+#;eg4@nHqnoAY7LBbzCa*pJgUNA1ZY^L znKwc@^IOZ-`!H}34d{S@)=M%9_;JXJY<7Rpm?i+fC!kcvzSul>OqanIN^F^gm!v0U z?8sJ|>mFD;m-wx~zZNvRi9Pg6B5v&(0rR1|;y8-#G z)xFGr{k6X&YnNc|dTPpc41!^p&lR+68}M?L$)xZl0LEaHf21B0K(hodwPg!Hh8H;CA$f9R8cA>aD`kE7trFFs%1|E_CwI!hDgq4UMDh;;igm8NlcSXzgN z@TI8-Yxl#L(@~$q7H-C&$*#O&8430xL=T;=a-$&%9^?&NpO4KboueIa zG>!_P3pZp<#}ZM8m#W5dUMJmme@O=LlG?c9d^H#?vhvE&c6Rc^2M%D`z__Mj7vMQ# zdSm`d4!C2ojIWZ*iOj%dU2z1FS&VTvk%LEbuBhYX7OiBjR14&y_5itIu#7bOeZ zGzNy#i)EtAj1;yb9pS}J7vrU7T?E;KlTxwwjPzkAolzJlbYw$gyEXx195h*sK#l>g zh_}!Z+Qlf0m^98v*t&I>;-@k4c*TbGmg&Uz5_VNt~i^$dbV_09bhR-JnsGBEBptmw*mNas-NDZ--q>=oAo zi0@&fvs=o_QCCh!;k1{|!4An_5pD)^q+{5T-?_!7XtMpXTsR`jsV@yEpTWTqotDI@ zNdtq!>VNW)pXLANAe&q9h6{#o+t_3G2c{X$bS)r zdtai%zMMdwav3Cb&xzWCLDvp`%ooViXu{(*N(drPylW7j2~YRea>3G7;zO=m+HU{^I5 z@Pl}ndi;s|WXD6dD?Rmn&hNffX>xfO>7Y&FC5&HR_|i&4o@*dfm&ml&AQF%F9g<@A zlhWPzc6=$xBTbv%u8dkbM5RywROKYOsDk@=9v6KT8;6A#@TVi8r`w-@8uMr=3n`k29KG4JWy39 z9HnySJSj)W|JadjYP4^wzZ%nH*62w4@l20#mdup$LIByDKEp#f=+C++Qb2YSU&~%k zZVctlSmnTndI)p%8&8t54wy3>jt8m$3JgEg6U`sx0g7I#=t0Nv#F{L&a6k#44gLHmi1t{ zro{X_p%5%*UQEjRX6d=U%KB$`U=E*hqMq=lF)+?FO}<#c`AdQu@(g9#5eEw>vwVj6 z=ae$)mwHZdi$E9~^X_SCfJGYW0U_HEr^<|{+p)S2d(#((r3Smqun2kjaREkVsku=m zR$$>jM&b((V=4{iyZ3cKf>Fd{Y6c?99=&S>Cr)jWrWS#Tn zxu$_H5j5CUC-DTGFhJ&q&SA0dfQ)3W)TvF{PY1kX0qdP&eF~)XFr{@F0wmxUR}ZdI zdz2iF> z38pAEJUUCx$FeH)B@dtqgGwF}j;I(W|A&ke)U~$BxCOx%j7C<^L3h zc{vZ}GvVm>-YEFPtzw0;Rex3fdCO0@LkF<?UaRam>Q#)L6S%r zcF!pS9Ch@m0`Z-FNUvPL&$t;s`aRD!?S@kgh)oCCV(C-2QJxta@~dIEr@l=-np2GD zl{AFP4Hu^G3Ao?nC*HG}f67kT_)v8jO_KTRGu40^ZeUV}sSjgyZ8%o_h@@JUYh7Yp z=cu68Y}6S}i|FNO?H24qjvYn1u-i-t^MJK#fye)4?@IvwDynuz1}p$`?UD+&rm5+Fbll8}8*zI^+> zyx;GfIdkv-{=Y2=ROQQj-~XO-XU;k^ckbM|b7u-sM(oV-1J((y5`zgbO_&;B>wTG} zy_kQ!poY_8K&LW8ok6mfZ8i3UsHtz1vBm9DxdO}mV#K`(ou(Reib`Np5!zC%dZGG_ z5B1?X-6q4Gdocy&uuKeM9pG`!FShy{i3_n}N(q*nEXAj_u<#8eEbgADD-J zeKJq%N(zwHXldYn%+Ysr{;)b3? z2f9hi2>Bp1(SJyW+ZRhMMt*ZToX#)>^tQ<%$xYn z2k}vOha9ZFM%uAMZ5ijcQ}1Mf(Oy`SrO%PL+4fGJ5ZMvhAiFq&N3n8i{R9PI2^tF2 zGksDV8!ZnX9ibsfUHFbi{&?AYh4b3wT643I71L!f#qrt7o0&gXU(zhOeVKGdyU9#~ zuc38N!kP8qhR2a5jE->bsE!t?k4ZzKg2KV%C5>->!^2JzdCgwO4bn8`-*o)uPNI`? z5_yB{v}2yCciP+9+TxOrv+uN1I-6z4#=n-d))5~jOR`ZZ%@xa9ki_g(c1zTHz@a$= zy-6xjU<|-+X*l6I8#s;EgYvLL%E&}M($qL3)uIio02;W<;RvS8bU*Yxsa zE1Gm-^nDa>LnwMNgPPded^+-WkofOx!N-1^^Ev~^7 z9gKt*6rr=vjPg3_xMp66_c%J?bPm&Tz;(I|JMNe=v9Dq&mifCIo4-9>3d^fWsr-6f zIKS6-hirHFwv-ToM?PQ?j*T&Y80*YjZW9klNasku4mwZ{d4oK^o*c%sAfI%)G8^D% z=N=g@#)p@9=+V)3&?K6uck19OQ(~j4skx?3QGbySqo}X#2XB*_RWGn&V%{L6{YifV zp?KxUT)*z2{~*W$^;k!#aKY#0Ml!3-dhYb;0t|hfw76 zSP?#w#kU?0ZQ4Q+8lH5)x!9Kg-*KSaxv7@lJD8{gbjU<4bB!>Z9|89~@?*JAuEIks zcC*FCSjD>WQgZ)d-qR8W7N_E~Ef8aS!X1+<{t4&WY#j=l>3E;ct_NKFR#l5j`J{-S z5AS(}@jC4_Dx=OTIw1oqtjA?Y?bK`MEagL-)!(Q;Wowz z)Mq-F^`94@#=HH1iu(Bs`HOp{`md0JsL#ug=b4c9e^95#Y}8JBb=#@Z+}w;;t#ueB znEDl6MqJLRHAW$4kq2xbz0qS)EMo$tp+l0)(U`AJ4VT)EGpD2=jp~q(op| z>3dewVmhsXO`1K@nLx@^7V;uQoXLgwNblSi%F%m1C;iy%uxQ?y`kK=y?dUfN9U+HI z+G8DWvnN}|={z@mxz_;CXobO`!+;(wHF_H;%|8-k<@+TX`|0FJjlfS%Zy(IMJd!S!GsJ%t=WVmNH)@etMhfHUQ>-=?e zzGcGpq%J{6|0p`qy{*{wZs!3kZ(E_AmF|uXtg_Ls9Um(zt1k^)NhNgLj|>j$FagE0 z-f zWgPRGOYyN_4X1+Oqdz=^aNa)Ytla<)I%Z>N2*)u(JB)@Llj?h;R zc1UekhqRULla}HQQa*N63b5O0Y3)2Iscpw^p01{%?MAHske#}g=#^!jUn}Nm%ut*b z<6))@_LWuENhcmQDt7-?s!o4~6{oW~3~iL@G?JVn8!raJ;IHBq8?7W2f9!-iRFbTG zo-;V4+(0voO}CB~w4?pw+<5DNQr2W4<8>4l#ezx22Iee%O07}^6QUfhBEyHp+MWL? z<*m!5q<*1E$`GorNO3!}{Xm2UffsK(LLWfOhjwbnScubctnuIZ!2gx? z!)MEQEA}kV6>K;)$GxCF3l4df#zXR)19PI|o-XJ#uBGjFdDfM5#}AE^RUmz|N2{K# z8s}uRMR$A!9HC$DJWpo%x?rUctN|KVvNHv4p|HoYTHZJMDGzm*KTwQ(QfU zAI}_f2>InjL#Q2S+KmRt&9P23&fH<96UQ%P5jeG3L=+zukrb+)+5|fDA>5EOUsPK# zIi(9_N4>6!1bSl+lmwJzl4t`7P{pgUOW+xQB)w}sCB;{~U#e=c6dH!M%ywf|wSmFy zG0619MxN0Ii&G?pQ=x}6?r{cf1m33GcuJznVoZmpx=ufL*#@)ru^w1DUq%lek@^GGQh*O6DR&Sb>Gk#XVcNE8TvxugH7GE}t` z(--P64G}BNU|I+}>|D;0KKV7O9V&Lv^oKdo=v=Xb#gS3sqv*hoV2=3c=mO~+8k8gG z+%s-`bU^Bc4oO4jL0N#+)avDzDzmr_AA8oZGe2JntMKg$Z1TKJYP!Yn=yCL)_fFbX zj_DTsxb#r>VyuR;PAZpPDx8OJ4IBjqiFBAFFWzW5uA?!O7y{2V4d-qQQN3;ndEbGo z?6r(2Wl$J+iANcZOkgU~9_cN`l7z@#x5rZ{B^%OdE^K1O{!2OyQw> zrD|ZaRIPdyn0eDf$mExfXsgdjy0>LoFTnT6)*W&!HdyM$F2M{O6@TB`{#?#Gd%4bO z=Q52InA6U;Pig=yox~B!8$q4=^_u%t4i9IadAjg}R9rUrsr9fm?J22}^YOtR=~(v- z**Y*EAEaI?t@U`Y!$+xh54Jm#a|CHTJsCm{%!!VBEN!nhg#Po&q?HU+Bvdvg=Tw+X zgm#2*F?gp&Z|P_-6jGt%Y(xAw^pt!Xy$ObOwqkV zS+TvigZq9@7lk!{ku_DfQ7>FY4EXCEh##{+HhQD?IzNimGJFeEnFX|>FM+h5Cq))I6U zG{z46o&ckO9rxZLJOAGucu(W}%s<|U%oiSAA{nh?Z;X0Xe*$ z8Rs`gR~E$8Jn!wY^0Fq``NQP-#v<5JtwfG`vf{4{*hl&x3KaH>LV{NLe8(Qz)1K2 zmIUsp?w5XS95b?Hfo=jccBEU1u&ZZvcdry<)sqEBunYCx18N_(agtJuVoqS{!3ef` zFT%(=JF%4*wdcq>Hwj{goy!zb4}UN(RwsTu+q3y6a`(t(n9FdQv^Tcuh$MHE)RAD) zBWOFownWds&iaHO-6qaXJUeycSQTUpbDKF=nw@oy;%Y}8kV_1Xj4YHx=*;iIegb2N zD<3-|4gLFN-jQ9>QnXfzCx)e{c7c@F&zE9!tmuYF@V6Tu z9rkzCNhLe)OD+!9P6M=kL%KsJ<+;bXDo<0$Nc@-&5#|us31yJRWd)og)1%nccCh)` zy0a~(S7{zo9KsZArViF3f_!t%S+SgWBNosV; zu#}ywU0g$d&3z9>YgV7LT&H;PV{2xT-P595<|6$9$0)wf=v;S;Y#v-7JF70k3N|fL zgH`Bs?e^f4=7UNn&m?w4&2tm6tN#h^yI#Y=N8+po{8>n_bZq7_NNS)b@P@CI%x<=n z;#G`tt)8oizL1w1A&nluwV8%$PkZ#b>~Q)e)MAdM?6Bj<+y zLxq~bf8!>p=W7GnAwMzy)}3|H9b$8ONW_)v8Ko6ZbjgnbMP7jUJMK`f2e zGKhz{wYFFFVT`~RoxVsaF^Z{ch-1{goEsCN6I+2PHN+XumAE*%&)R9%tC}PIJJ!hV zu_ZEGdnQKa@xf{xmZ!ysvK$504`aa>+YGRH&7qIPlLu?y+IdF`6tELNsw45(ya;%@ zBR_h*yFDCJ;9B(b`*6=fslw8b6$>xGoO*OU)Zsv>h66Wtc%Tsh0(c+~e+-evmazeykt6rQ-Ga7*-!D#L6uyjfW7N+#z+Pr{jey9MtAl){Ly$=tIz< zZ3b{o&1K!H-|;!Tp*c;;nodJx%8;yJ6-1pb2yGe59WkAFuzcaG{!+H>+!MSIdDD$A zm1jKlN%|3LHOfXe8AM^l6?F0pb=hchA$J%L756{1DayNO_1XIU2+P|kX>YQlyWvCF zo|kg zMdC^4$Va)nx*2iq883(2o3vFX>uj4mDxh4RQ5WceYv;|Pa&R@q2sPE%s3R3ZCDj

yObT6f@EPA^C5e_D8hSUV|M`xpEAr$W-F%caF4s zU10Vedr0N476eb2PjnQ18AAVfpyJl(%MQgO2S0++*vj(5At&N-k;vZ4P#7? z#k+UJ?Ov>IQ28 zN36R9Z`-i43`dU{j271;F9ZhE0ZtiU2a2!$bvICqMAzcu(Ghf72GMcn{9cZVvJ+oF zFf3;+JH>c$?Ck2nvU+t`3%kKq*a{tYPEX({G^di}3>Z+aU4Hr|VDR9>vZM4mX>4ki zT6Eg0u-XPctmWH1^7{2lg>Eu&cAS`I(-i`a2Ls0Wy{H%`j&p?(?IhwAm`$s(O@7#+ zXGfktj=*yRo}Qt_=*$nwI?UfM#z(GA0|%t-$YGgRwi&y^c40aVK6vFEc}}lkN4^YG zijblF(2*ah4q>!@5NmZ0?SDwh=AENy>|8RCj&=|xff+JIm@FxsDUPQhW&1gC>UMo0 zex!4kbQR6lZwUDC<{!kmIv3$=N$#876r;X4Lk23J(wJXHpP0qAgCEngT#EPpP^w1u zNzLgm%g+S&!yzeBxOUoc-iHUu7v1zWc_6j=%PVepuDsxyKax7E_|t%ujJT^Qzek84 z^}4o{JT!Rf>IC21jEu>;2e8X{zrBeg-+35u=f}aZ{G2zy2`pvVv*l;fcVLU$HSiQH zw|xe#j~1!J6qy>#ZD-xq>yz2YJhSX{5Fy1SdGChKJvb*i?phI87eb{V)KLz6sMERO zM^;gsXzta4-y4s4z4Bum-Lz>IS0_ozThEYEtPVju=b{c5|Ip!W7-`?Eo%Y&!%XD+4 z8Z5)cPCIX$v7tHMTwt9-P<6!BZXm(_Xp_=0UdeNy(rC1$2Uexoci^zx@cQ=!rhIVr z^3!C^?O#e802L`cDHg`lP6jo@4~SIzQyqfRh9%fi`%>v!^Y2o8`MdFP9pwM59 z17CT0P(IGpMjGmR5#G*~mhdCbadgJ9+89T!(V-c|l$(A`Cz=>v=A7D&k&YYLaGpG; z-0*`xpZm>9W}1RN$k!M^b{>a{8Zo_Q5!S=TNAsx-X>3|t@C(|)C8$u9!{#(h`|lk} z+TbfKam1ewyI5pV+1UOfQSIkc8v4jN@|;dH%+YteNwBFk!_q&rP&S}rzaAgt7L4{w z!@xml?l^>b@@u7f{IC>YNy=h$;EOAp(bi&`4D#=A;X*0j@oVk4Q(?OOP2+iv(GfZu zLcE4O0&udpn5-Frvj9nJ`C&4x^e1HK=pH#*v`E`Ko%TVwQAp|}L=vzBoknq-gi_V` zbb2HV3|V^ikNE&K-nm7Z_kCOH&bt8vFrk~~g-v}~`{9?zAzASul*_CC;(hYVyD{(+ z!gHVb40-vBo{tYvYq8`tK9WmqA z*K=@d&Vf16aSzvsU0tjoXh26x>g1mC7fXL{uQo{fQA<+Sv?63HO}2KV>Si|SxcIaB zW|BXt(qo96NnbWSD|#mTj+MU;uVlCEJ|GvLQG**6N0@MevRE$IVA@ zmHUP6kfcM0w@d%Q&6v}EwNx}M(djYlwEK>zHUgItdORoUrvoU1_BYiX0|sKiDe!$%&KzV+Y4 zhfz1+evX?u0)0d;1sD|u$TP%Kwr}MQ6v4C^mcqdoSwr+h!8UpLwufcmD^^J{9MAJb+B^Amq^-S3EL@XurW2ZEGpMwnjY1KI%w@^WxZp)%b=Zj>&Opb*-3Wj*_rPhZj{d9Q{~}4tf(_KEHx8HrKt-G zSeEROTDf0F`;W@e{$govE64JX8>PH$dDQE65o%BfZ^LP)bXtdh%}#E7O8-{t>j{0@>D_f1Rm1l@n`Jfw+N#{-O~E# zC$Q1d)Ad^dlT9HiK~E>OKb6jDHgEgDN9F(h@aM*k<7Jm#C~tV>4f<^bmmaOe2hx0? zWW6+bBu#mv@{HZ}F`adP&ARoLfa7X>+{(^>@|MT)bqrI-dbj>u1`h9#d-~3nL&eK5 z0Fyv$zrx-q4UHIO$HE8Y_|UqTtI$BWK9Eu()q9?VQ;#Ne^Ryah#N0g0iH^HhO225> z87#-96vGXtOGz1q{m}4e@Cp+utkZiXp&XARz2mb@HziXaGwQQ-I5rsWPnIB4=kw_E z?4=L*1$UDR=6B@(V9$W`9mL0ASmU@B(^+`qn5=Wn4&)GCPZxGzjvXCHI~Fo$2p?GX zAKapyc5beeo*v_`K&LK=V@XL$P4iOqBQ6#aWm$$=M-z=2Zj{=2)M(r&BL&&Wo6^t3 znL?nkY#=jbWaKawQJz(%gtQ3c57T$NQ>!8nS-bjr%y<9a(%XQ!&J8}INIu%lo1H-w z6C9zA$pU33NwhC8B+Kd~i#HWxg#sGP5TK7JdFO{Vs;)q0HYtO$O=RY%GcrA(nnw>| zYGZW3t#znkaiGHB{}FW95ysJR(Rsyglg8#AC_6osXEOhDIZmD(Zw|1^@W-NbNLg(A zIhS_093QA+(-%x6whf8$>LSZH)K6lWx?%HiDUtHW@vY2+PJx-<-Xh*{--QoVO+Qzu z!L%Ekczc$WXOiqV($^t9g-gQ`)fk9y5M zy?#g!e&m+Qz}w{MlpEAG^eyBay!A>&v)1wx;u3QSA;?0zYI$P?me8lIDm4;I{fX}o$d z295AQu2ZW(%XAV=63FMWnCv*AhAa#;A=QNQ1a!QwF;8o^epbrsTBTyaxiQ^saBT}LZfq{;~p;}lr}3WpYkeEaoL#G)Y ziwp8RPDQh}r6U}liXodmlYNi3{2BCT&CQyl?Sm6hmg9s7W!-&JcX&`1ENIacEGAvi z1ck{4uB)p{$`5tRaPN?m;A2yrDXv#JI`thp&}rWqJMG+DDLp;LZ=R<0M33eKJFpsX z8pmj1kGse;jSr1f?DU4tr7?p>D=TEFxR5cPI2|Mlew53g-AbTrOf>;<>4w?#gMd9* z`&X#?!NYzGX^P5grEb-=a&YYzW&DDh)jreUS`6}--w}miwTF=U6He|vX{SjX?U?`6 zQwh6@-}}$lYDQfMInXAjzWuLd`BRpm z%VA;o$adL%+ZW}bZ~sao+TQ*dxp4J>{PwkPmIGWN4f53*=9#p~ihul=EUtJ^e(}Z+ z31@SW@TW-Nim^`d|2ASqKp#>tOw@!Ic z;iwKYES%RW4{qG5Bs=%)*C{7f-Y(mqUYFoENOF@!$Q5PAOY3zFbaaK3MtxAJfzftXwmKH&H|A z%nxC1JV)odJL_fBp%%GlV2#vZ!GN+RyWZ)(stNXJZr~73sU0}-OD(}6*GNW$IC2mt zr9&p=wj~tgFU>%x^dp@+rMq|m-g4kWN%RePpKvt}m6R$%;M5jsB4lu!Py-RBd4%JE z-p%CN`^>+hUAIg1@J^{Y=cP1SSt%#+KM0~f6q;ig9zKxw|Nf<0DrMU3qo+QI52uFJxs4W84shWJ|^2%+2RCo(M|f?pZp&ot^ff z0~>WwgO1?}`R%}EGE~+s4cG^vrj9!TW4aVJA1lEdD4RCMPCJCSDhhNQP){?^B9+Vt z=IGJyX-{bPy>4rpgPOuogwkTHCQ^mQ2Z{PR5IK-{afC|cwU%;N(W5-@uwHqkc0AOz zNNX2|@#XLbSxI`PaZ$-GHPP+#QF&14l&(Kps2O*_ZWq5d55H^wM+(aDKvk#fs`KMf zuJ|%pC!3?~Y@jC4@QmXFv;O^?q@rb+KJ5=5-6{Qt9@S2JdUGYpI`I!tvpLFhD}BvF zGv@sG;VR+x^=R20=4rmj$TFm7zXytgB^H9zwi zvi!!^%F@%hv*M`i`rdc++GM)%w9AT*eO#7dRTqZQuEWxY%_(czuzA)wSIIMOIZM{O z_0_U(CvWi8a^}ZACCis``5JWWx9^w2hIXlGS}Lc$;;qti*Ilw58)p_{wT{B7X3V+7 zC$@GUQrULEvV9Bd(J9C7m&iDJ;Q2a%b+0+1STu+k`uI*lbiW?V{Y3|bllDijq#pBH zr{ZM&_d7l040Jhw5W(zlkftGKL&6?~-PGpn6c}VO!m`_Lo#B zHVqE59Y4lfoY4{Ij?oJ+;_hMB9xe?@nHJ-aXL)r)NvwvAk6}qh%n#=j8FnCZ8WNJE zt|R)Xr^c#MVnrn>;*}+&>-C0qO0O#w7>dRNHN($l57p%!}^V6Os2BP4>B&1N^&hn_C5vF_~lv1ZC3HTl_rH#yt@MPL0=CXAyQ zwAn%^WyWc`5ir&2u65+A&$wDT_Wm1oyW`XlCDt;CJSxnaeab{oAA&-hQn#tCmUI8l zcVsz6teIIw7_}}=jiB%R%DZI!FR(Rxu!pem4Id7j_QSIAe}7mu{`g^Pz#5WIkqbZk z4q4c?P%e7UJLR^U{$9qHJWozxr@e2p+;!s{<#0c}!H3Kjy;4@c^mO6=BWjDrqGO@{ z=t<52{SI`@(E)*tG~L6*;oIeom;aObUwyqi^DD0wS^pLJ@q53aa$&B8JpWN5!6zjE zU6Lm*{Um%Wa)4#v<>19Zxi(b>12<@2It$ule(w>f2xXE}HW?#4jnG3H=KuqLOBS?y zvh3b}5CaVSfE5c2hN&{nlZ82LCn|+@ZZMyo7YtQ;eRVvV;n!5Cpn2&kA(}*Kdg723 zTWtya)I*9X^=0IdkW>HAch?l$S!KM)zUAPz1QVD-#EyJb6~6@-)3xZ!(Q!YtzgR~4 zzaqn3yKG5L`l!ZG4g(_eY0K=jxYRNIB<9hnIKS_Di$s4t&dJk|sT}m_t-zA6cx$F@ zF?eRmPt7PNqQK0^wU(3SanRj}fSvXcOw(!K_6eyv?P++BY&PN<{r5cLG@A+e z$(?u0Yu@@nm}J76FfHaI?|HKZ*=WpL|HrC_W}HjFHny~SrqIUW#NFo&HFX^I2N@*u`W_xulPfFm&q{wv9%ZJ(@7jOf_#fBXV(2iirB<;&*J zD+AjbEgr2Tz<2`tH6Hj!{?FItqo4lfahtbc^ER1x?$y)s-~EHH;sc@;rkd(ayV(^C zG_%YPt+=35KLRYk$TFwQ$X%ZSH%8obbQsNifHZ(N4_-@z47>@Lcsr-UfT1^x^1}~v z7BIUcX@)w!`4(CGjcv-$4v)!uhMaTdLd1>A){nkKHvUqRRf8P5Q+|6Brt^H~Ef{^j zM3!M%=*FX#KDuwM9O{KeI)^ZD&u8V=zo8Lkh1+8OShQb*DNBtRN2k~FbN`!cwVfh7 zaL(*Mas)nk{yOe&FM^p--|qu*ecj57U20A=@U>cGmJDY{b-Gh7VMG zHljFDvf-Nax}jy{%*qr>pK&U~s+HFX)*FsOFG_dZ_o3sCHcD9npUcnbs6!s>IqLyo z^480)K(fX*pR_b!mVvyq&DCewJdKdSWjoCBXX*1ez-ZHZ2-!aHo*CxJqa)8LHk@9= zYi08u%#Fr+^$n{Q1kKFNs9E{_s!QxZn}QRkqRkrUClIEk^OpI^z?Mku@vGQX5h9Ea zblUv(s>PTp!%n76BT4ic-E92V&U-L0%db*`PU0-{5lkzp-}WDv*WM847$faxU=uLTY3IkN+&h3DXEUS7V;Sc1SXPv0 zhT{wgmScf}OnCU3z*Yre&;$u*HTKJs_tfL&7_qEgpn`Mhq}#r94;Ko zJiCZ2PGds4w-p{|)CsT4V|*HSx(H{mmvrfiBJu?!`r!#P$jc;~ZSGUh2PfA%Ouf#u-Osco)XM-b|Ruk9P=_!n5AG1E!RXDU)fI<1~r1W*TpVMmvHPW^{TI z=`}7SD7De(4Z+YD>r{!Wy|V6&*UGkocB7c6I!ji(_x-YB1gbG>W{dwZz7 zCHPQJWfn`*6@Me=Ux5xl&_OUZifJu+UM@}NoF(J8_Hf3El%0NqT>0TDx#uJQDxJP0 zygQ@xb_gY-29FI8!| z>a}8}W!H+;U#9a~FF^<8ltn&YUZlIHPr9)AO#_z7(~puVC&*WRoD(u=U7&oPptDmv zfoWF^Zg_yKzGO*P<1}te$uYxj@>H48WaW`Qm8`lL5zJJ*Y5k6SdZeY05VXAshH&+f za;%oawYv}1JsGRG{5v+nIwD0?P3o#&du9i(%v4Cp)CMb+q^OQ|he*#9henXgk&-up z(e}=w1v;IBtIqMwOxhRsFE77fpvSoirA^7!3=Nvo`NcZLk5Nl@eP60ac1!K5ms&jo z?d<$VZlPh?v}K1}iw%}~v5PN5DZb%*|JzSHtJ|uA498MC) zB(s5u4AyDiw0SdjU&Tk=A@nVHjvT5vA8WEVW6yyW%wfldV;C$eErC7`n89Jc50WMA zzsu14@{=rdBM1>ZuXWtSqUPi=egAvHW&jIYe<=J26-UK)s5OG=AZ8_DJaZ&K;>zn# z6D^Kp3?WJ(ag0kUPRJi+IaIkMqI^L6fnIS!CNkcj;TjcafCpZ}$$rY3mhU*B;=gD9 zYi^m3vp+J!@G2T3?@gE9B+Y}5NZGz$>U5jOz0*!3q24>fEGfGYhYo2dJEkMMnQG-k z6lqGE6fa{RpAcF`w}MKnW`<5Gjbk>_m(a4By`%&e;zqji6ZUPkdBPdHN~PPkZg>mUGlg z?Ldk{pwU+HxGju#{scEHH&2`=>%ZlL=`lQ4p36E3Pb3Fe%=$w`j!*_Lc@+UB+$SzH_gBLq%Q&mj^Dz3He7O z9TxKYt8q+`Q(H_DQ1t^$5fab-Nar-l6X>-|HMwE=OLunn)?8%am6)%l3)o(zolks-B!6$-ymG)RG|Cakd}1ff{Ga zAH0JAlQ-D8TMid24C}XJ3M~p$Q*elo2sr{&Tw`3y%>FAcjO3sbr%{a^*(gm1ZvzrV1OQE%u) z&X4jKXQgKOn9k2}Op#$Z{p4GpRy{nhqrUjGRWg6U5^0~mSQ?sI^&`}BjITzwLa@v2Q^6YN3MejxJL+&IZIngY zpy!zVPwSpZm#0&g>f;J14bu4J*T_*!iRj_$&ebo}ku@iMe4yQ0SZ2I=YY6r*LT~7Z zWL~gAVkg+L$j(Dn1;)n-u%&}iZ4hY?|6#A&V_InmEwk)2OiwW&IV_#`e_eJLuRy0~ zpAu+8ii{_n-4Kk{xDBVTIm;iaVoMuKrICi=;WvFuT5o-)G+`v}sc(9U{PA#OYf9hfB9@V@AcQoq7@6Jdf_TL?`@xw#TR}; ze)dnvGHFgv=zbVXN`_c+W;k_{E}2$~+nf?5l==Rrwc+LsAx^`TqJ%Y<@Z{O?=YZ#_ zL)Jm$JHPqhbeZk~e6-hh7(0EoX&LkKU?4g1$fmK$!P63YJ`7-V5f23O@gd;$-MCH= zcJAG;*BRyTMO#{48c)kBDn)(o>-NhrfhzGB53a)(&yU1^ z^MGx4C-w_?-pk)2J9qDo7T$d0OXP~DUWkpq>aoF6HCErkvWu7==d&=a4^9=C0(bn8vP z2HJ|B(g#SK!C@MotvuQmbr8?|D3fs>T-TKHL70}|e6xT$Z?b^TooLQ!Q-BAde2oeMt;aVJ2+W(j}rQLWH?$4O|sycnp!+cPK8xD(JkYgRXoHsGh(MVu!d zqP=$8BcXR-dmjF-bdQwDP}8}(8Ub&IIz9tI4p|1-A4ioLM`%Cf5HksZbK@yrX%quK z@ngB`v-fIx{gpS%=}Q=gk*N*4g)=wHaD(9L6&$VN&txZhO>R(tlksieleO=7rF{E2 zua*1m+@^AyF2706Iy3NUjG3f9Q7-vZewYJ;bN7pB9^CJAoYOhX_uuouV7~u}?XKq} zJkA_o-S+zJkfMOIW5M==%K~>wi%!*Gneida%Z~MwIN@YJCK(OsoQBf^3Uvh!#Aw*D zYoC5Z&#mHB4(b*vb%LklxV*wrY>qS0s~>f;jwqk|Lwe$YPn9c9I{Y*(K;jGz(>UXl zHU*;`#HsI;={VseKSXIfA%p9hQa&GGxQSIUMuPExQrjl`3Qm!M-Ro4R=Rez2Efc^r z?&(@KrqPaJAi`cHL%G`T(5`pJCHy4eIaYz{J*-7CUWFY~xss1glcDeYL41V7Ge2r) zoQGNXubV7kE_%bGpOK3CdAeek`lI%kjcK3LIO*7~^kG`e3vYb8JhaJ{9;NzMzUX=K z$5&k@&G;6DE67x0N>Z_R+SP~2{kJ5`3(p_s3HZza#zP`a>wB1vUU3| z_f*co+s_7!x>r@z>S{vz@hP8983_1e-Z;$WT{>+ldgq-i9ir!Ic*1wwwHp#D3@a!r zQmDh)`_K*mKDe0R4+#+xS6YYUbDVG%zqHIW?!0KgRSw}~Khil(8h6@s9B^FEDvwZc z&^e?W^)*2E+hO$E?WFAgZvRzh$dhSs-gr4jwXn*TUgO%r%twcX<@ud=raxW*+GG1f z+tRwc-LW{F(To5dFMIOj2H)8?59*a9<_Hn|n6CCSH6#x1dl1WQZjiloPZKWD$q_~R zqW(}GWdlZ^f{wuMVkj{O*K-h&jw192eDJovk_W<+nbjY7zs+SowAZGRw_hz6U3;4G z1bNCI@bRi@rDtcKKc*SnDv$jAjdI7&b|a=(8dsm4vZY*V+pE0hw(=x+ zGC}RdLH$qC$nW`}I8Gfh$#Fvc;SgcKNp~_l205S^i~LPN#kGxud~{5T3rl36Y(6?} znCH%@SVC5%85gRyrVW%KL}JF;Le|h#T8iZ z8oQ3_wM8`y!HmSWda22Ay_(-<5n2YH}0#R_NwGanl_j^0^<= zVver)l03A(K)yflZ0V_4jX|4Y={Rywt>oeS3!Y-@vh$-?%`-&K;J>;@W^*$idKw`E z=@^{eagCh2hdD7-2KCSaYA11RK(UKk#rCfwUL#_!cpb|8=hU97fed>dw7ZJNuK#%gR5d^I%@u zRn@_YhC4Y+h{5y*{~5t_`WE;Z^>_yAcU!23*8oRy)PMl&IzfDBgt^i}_xf99Th(RK z&@vx4*(&J|AW#^1vIPBag`ckd@Dx zFFU{VMd`uoT}owWSX6IFI1Y93i`8mb_Qns&w!7bokQb4Wep8zJuOdz!9)lcU7Hc*mYaLRoD2+Y+XRWQ8~q6qQ7GBWl{`21V z#&`cu`QDE)jWC2Oo^p}A{*~8bdQ5{f)Ypbid+thV?vvJ2@TL8y80IeZjrMm)*Tx@7 zci*u5y8m(+!3W%pbq%^g%)<|^RVp5un%iXY(lc;R<*HiD>yR{!|4QMRPRKw$=OGH* zZPVmbIq}>XEI%w}E*_uo9e1rBp^mUdu%2)k!O^~>(tPL+sXF}`vn%&Z1&fuoOj)0-+4!!(lyiJZ?u6Kixd9&4VszFkubMU({WN=St0E$ zjb^kTgAe5A2TX^;hew{AhRloBriOHs;XwvZ_>rzR{90~^qiiiiZzbsmA2eXP(&B-r z>%MQu{*q-<+PYjf#iATG{G8;%k?d@sM8>*m%F#Hd4d>)5K>3U@UK!G|fcJb$?zr_z zx#mx>tmpIJCa3=B2HEn7_sWB3d{WL{RxL~a^zUWCGk43rtr$VatM=w)r%0u?WW87q zc|yvcd9AFxW*L4jk>1_gq+?IFRGzs~n(MhC({6d>w)N00vgae8lcV2z7jAZ|APwXmr;9k`=FzEC23n=_y$#3&FqdKW>s=d~0jQe>x7u z?ylo;WY7~pcQTw<4utBhg~p4{3oqcT>+HBo1?CoG%H0^I1r(vZNS8nwkCoahH?2$I zM7gO9rzR}My4fD~>^r0zE;n}^Gx4DF!%L|%eb`pK|43vxl8)% zu98Nqyi$Pj^E^ppCzwklWSFi0)Wx^kqxha;9_F>zV5GgMGSlwIT!n{F;6FKPg}Q=J!-L4;NzB(s#V!<(M9W#Tl?}dsPLdA89+yRJWG04A1&cDhm@jU8#5b z-SXhTGTBzVT54OG@V!x^&R=I)ZQ5|Z6U*|It91d0LaY+RIrNsxA=xuWPyNIpBU8Ae z0qVU6&fEQpqyhJe;Joo%mCU)r6TaiF70PQB3VR$KA@1nb-`^_}cvV`t^eI~CtOvOl z3Oi)E6(~cHTwNvsM=J7~sFCq-?DB&sw>`=Z5qSbL^fV)7PUbYaLZ2D(#3A7i$$z`x zS%&VA;k*gfUxd#48g$;_XMOl!HNM(Y?@#QYH(+KU0`pAk0HE2F+;9VlmW$#r^=&u4 zQ2z49>y=L*0FXe)z;5r>9eZTeRj<(V$qp-)EtNI5e+fz8W5=5ie-M)3h_|v-%dA?c zxiG+u3Qz$Ky^s7wYA!TP#c)V$Y%nIA!NuxrrEm?O-6bjb{7R+ zjqT5|b1E6g6Z@@3rpNA2e?T6&c=3&mJg+bYU>o;WZl9$Qp)wtQhN^*=1pcjuYh&i#*|NWnrwRb*4E_v!A ztVfMGpGwgyy;us6G`aqpude-Mzg;uy@i{P)`D2B4$Wulmj#n5Tdf9m&C~B74{=?|F z*CN&euOXJYUQ38_+-ibTCD6*08z-UTzA)*y@5geJT&G)YNXyG>UsZ`3Sb&da3(9Jx zcyKp5{jC^mh>wxMmq1GpF2L9yg&`iN)=l9**%&ly0-QWTW8=YPC;E^hs>5<1 zgC`L(s6OC?%&THdzuNpU+V&|tD=v|;)-w^6vQIMx zng|Kh@5Fy@wD)iS@iqD9Pkz}5a9n=+QhDE7Z?Fh)o(LMnIAZOC+X*J4b*d4|Eac zlIV>GY17?!oRyr8JG8jOmOo_wi}C8Y{=%1Gl6)JT43U5s$Pd!wKd9 z3x_{uvcqr+p!J{ABKR@l(eVap#?Fq3X#O}#n6`wF)Ba~FLs3<;l=khy+u?J~SZB(@3}5us^-Se|ltT|w8Keysn7qrCY`-s2ddIPUU<7INjAphV#G)=XwCI#f`~ zn+JpMXlXFmnTUrSfj$~Ey~aOu{|hmKRe!;sO6R>_iqmEulg_(kx78lvnx8;IzIr{d zVqv@%PYzTXjkfeyB&22HY=Raa8LR^l^`v;2;?mR%KOj=sF)}|kgiT}azeRRcT_Uxu zSn3O-?VQ`rpbs8^tY`kCpTspd?!H@W)X7uVACisJ z5C0~fDV^KMSjPbeL5IhZoZxObWMU$lYH=214DY*N4&p;D*O8{p;mOgEdlK_8H~Qq+ z3FH8ZN6U60P{XYLXk4gEuCNqqoL3#*Ej^eX!6_D8O-bcX;ZKfL!B@&v zH39P6{ZHtfI6x%`O3082`6JGdNmdu=co3IW=wuGLt7h?7sgAlIsXkvfWEvmt6|StK zndyYILvbGPV}50W#)b45^4IO<{PvFGvvjUKcXibgbRV+vaqhkosg*N;*0 zhF=b?mByn#!@Twvp-iw&yfRWi2pK-L4{`j);fFuJTmJm7J_47`=N3$j{JVF(0lW0J z>hu`aw=%4d#3dh9E=A75|G|Tsq-XQbWX<4eIZ}QOHc@K9##nd=z^SE$?nQ zRQY-7PG|m*hW>c)FYE~oPoVql(EWCpYpY3|$-kM>b7gwIIz;~E`Ip4eg8cNE{pV7V z?7UCxzEh5_`=%}x$@!xv6ts>ct%&9s<20UjYOV&0SssYdB66H|r$S(jB0M8gBzb5M zV5IE0vOa@~KZalnZ6l2(#Y8hw$tLt;)FVaMh-BVso!4HCwU$eDBqy;_`ElSVliEpW z$|F?0nL|pW*#KsfMSD{?Z3*^7I7vF4QJK`8!j)$-e$r45VW5xDbxsE%*z5r$uJQr| z&cL{-0(lu!24UK#fGfet!wKhr<}LCxr_B!qz&Hl|Koui~!)0ywD3718vmUbDw>H^f zZd;|SN^6*xrdB~zLtAQcUYo6WMH;TNJ%-L3ZHlF@I&e;?IFpjLsJOaS>PB~mbN0xX4s{!enhfbG;Bw}0$)A?`t zKQ3cE`=o8xXQlScD{VTK^E;%htRSRu*~q)sJt%+nruSmHl}*#DsjiX_|JCc|^ivk0 zquqdqM2@th)6O>q43x=V+UGQmG`6&T+kY*cn|~=k8+f{O*Iaj?=IDoWgL^h{zrD|}2>oE-=5xia6b_Cj~^eHD29g;4M8y`nK zIbCL9dy71{(bm}Bv43NXc+n28*j(q<9#+0AuudWTZ;xUwgf?nr!;i zKS~Mg(dSU-r}Li@^5|ir=2v)z(E1J8gbdkh#As93G-;Ji$;~hq!YIdGppw~}kBW+F z*@c}{TMpbK6${Td0l}}tPwT*0V(O?F0AfPLkrsHB%JCE&?b<8DMfFmM50p8Nn@d!( zh#o8dIWK*9uvc0i{kT-NpCzS@OY?1Wdo=hCn+#FNmU^ zc$qi}2@so{&Oyvuktek4Uc}mFz*EAc-LO>al(Rb6xmAs%W6hdx*OaF-%RJ6-9^(~~ zhigRY;Jj`flP1F_*Edr~Kcy(3lj%#S=92$hJ;J8Vp!1HFrt$pib&lil&3R|}u(nw3 zU=dngGTNG#kYe$n62|=2dG1aDUTuJJu2Jdwkd)P=7+LU3>Mt;`Yl9jpyPX@i>8G)# zYqWc>^liCA_P1Y)rM_BqUOPMOocHOCvg;+Uud0$~;WQC!;OUn_6=qh@Y6k7V!$|MQ z2M3T4gfc%>28574;4Y6;ObOI#H>Vl6{aJb>nCy^0D${lGBon+t?~uu^E0RtVVfusJ z%8>D&(x%ZLb^JFXmtTIl3=i8sG+2k1NY8K*!y~F3fS~PMH=c7a?2sSneLj}AB~La1w2gr-sGWLd z2j(+;>+~3|VPC@a*pUG$Dr7O+`M%yR*}C~*YSC~G=C_w} z1s-SQ z$Bi>KsEl(85NM6Ud8mNd+L3JtvgyZqZk0d(Mv(3#q-9!#_s+Q4?r~{4$1N>!hn{+z z)66CR3(IRT^8OOcmHw)9q5nT!KSBG5)qfINj{qwJIenbSOfj+K5E{%amZ4}SN^f&~kH_ePw61zwjN zvJCwJkvJdLSZ=zT=`N6%L?lm-Wfq)~p>aj(OU(^jVIF@e+{{b!o{6zBu_ zlps$Y)M%Wki?)SRb=Os>P|}kj5V-wT4%kfwT6~dy)LDqp_Nm&Dw3^xYugm4)8@l5CKbN|``|%+?=e5(o zv?MhW5rj0?5;@Rt=t!qL|5b03J^Pc*j{f4N7t7NxIUj4P*XT$)KSbqdH~H07iie2k zpPr~-ikI<$qtf}{ZL)2kRW=kqT^caTUR_g%m17#h##56F&_D*4&c6TNU+Yqolv7br zC8sT0iTfP#i~8Lyb3?reV?iX3Hqjgn!Oq~e`3&iGg7g$i)*wE1>|+Fp=3sc@ zcidx5P70ruz$=17Ru=6Dgc`=DUdHh^FpSql_?A^GlvfhUC9TVMoRIQE1)Qoe$m(;k z4zv7O`n+`N#LusfXj`zyK2WaHAl~!EA!QYoes1q8;x}ou$+ER?2d9mE-P!D24tqQx?W3T4j`%My&@ z=d9(v{}7fS#t1BaoQ|P-gJHo8tv9SoCRYuU(wcUu@7yRo7`f$Q3hXGw5mA+r?@uD! z2!fg^{m;^8ufgm!nkNtb+cglbnc#!JHPcflaX(4racSN3t)#3>S2EdKaH_N)xgXPI zE+Dm%IZ&ukUO<-2W1tKXstLH$i{pWiXc6P@*w@BOG zuSxCM&&7wf6~U$iVu#U;9qkv~@=$zUfc@M6S6KJ6B_!_C|d4Tv>rP zp?IsqDVMRnzDurESwDQX?5nsC`+l@y1+7L5`c&w*1?d5rwEye_ z@y)@yd+&_4oUvk+uD!2=Tw0KiP5H+PLQP&-fO_dU%q_)e)|^A0_#JmIaTKk;tR(bk zt*D{c9Vg^k??2x(D!<%1F4)jBU6`yJMFOWYz8%v@!7u#3v-Vs5bY9n)hZC;)r8Q!CK=P6DCB|O2oWDc$Rlr>*;p$J7@B zF*r1OoCSMG9%Ya)_~Yq5AKa(q&@YV({wYuyWJbv7x5)I-@+D+eFw^o-*Wu~|%i z-3Cpc!QQNXlQMh%xvE7G=9y;mJ?+6s{Md4UZ9Plf+kRHYWEBvUHao+}_uh#bblkDG zN2sk{4+u5zBGvK_rRo=+R$?O|0joAURZADPd!%%sGuGAHD-E?(pa>6sd36M&VD1ogXU8Zf>8ddcO``?crLWF^-KvqN>6 zne{lSM|FzY(~BIKxdDONM-sa%GSM*j=(cvum8(9x`?Y|#3>!8C+p zAn^3GT&18s&m2SyckPjZVthw{X_K7S&CaU^(l7*QY_s>Dn$uaxWm_>XR0% zqYgDv8%ttVFe$$KPnZpdZ5X`jL0t6g!MVL0jWxBf7eNDR182QNitU6FZ z9<5Rpp)uvTkB(Wh&3b@X_8K@+LUZt^n=T<5r$GQAevAtP-d>(22Q`O^@Y6<)vKW+} z8h}s}&qI=*FonVE-WayG?%DVQ+12n&scpcP<5>2wOm7VNuZ=w~%9kva@-3UBzYn96 zE-*j>U+>@a7>S#~IJHF)$B{~78N5fOP$NQ;rCTyQbb)+Qq?2I;o}OhpKYG1|Izcwg z7m8*eZ#tuZyACEZ-_y`e^k|1*w&w#+PsIOnbk3N+44&V5a`JH8IlwY!a2SzWd$qs- z06+jqL_t&#!F=X*I#gIEqeFY+wP8GwOY4HwB`b8t{aRAFtUUZ?1=nPIYZKNdFTut& z81+HezH2{5;W2&2>x|YtBoH!aVSYp|y~}Xu<~dZEQCPw|WqJ@4S)9-z(!H*;^UT#<3bFdvaDTmwbnV9;3_ev$ zGYHg#A(Mh^1Bk;vTR{FCCyXKF8P7+s4CMZCuHRk?ol|#%+)Rh$oeXCj%V7-V-GZG_ zE9%>%V&2&#*Wms;H2c7Vwu^l1Aiv?={~$m9Me-5o)t5h6Uj35i>3sH@8h(fhTk%Sp zJM@}g^wT7o&sr;0ALaJ%xLbPmZjfILT_I9;x-`@`>MphA7->%rPNn?`=9&!HH8j*O zk8FG(Mz36Tq4IhrIIlwz=HP)d`t6Ahl|H7j04+G=Sqk^MPB@oC=o&n+vw&sn*Y>wDFR-v7Ct7EJvE+{}14890~>&Q%iY{V%vK;pU_qeH}NA1*-a zgIS#Fnna&2oDp-oa62*VH)>z%gCwxM*~!Vb&*HQ zm2%K%j@83@z3 z#Y$lrJMTBh*p6RF=Y6;6h8eSJ93akb-qc_>A?7hkf;KZQQZQ3BN^MtK&{#e*p2Cce zae+O^G&zA=WxFnyNqTF#O{Vj^0=JVg#<4bU*ZNy!f7wc@nzury$gtzCn~?Zh5?2OD zUA(c@E?;RMCSjCPZKZM=%b(+Fklso(j=7*e7!;x6<}0j4JY|s1py4KRDt^_kPLqc+ zKpCKE3UL&MpUMtUb*60yaT<)Oukt4PZj zr57M6wUBY(&YQf>I4U}?tx5Isz$GC$e>ny>ijcOb41=zhu&l0Ks`@vgJ+cpkyGWY1Rrjnmaoo^32l>G3qN18A%8>{Lt zZ}4L=I3%tH<_N=erjA;HIZim)ZwMpq-6c!4Bfin$@F_1r=xI_u6O^>PhgX!A4QLzeE!*T^NlaYoc3Cs9#e;9AWQLp z!H-nadEs)jouQ#foxrGn_r@Q};l11Br=!^`R!oI!Sg;L*lEK)(?(F_>Lp$%Im|N-%OUR{30PUhJniG}|9E^(d-X5?h%W4%gs!ycl z0GU8$zdxyX3$b848Wm-wZDGTu=Ok3($Ppx@?Livt2sRP#-|{o*9meJw3!biLBV6|2W5ev=kjhblfc#^k|ET>xd}ZW&E^V4!BxD+(Swc2VnT&;QY2` zPn8|mgoy(Yae#uX;BmWD7vfw$6Ns18&6mcJM`dJaSU2P01)!>wpE5E(oN62)g3<

2e(^Cuu1m#gl`38X(_3a*07s#puLtwj+Jm z%{CHKjtV1UMi~~4X#c4uwD*n^W`7f-12Tb!g2AfOu#yT_M?ohv*hgQ96Y9@#LZ=Ck zbm}W0JKqHd?v{q5zmU4Ko~hHXvU;=URK$+^zkK?u@`=yi68sQ3`^;12JvYA+yY1o| zth(y(F)DZZR_m0GvKU+@{U$y(+$Wt6d|UP&8IWI&KS!#XPsJ)>ZJ5)JH#g;#nz!~L zF1vnUpf3)AtJnZ3^S-khcf1HnED;j6DCupFJa!e8CYS*a1o8+X|;2p5bZQ$CgJsHLI#p}oJNGWfOD6$ zE3RB57uOeJ-aN_KAfxdt0uz>5+q&hb-2Py%3^;Gp8SKK<;(MB3Ano<_Ia6dzL;jHu3hcadRJitOm&?fhz0!!cQ%4T$r_uQg zh*A4f&~1nihTFrG+s=n6mK-62<5Q(4I*got!j_o##2MxGB^D_?6V(`#k591>g9n13 zpRwp_DV515xyjUX$D@3WjfX^A(4UwQXw3hncD%C3#wW~IPE}$)&-D?dn2C=w!XjZu zX0*5t^C$XZUGWEFF9&OXSYBS1q3*<~3b|E-WyrPMJ?zAiki5UJPVg@)A5m6qT|8;< zSBwu-OGhyBJaiPg>-1q&d4u4rT}aQ|D)(LO!VxK_T8VCniW^$W7KMU$O~1(%)<%mPH3Bm zk77#hm%nwpeBhrx9j!ZM@qAsYoy$Ge)>KI?-h!4QKbSXsn-l$^!72GnAI04Iu7|%T zTl-t(p`xc^N#7Q%5Y{MFn1aXU9yv-*U(+G++hVl*bq9dPti!`1u;H4E`x*k+I#|Y3zY>_ zYyfak#0!S;UVD@#T#$hSgm!TfDrvE-T81rxOZ7DGaKCmH*)5X8KOXMHn&y8YMp|p;OGVE^GS;|Aic#Ov*Qw(UDJ+f8Q04I9_9Z#X3Yx z$Tz}aECac3Y@V&ClG&%i?Q)p4&*L3Z9+weq+xju7Tzmnx1h6^zq4<0XI!r@Jz&=6- zE`D+QPk$wEdhbW|oQJlSM)}*fzgFhAH%UD{MyxI{es5pg0zfezy@v-cNht|f_vsYcDQ;YbZ z%ZCUL&g19PL8o;{+R5lp6{J5Lx{!Bt{0xBxiPjyrEMu7;W)dQUY0R#{-WbC!hF_S|&NYXemCh#Qb zD#s5lkNF8d_?5lA#^Z&a@VE;hmylwOl9va?rE&EWanVo3(-L#(laxef5|_}Xbx@}V zuJc}d?sd|A|2L=ayaxx;{%8Zq{KLq(Yqv%dXI?M79B}RrA7&NuK}S|VnzW6|XSQ0+ zwKAbr&%uR^gCw88s!nD%Kgq(~+F{6y}n`l7i1k+=xmz}BK7&PF4$akv!eaWUBJ9fkk znC8u!hmYH;e_u?Sl^x#BJ0w#Mj{3^gF^Vv%IbK+f!J7f##VjJN%7hkxXfMekvJd{e zHYwMx(Ft8yCf#x0v+t138IG3->wq>ICXYuw4ceIIHk3EaliGtD<><&2_VFg{H9h8w z>kh0rzNxgRAIA^)h! zEpo`RwDys3xN9Hgw_hfe_%Joy0ga-jL0lps?HC;zl*X+emx{VpshoeVYBqLSkyE1N zzz@?C(Z{%J%>(lCxBQJh%#psnwpu>?j@Mx|qIuXAwN6LdtI^hS^%#Ay43)usSAP&k zICua{hCX_y+|~ai>8m+gnz7s?N7^}`oo_EV(oQ-LvnhYxyEi;^F9u}#l#LHlE6%wP z_0I+s=m~>+m`JyHKUyyMbzybS-g^LOb`6!mjpYE#a>gw(~b^7$_bJX7RuFLJ3 zmI{06i`(p<{{?Twibe1NZj27{WjrP}5!Jd&8YMc^3W%$0A4(!cmY+H6;?5WDh=VYr zPi*D|V79>eQzp(Z(Fjyf#sbfX2V_<;PCHYo)<|IM0v``3EN6? z0uyt8QbJKqr)PSS9jp`+qeRc8HIc6pSxc{&`13-K{NX+K*bug*>+QbTR=0D9fyQ&1 z-KCs#^0^+K>;KuVWfxy!!)w>$0yb)sgIFVs^rdiS4NSu^;kJ*JSQgWW8FtL}!NymMBL@%nd2{@#( zrl8GZv&i-V@-)(P%1kD61|L5-Tb^_l5!OQyK8GHkZP=gvV!je$kvf5hq zf6k7Mjba8CdxGPt9pjs*#dry+tO4`SRR2<%Go5=aS!N!J@jTUIAzMJE7UQM(WX$%M zti%oCBo>h3IxPE$bk;%ztH3!FRa73I{;3iD`m0F=D0~uM51k%4jja*aTML{}-n5;| zc#FCc@_o3*egCgDOrONds6QYZEBRDMHYIJkhlq1RaO)F0?T6m;Yc_)S78v*i)Zc#p zPuivn*1&1Udsdifuh&;my)TIq7^xM5VbaCmzWc5Jz+?8c!I#-Ic8+Ps3#goFudBxr z8?E1`w$Hu&IpMYyA6|;E;eySuPuMyWv+dMZCI%3S@xTulm;sqyj0avo*VFv+25rEg zTWoMqIPUon*U(2Bv#WV`%N<^H89146Gm%FM{=!3MJi2FaOOOsPvj`D*E8;^3DVtKcn|iYXMA*Cq-G-U&(`R?W|Yl#ny+T0SA(NQq43&A58;Jky$&6XFCU zvA--#CJe-j_8~h@4@it++?SM)Ea4?m+II!8Q#`9C@SwK}tb=U6{EgQ4=)W3Ym1?}~ z^@$({>@~z?aJ?dqbCsV>o5R#(yc2pNqv)h#wtRtv=;>rC&IuB!P#&{z%1n~+QsJBu zniJbZ-$5J1_F#KjZ?$Dz-PsbE(%BetnlDgB;&WD*v%_@axwK<&a8P>?b0&#(o}u^o zm@U$DwR6%V*R#`E=a*{Fv=!$fRn>V_n?pjw0{uT=&M*dIAq8zRQC5qu2xGaT6XcM* zkf!ljniuCeE7=julWO)flKJxD{)5M@18prn$u4KtAC#{+AVb(9!t&ZSE#Ije*oFPv zuZF2D*zYJ{UbwLO2RW-F`65BeD5gZ*<;-jm6~sXKr7X`c!Jfs3guO@-|dEc6%vuE5>OT&35hWu{XZ+m#nwXpC)i0_5X$!QLnoEVr#>? z_7*IWsndFN-cS_Qw6i{{xa*aU^}w#%t@p%X``XaWR@J%&Pf#(_jwLd+b$kicZ%QZu zYv8p~q$D0`4uuOrY0DpZD0kd1-f|UQ~v(EdevBc7F)#4SCa zS&QfY*`jdV^Ipb|R`{uXQx{OjLp*gr%@hk4mQ$TEJiM8mf)`>MT*3&6qfR+18G$u3RNl(!THc5sm5_iORVMv71(&7vZ~w~umrLlJA<4&-6!0=2M*cg zo7aOkokfo2kb-A1>QkBzbw>kcH5{}0-bZY>eG8W1aQQf-~Ko~z~ z;m5GMo>#;UET)b39kL@6*ue=OdgAgZhOpZ@fiTe$aXte=)=9hzS9WT7%nx^5}+UiZSA;fG8&*(azNVVngI$8}W=mO7KQg$&2x@I2`wA z+2{;}5sw}w$WGxsL|b<$ii2jyJv~?jjQ$fa(sNjml~9G3pECcq!0x6kcrglL@W@F! zfc>=@iu`KvlGA%%v)(RVR)eRcbl@>?@YwXJCwlD9K6e;1!7w%$xWD(K8*F0d4*QjV zJ0>1FgzVGx_P%#tX=@HVWxw!mSdSb9Ko63stigWn-B;MA(WCakkM6QVq9ccAR)4X_ zf#rBvgfvzm8Y5IfN`UjPOu|BrpLrJL4El(NKA9eAVjzBq75PyXWhQ>9$dq(OP*IIF z@0Y&;Gw+|omkjBoCgYXFCtu0thfL6{&ceeX;TNHBRJ!L!9fiZpWGLxe-g!|49v|yFV!f5uX{Q!n-=4|^W*&<1Je9VMOKGPE583jk|ED!x@Di(R^cRpx zL6L$o;)JB*4EvjY>Q`*%p8XlapZ?J|*sEXmLOJbi*!HmoA6TkFyYfX;hTxeIAi^z6 zPjCOc^$a%J1CuYa7Hsv{-nq=08e6g5;~bj|1e3W06XVDTh#M2sao3LR*5Aji86XGb zxfJW$egF4#-OKck$g|x3Dm3GdvtnF>P(lf9u^twQ5~B(2i84zV2DVHAV_h^%N(R5F37SXp7NO#q#g%;$f^YJla3{ojH_f&nh<6j zn8$6tnz2-dA1XEKCET|9}glTEbem2G|!lm}c4y(a-3n=>%vK(|AD@GralJ%^Wd?8X<#*J>t>j zeO&tZWSeqG#$N7^J-J&nJqM5I26uXzlR0iPq6E@te5V35=9!iXsA|S8C=*|`se!{b zUC-?XEAufNOk6S?DP&4qQ2h(|87zm4RbIES_y~5 zl#x!^HvF3y>9yrMKVtREH(O2DM&LPJPzNMe#@T)&BV+c~_x*-F^w?8OR(Q+nZ?-qz z`YP=H(rj%lSRTVQ?YNucyAcdtY}A>&uzY3kB%Joo+2ez&ZD-|;aM(NbQ6Xv;Anfm*IK{yfc@o;*Y)9~>foF(#&__7|MN54AF$kah(hZ6(oHO`AZz>}-hErJucAIIr{Pi~d$S3fT zDHw7*Hsxb+8W&=^JSZc{C9+he%K*N(stZq|Bsxbdq^AThb%_ZnFAE`irYho1h%@FD z(b*uc(j*q-KFtQ}v@>*(mvnshqbj+e?fPY}EUo+rB{@+A$NzXmuy()eY+3`H@)A4i*FR9k|ZMBJx_1G5Dm$`+EPK{DkSBIh#k2BPN7Jkly^s`qa zLMAv*xtz{B#{MilNlB^<*&#&SE5>8gfw{q#apKZGN82fn$ng_S;h=OJR3_7z;DNN0 z1IKN=qS-3yTD43@%M2G-e=dg|9~-ugoquR`4b4`+=Gr3rq<6n&u*8sgVtmrx^$Q=i zuiceztN7YizR2G7mhYF-&TStX>abKUJ$Va}Lk|JRdkB%UtQ-|2tp};C9{B(FQi3y2+#5PU3K;W{MaD^Dvo4!0d)h;_7Y7? zD}cDvLY5u*@_k=zA$X5ILPwux9g>p{pIcUH52V3nRuLF(C?Hw;_u&;i-$LOxFd%-$ek#Mzx zOjU>7v?kw3e$%zfXUe;BwY`X6=N>*{pW1<)Z+LN0-Jf~=MGuytUqlrrRA4B2b562`d-N%SNkG6T|OFhvDQ$jANK zd#WzAx@FvL0NaD%MN-a;hmm!{rQ?zYo`)Ln0pz0+agNU1=BGh~YG=a-ProLn(+n^0 zY5wH2(gfni5RmaAFUH9ypyC*eoIx#qgsBHxLi%3rIWB4VSLCS^%FjTU; zzKt=SKITL|o=qq&k|U=RZ>7y$PSl^!>yX1DZ;HSlxdc=kL;=G&3Dd0~G9*v3o;oo%a<@R6R@n+1j`%HUd1DCh>a=8qtBrK+w%3}^R{>6s9<#r_lY5<* zt>1uk@+bG(9Y=vz*JZEYjK{ISCBF?ftTfK2pLk-QJ&BbzHOp4mtLvQ?WqjSsS8<0i z+jajQJAhH4KSKS8$9Rc+iFcF{`A!R53YGD|869k#YmsLm9(v|jWt^}WkNi^T>|L{r zv#boJa*O4c(wCk)i~0G*4b=o*ZtC6kS$nLv-TKGyLfZr!dED%Iy}<>->zbhphKm$O z*2@t*g*0Kzl)^fQSXhWk6R2CF6Ln6(Is`s(cpkZfPk+PuCu(f0`vz?_$R#pMANFy* zFCCx%8sLzb>&H~4=eZu|Seq)VhKa?*1^zhn0{SrYG}#%BJ8}tFj57^CE|sA(wQf~6 z3s5+47>;{P?7)7igk)DqLmMidjFTe4h*#nT)rQf1*m|`OOM21{PL7FV{E4elNmu&w zTrp3%5hp}&Bb5x0CuM0kl`IK*oDMbv3B&q0WD4^F7UPbMeAH3-s=sqm8gz&6mltfH z>npBELtz+_fSi<>#5()Bo=;fw@OEq7 zd^4WT#7pD_x)RMyp&$M3pZ zVT^MFQ4%jPtbWn*n}fK31@wy<^#~rRon$T$CoIMz|9oMoJUr))1bvTx&bCiqW_@LC z_#$kD9*6ryAQ={l$f_~`(va~~`cRr@x>KHV-P+|0h=<^OG7g>wZsb6EH)N?0Nal&F;gVG14-v3y;(w88zn!Wvl`h{c4G#|&1iT|XdcnJh#65|zBm|bn? zvYJy{(+vPh)|y}13f4uK86?^4;5boG*cd|oGzpQQwh&HnRFX>Z5RggEK#*}hunwq) z*T4x#;)oL}4w(b``v_#}GL}zc=_tM&8?w%Xwb>*YC-21@9597|<}sxe8$r;gf> zoZBrD)B8bb{dw)4#P?Pz4*Z98ocOY}T=E)-Q#+G2i-=+pq_h9g|L?Q*!9V&t$Hj5u zx>fdppMR$<@9MPnHaP8dHF)aIU0OhtbeKU=m0|nd`|R|-t@fqCSJ`CCX6zip-DA43 zlUI`*X};86Z1Z9~D|l9`^&j$P5&YL3^s_r_pbehor-PM6&R4fzv`76jH_gNpLp8tNlMa%2ze;*nb#jY z6_OsoO#B3;2j(&}Lc~RYM2=X-35#*hm!6RCy7Bx-6P@f6a0!cX@gxt1fPOaf7nN!!MNK~_nUC@k16u$^5nQ@qmN8H#E2wR9P|Na6e3#WVaQfiVzU73DpOs>DHsoX z_8mfm*NsUZ|FZ40UzEvVFk!ttV~L*IfRK@p6%(~XoQ#!t$6J+&rlMtVh%( zLCW2~s8p&@EQkwgD3`_@XLa5Us28ZG7ppdnghkuG0hwc%C5RXKH^UQtw+I zx-0QEy9n#swHyaai(FY^=Lhm;2#6?k$l$s}7V=!j9w)M75)m_ronWE;v;N6k5D+b~ z&LAllYlzl=9@ihuomyQDpP#DEnPDs%2jnoFotV8WAmjWC=y42Mr4cu19Fk77NxLcw zgPUob+0l&n^l|&|pFSvzRX1E|Z))@Wfulomup8DbvkjDkW#H%h95UXvxx;8q29KYz z9^y?8*yrx!@|kLT1$Lu($&D)wyA#{qNA}n;f?V|M#j z4kdX)t|a?2tiQ_(){@Ef=yBH{bvWY(RGkwtFh5{1?y0l#WjKKs1D?0%N(0obI@vK! zsG>bSSNls&l?@z(iT)#S+P`4CYp|E?%4@Xrq`9#HGbVT{k}k?*>jsK3FNPNfkF$qF z!AOx+BMvVZesL6{7$_zNiQMafWamM0UR*#2mQjY-<1p~}=j}-4MypxAMITw>e(g)^ z+LKO~z~a%sJT@OACtVIarZJRKrn&tf?G?{ua}uOo;k0mq{vH&@pQegaK^peN2YLFj zgbK{&({YQicmH8HcV33*x*}8yUoY;+(Rf9bkb;4g$vK zK1K4}^(PClDL~dkHYh>H5mg$Y$K8H~O>&xqNCbv}h=#q&@C#`~rQl(=%ct=1rOG;` zu{z94)nar;&?ydCKFU-Y%f@eVxYs5s@nvGZn2OnU&K{E+>l`rZ&nFENSm$jg9@_Z`#lfaJO8f_BktD@?ill-U;HV3MYp596;69St{a%W zhT|Q&P({QGEctHI6JNHYxSs#V=&P-v3lnRYX>V!Ahn2A1qt@p0CKLG4?g3R}RT>8u zOMr)T3K%-{7>kYi#}kIf9=!47{=}tLPrCj-JItB*>Mr}S zm(+P#D2Ft9c~tZ@_O4g0M4H*@DEkz56PD9Ae0Bzq1$lEN{d< zb3bz{?RD?E$(TKHWS{-xVLG~bT|@R_+PT#qMIb<0{c#*J7UPj;iW4#?$*wXAap4h% zvIj(5yb@%**oKTV1oSiNoPm&QCp}KcUy(lTX(4% zULYlOW0V|n$;@iLEk>jj;guMrkCdZw)42edkIHXe-jQLKVGGL}D=^bt0mqqb9`z1LT26*> zIqOvK4qLgGd$r$a)op9R1@>0lTr+%gnSXkNnQ_oPy!|PA%X@#rM(Ok;)MK}p55E6h zcJaowSl5mZEODltYuec{NKJXkLws=Lhw>i(f*m|PVYg4*j6K@dU>nA6YiXs^-rzgo z@a0fdojmX(2#EOkv_I=}7!!Mk5A~3#!lnH15}yua<+6;(mE<}}wUAC3KcDuGen7@i z7GcyopbB23hehGId+%niW-BMCI8Hfu@Y8)`KqjMiXjZ%k73Y9s-h4kg^vp|+eY*2z z$(-@k2OJV)oR3Ti|KWcUuwVf5-g(;9r=<^gZ=E! zyi|_2O`f!med;*UVcNL%edM5h`Dwg`n!@OTz4n{>kiVk-g_memCgL&$2{NviMH5t<{FzVQ_{k_H z7zQ#R3RIaHoU?owaL%+p`9<4Zd%e+VU$$(SmY(oyp`3y6fVzpF_cEnXT*->G=yt4O zqzTE-iR$Exg@dmoVIe8w74*!Y^PqE;g>MA(?zf=>4_i;mD=)N@C%&eIm=el%U z(!hc=aO6nP%L>J}K4lX(vcWSd2Tb;OVfYZH+beKWr12o#nN|ZG^~i zDAc(aPb}uEKTG!@K@cURFicHQ@s!16hGf$O5qJI$6&GoNo;(V_B2L7qzN!gqe2kCY z1uBbTJPWR*>{PZ8X@p@X0}^Qp@09gUwqUj$jx+vPmdZn3K;m^G zt-5m)#PRwB6h34O$m`{fJqPTK@A_5i8{j)+5FJlqf8*!gZdYA)p><)Vo!dTYuXbEF z{KgOTg*r@)jbM-V&)VL;T6>`Ewbs^!4{l*SdrJ!@iD1XGEqp@>p%}I+jdCoqkT1S_ z-d0>aci!4@&@pgO%3)PUgaeG1Q|VNf297n__84xb-jJ)m7TWnmDk$qAN;cY>(}nGPprS$Zo8<#uDa#L_V?GFv>nH=LcOxa zHeAqd4J5{bn$Q33o%R6N;vrz3JT-sqC$H?^b<;z{BK0yJPRD-L?*kXhsj& zC%-iW$rctCTz@D@I)z?jKp~h^6nf$SWIPST&}m#ZXO)w59{G|sc#%!gX>JUIxg&`# zwVh~74<+(U^dGhUr?6HW%VVmrJcciMa;BZjPU11Bvq%PzyqSP{>SZ@pz)#gb7*9#w zMm|~Z)dy-AQR<4bbqvy&s^RCEu8@-(H@w_6F=GALE^JTT_13&>U4C3r)$I-9b@!=87;9~g%mVN}?Y&%He`2(w~s%94e$WbUb_G~9) zU^$V-^SR4Q_R0x(%<3Wgh5~g>K(gr5HIZnV%z^`1F>(}Z%DId)1?dWLLi{xO6M2AA z6>f+KRXT8FZ;6jEfu%*m<;%6yq&S%o^`Dxg(~fI?-yS&aAF+m2ms@T3rJ7I|1I(&( zd(b>VsjA1JBggIa@BC#udV)(-eK(=^|I}OTMK9Q5?N}^X>7z!VA$@FI^s`(?ZjwmuZ6XTlg@RVS^r6U`-PjY#204q zVJJB6YJZfuu=0|uEYQ`T!LzkLeX!^gB#-!f^(W3^42xvO-J3Ry1!2VbX>0v}60lrp zq#+(hh(pHdv;}krj!wSg1YSVmP2qD|qhGWSe+Elmz@w(E-7dRmg0`D$#3#Rij4!nQv@;ddBdC2Cdt5S*FJ(!P zapDJ595Mw|TY*pDa4%kz&HRq`dB zl0dmx8|RTerpZX6wfs0ak_f>M* z8yg$7^RfJ2-%hJ%mG7pAowfQkIiVKJ_uS27G;slZnt z(M~9rD7y4({^ZWyB-V zm4JoTpR)RoP9o#USxoVeLpG(DkMfFkCYdUktYSRG3L8dPj7L7J!U?n5v7=T=$30Yp zG8HG}&ku|gf-?L(q6vQz&hc1zv(~mt+leph%4_Eom&f#Rrv3Mz_UDn*(hL=KdQ>H!I3vOCTy;NePg#ZP((DwIh`=o7qBwyct z3HEQV!p=9mG1O!d$_cp%qum3t*kYR(;~`S9&5QAc=4Z`*hB^nY=QEt`g*@+SOGheI z3Y|@Hj#mC)39^nYr`H{~mdarm+)5k8r;6E8IIy!0Lk6TxOoMem=4m!S{7E}vAO7$s zSd@nneu3*V2*dVAzxPS|qs&5q1o-#g{-E9e%?EAGidO4x;2@r|-ot&iqaU;ENfv(n zPXXmj^MWuvYJdJ`U$HM$58I=ZKNEoQG8@?Tu>Iy=?Xp8Vdwp$TCNJ_sMly_eVy}9q zGMPO$Zp(lmh>#a7{^c7Z{bl!-bna5Mu0r9uHxnzaNU&I@WHJ%+<(eWcD>P7;1k%b zt*uxOiEZPuT?cX&OR*&lEEWxL_-Uwf+;9+btRZOJC@mYA8dp4yu{Zbi#U7zT?p5Y7*Oz>fl8OGh~>CRaMnSUNc2IPPG{*mcoQ;^4Wo%2 zrjN9fTG#YptHOr`qrNT=>*5nkG4Vnc41PTTHsn^EY51j5@Y3g~O=IZ|KUSo#<)#8z z%}MUibNVS;v*!=2VfE!!x8kxyE%hOuPIX=NQ{H?Hn-o*hvYyX?yAeFq*~OPsGH z$Dt@t#GOC0ygX+!t2|dku2_l@EJzaZ#4c;Z5H}@woxP03cw7XId+H2qUG^w-48Z7# z0ofrCufQPN+;r5Aj9g$tLs&wh*>uv90R!=(caa?i2L1>=FhAiD8}TTTsWJA$StBaf zW{ZSzWQZmq@H(Qt4!G~YN#koHq5))}A(IS5R6uEDfYbq8BpywtaoYQCcjD&!UoTdJa*vX`}V%%0Vj6jMS{76hX!X${|OlBGb^O@)8rF!K+Rc-;_ z_pMFzK`Tf$1;vp)iK(Jgd{$QABg>b^eAS-j?lCK_*76uG6`&(L*GxM|P!8k@M;p9d zPRIm>&#VL66)90Z^YIftA^v!gml~u4N6gb%Po%L7#y~SMc*2JE++%xNUu&%$U76FK zt%sz1OJPX^i%|m{KK#*MV5|sI4-qG1@WCWFJ>xBP}m~& zmfBDgW0;JiGaNeMwS5ZhQGWbe+C8RW-St+}bs@5;`>@HW3}-6l%pb!GsQ^A)vJX)um^YBuS>(u#G@gTiH2i+d)(7rnib4}^ zBG#_QQd_*pE5TE-t{0Z!3C`+SZ2yprkYT?1&*-0yx|oiO!*Nf2z&k%+slN_66^3bYsf&QhYZ*PQjx>)@C@1J`k`ZVPfDI_Rjs9oc5|!*XTu5 z&a}sNiizW#Kr*HV_L+6m80YFU*|a$%4wXR$#S?T}+iWlhBp{Xv+Z*>(^BGM(g4Rfe;i8zdCFv@WkU%O#<>)rj4!A1WzT02 zR0QY<8N?su@Fn&}d?Ax-olo{^?V|18ci6UEv_9E~z<1l0OXlhG9V?rbS=H14cK_|M zsd{`&59P@AKmg4}WP(5v2`7o>t-okkm~azR;WHETWS90Q*(ehDEk-IZ3J-=Q`p;xZ z=*0qQFFEsDF?0x17!y`r(+WG6Aqtf^va5hjD9NU5<(~#x(<~n!(5K}nrdk-9Oht!P#u@OL^ z5HOth8IvPu-vhH^tg5H`)g~MI$_OQZY{hBo8*>!kUNu;MNVU>IiNex*aoEpJQJ6=S6Xy{Tq(Q=cuwsl%_Yn!%v z9`z(D?Cg?O6N8=?Jd3pPtz5h?(24(_!dF_+0~l)FGD?eQ_B`) z;IYt}&RiLue?IL$H-DxgBt2yOA;n@2i@{kw!U7>6um>>nxS{lbmUX)4mJKW2_5bL~U!si$TrVns$<% zF@*R)l9BgAI!SU6M|e0S{2~lll}VH$juhZGjeZftMnK1Q*vN^!cA)+B)(u%X(_T}9 z*^wou{fteul(~2`z+n^PDj;hgPDeN*t`QD8I86FhH3&8S!kQ4~<5b(!{Zf9$C}qS<%wi1uKIOYb(1sbDBdhwHDmwx(9!Ii?!#B!$zi_vw-jSxQuvkf-=#J>y0wYDmc7IykrXXL}6} z^x4xpwmTCXH{-)g+)aib5Mybf@aMAqS$OoF>`BX6>Q9=*7#4x!j%M}V?2q>lk4~-z zPfIxBP7XK8R$Ek_=ALbLFJ6ez^!IzE1lj$ei2qv0m3@vG&RMUjrtn6Xp@2k9Fe z2pPnmGKecvX3CqF=K48%K6w!w;}XBwOFL$cFXAW=(s_ri3Y%2K)Ho)}e2V2*7Z1mXIG;Fjxy-8W&Wy5$TS96r zH?|6&Jr=97-NE^p$P%J6H{G`3s* z-=F!0{o?=q=UJ(%S9IA2{_ESVyR%J8p_=hkRBp;Zr#*jH&C#buhpZ1X?T-#$gyk_e z;VJ1dERSio`g;GcQUz!j*j}L%Pe?+gdYo%}E(D)?-umF(+RcY$*5T#Y3pQPfWtZ3n zGRc*-1M;I!NWRoQQlJuq5HZ(lA&Kp-`gx0IbyfQVZ^8A?N?4qc#o@RY20wce2Wt%I zfb2nx(^0Ca;gSQ+9^>vDGw*WT**}USJK}^2-W}1c(>c>4EhQBJWeOflkN86Qot3^= z7~>&J$XAkInq4Z-S<8cK3f)ykG#wvq+F6ay=m^BbqrzAJayW6Q@_9usmEIm{i;6{i-bo~UcVpkNwPz3;J) ziNBKlBIp=iFEv$=f@$Qjk8sDEBQ=*`rsxtak!fneXT13`Q!=nYwgi_n@N8&c`}XbF zJ;OidwsGS|y#W1e@O{pCaQxvXo1P%cl+%H6l#TB+PW{^p{^FvK$e9wJ#hPau49H;z zsqke(eNtl$oV*Bh+_^k~ znpf1Q9#LP?Do!@|@d-yUNYzN=7@+R_; z2lMs%n=Zaw>+Q?YzgU*^6!_9hN&{^vKk_Vwg|>fMiItV5GQnt0{;FeWtXSIOi7yVv zJ+)_aR>Fw8GY13=Fm_Tx-urT0B4JHcxn4?%j+9uP6>wN+{3OhX&)eNf5lR`9mY4<* zdb;OCXC|B@l(H)=PJDs~=_5W5zd4GKPU)<3_Mifg)-|LlEid#Lkjm0UELBIASwxY* z67%@XZ8TlpNSjkO=MH)BICk(joc23xXVtYf*>&zkTTXP7pJU; z>J8m_u1lnJ={0bWFCB%JkgoQm?nsmK;n~o>`)mNaQ4Vz9U@NdhhE6+|$jt1|#9RtX z8hCaz@X$jKVVk()V#cjT`TK4PVdn#Kd@-Kcon+rQPPvb}+f&ak;&C_@$8xECoW(cc zlZZZ+u~=WOMu&SVkg&J&OPQhuSc$S;MiL$`~P3kN!F)WyWun5`bT)9N-{1p#K(Mb0ql zN3o=as4JPI;PodR?Iz1`$m~)%r6U(c)WR>=;;_q;m}!4t+mrUAKldTu#uM0F2Krn7 z^&Pf;4PGkc@)$f}uEtY(E(@jHDi=a(k_q&Em~44^s1w&4%(S<*8+UA};a9D(td!T9 zf*6_JLvIwW7%!~{^CC?Z5kVHqXoE-K5;9D}X`jHF|3@FbJ0rUAlB-ak8f|nyx}YUO z!~>S{V|t{`iBBGC8|E_Z?N7)Md;=EK7!TedKhqXtSOktchLQJCbP6_nNi#=+dJr;G z58~P`;$sd&oh6I_p#p(uO8CWTWX}4up;4x>XwteKf_Gk8TrwgEd zfkJrgR|-2%$QErZn5TQ6;_YDylsz6zGKoCcalm)H_kax_e!_a%e*k*~x66U5PoKg* zw;PRA))H9K!1GuG97_Cg2nC$0x-mo@#(1NH=!AkGEVajU%fKE(oW}@xoOGn&L`Egv zn=0<*-rGNDr~3x1slHA)88rpXLc{|yBaa7vZVTDe^N{Ty?$cK`HA|W#BgI1MPp+W{ z(eRK$=q^N?%`Yeek}!|Y(p%C?sb2Zj(%O^vV;3?kA8PjX?S*O}>UAenJXb}C>X1d_ zC!B!Hi8%9hiuNfw>ZoAoRQZsTFKq7EbHLvG?hn|&kUw>*#!}E!ryUcP0zuk9vfRq%IB06aOd?d{vn9r+joNWdH%aD&U!l z58i|xn87%oRRV;kEa(_Y&!VhrT>cziWfkzlSU5@WPXe5ce`)<8+Qh&K8{BiZX4>Vn zWA_-Y5r5`RyG!Qv$kiuFsR~4>9!Z?!QU5}s>%V|RR+;{p4m#1}(en5ZE&yM%o~9RD zbJrRzk>Qe_r6n>_)6!u{1G6;15u{PXF~nRALdJ2XG^P3&LmJ206fGXsSGXNY1k6Ct z80XSF(j&+r5nam1*bOJe^J5tx?VRD`XsvM8DzjPJ-TMyOs%34U;RKP>h9gIMmzuo2 zJg)%bAs2CAqbuvItQNfcwqdzU2OLOT(rI6q%>Z=86WN*SpAm?{3B2BJtLNgU0>T!v zxDSVWA~V=uei`Uu{@@&wGr=g`OBzW z3yzv*6IJ9rwEto2?d`MsroP|WJ9w>GW_8%By^3qcxjaUqxqhOZU8c|$#djSLGDx42 z-L)XP#K-g5A~`)yR8K{QGb4{h)z3qkJ0iN%(T;K=W>&0Hq$O8;^4hX`iClg#vMwcHDY^*E~P7^Q-8=) zI?C364nj>i3BAt*S8?%0@@viioUhAXrfu!-iLpFM3r!8x(v;D_>FZUVcMdWd}BglX}wQVb`_4HOdHF2X&VJ251 z%HB}S1=c^SSX9uVbc4LOUY1^GBpy&hOcu*{D&BQlpociwoDQo+`^BXijYqzq8=IQ; z3sl>17QOG0xV@(OxEA~)e`3hWCkCv13XVF{ z#;5Gz`@SMl9yV;+f~_g*wWADgE~4z`vi&LVxz|4_I+J{HMHYeMo)2(#URh2FO_8!LQX0?gqc0nRe+pH8@*>(X8EZ zuGtDe>Y7KHTUvLm|4(g@W?rOqF(4zM>n33M;G_70?|}8L_+DFun+>kTk0ml;?0~%# zmNc-WfjKk~c0S-7C>U-UeHfwsj4>a_$q52c*zSO`e}x%%vWOdS;*1TPveDx^Y~s{j zo2bBUBeiX|YWcFvaX(hM!J03BgH@GHn2q(rpE_ydr*_#$&%IVwjrGqB-P$##s;OH` zF!VyZ3h^3fhcgIDSlPVN+K${|$0xaMF_-7C9v-4@0VBN#7m`=RnGvV55Q?^FMm*^@ zqj7Y%aiHUjZ~0)4b)Wd0HC_6>Xak=JB0^l#yZ~8PSF#kHU;C7=LoqGl0f|tEM`H;l z4v15K1B1i%!|(lV+qPp*$}oHT55C4;`|=m!<4O(Kajp>`8LY$1IX7fTx-QgfYHSEQ zf8Axb55CB%+p(kud$l(OEBSk?FmD<3-r2GDP1Ev`XT;DM(CBO~_IU0)!n!lmmvtZI7L`y9%NvI7!n zM*KY6U;Gk}BJUYxoT2`tS&U&3IPM&-sgt7vWN~(usc{?Vd6=VNF}D48T%Fkpe8Zhv zNEVHm3rfpKjIfa2X4S1Lt+H`hGGOr(fT+j^>I|oS`3tn=BVVvP-{mnblZzzd>Cq%N zjjOL;L;)v!nH(Q~ITdq@m_5~)PH-0UIrwm3bB29v5Dxp1Z8kM{#A-S>S;J+w;;W{Z zLB|a4>eYAKy^rn^yM5SMX1WZ!kkn(gy|Duh-g>2z$Hd@K8$Z3@#tuDVBl|Ez+PDHc zt*o{hZ2wq>b<~tCvo9#OvaTI-0dUy+cG_g^<$B?jb!J#-{lf;$r=Q3d(=!~-g`T+M z!>4WS-aoSX)mK^tK9ZCLI+(PJgbc;F62igP2@b3;(+Iuph~z=n1M*9G!H-Y@8bjXb z=$QS)`+wK&c_`o0{Ee@9iT&`cSfk$7q-`G?>S}z~G|)PElu2H2{P*5#PmQdz;igNh z8{0lMG~)d!EJ4Yf_EaLlx#)}d>X)ewyjE6>9JcC#r>%PEX{#IBWmUubaRZ4Ri*Tb@ zS%-4*E5{4W<<*<4ytWbj2Tx=uG=6-qplo2sw)K3RtQB6k9xqZq{X14We2opRdc94y zTqOLtWDlAmJlFc1D}MCDfW@*GQ+!c4?&zgzQw}d0Ja*ctsz!U%ZnV)uBly4)9~;dv z*w48qs02{W$IAzH$aBu@#W#46I^tfxbLaPLQ=DJ@NAdBEV^~jk;mTU8!{?vNv9pib zYYs6v)3mbepmo9+@ey-0UqF&f44$;Xz4zPh+Uu>X>k7@XcXV`U#`^r0#}ozPB@*?< zOu8F!vU$kv3 zZ&bZmf4|y@^^61i9rwh4XxmO38Xi?1U$A}QOJ8jrEAS=Pk%QK{`y)17zsXLneydG2 ztX5wu*1;K;q@Wrs_iXKdF7?l~#u#%)d!YBr5`lgNv1N4!WQM#MdT=Gk1_&rCm?qLiacl^@T>FUOoaR!U!rRa$;2{>=QD zXR~Wln$kBtVvi5*weBX&AYoQ~J|~q%b_`pGjbUd7H5Dxz4PQWrEvJ2-JMEP#uhAD# zNko(!Y*-%kShb0XxdJSYw0LL$#JUZ-QM*>ny5hqNigmm_X@fG9oLl0|@@W#tt zk5vfREyk^ZL-4M^Bm_U$6JbAR+`)_6@o9nrMu0S>jut-_RJY=C{U-78wXu`CZS2US zHcBTRyRTHWt+VPTtc!2P%a8bq>OQ<=$Q?_{aE)6?a5)Rk5UN*Pt3B%jK9D)m#J|Mu z(u!qvEsN)lDC^9hV?HJtGVh={K4bBc4*tvc4Fs#8#*;$-0!^~ z-xlB&k`Yckw##}>584sTY-1gWRbo%|mbKlsd~Okg2ZWjx%tx!ZAmE z7-yLZ5f6xT<#0BA_<+?6?8NU0IqDUo$E~cc6E}73R*ea$inSNxV^U2~jajg?FBK7@ z9qxW$Tb6v`hSgfXUyJ?a8#}JB@>Q2v?cr@!_vEkJsm@!hf7NZ!V-?iPagxiQjSX;G zNc+!Me+iQGGar91wa{NP=@(aI5jgH>(A38`BzOnT7ewmub~m?g8gGWvE(4H|zN_1x zb%>$EibR}{YVZp~G7Y*&M4Ax(lMnTMgLb%o!q%?s(#|RK?V7@MeEt3X)^PfiO^i%g zH6D@rl10=L7v71XQ#QEo0nN12Y467FF@svhNt|f92@7+9RZ@938eQ?BJpg|3PWf zxVHE9p0?Aw_S-aWVphK3LR)*?W;pcz-HV_jD*4nSV@2$41 zy%l?}G6Iz=({kBzQgJVpTN?WJu0gL%Rs9euW(I{h} zQ`T`j3A*Ru$1~I`H(!uGBIQqh%jhSqywYkqF0hI{_gT~S2kg*>pS9_RHQAFMuUB5b zsAfPiFUE6SP~gbS>Cc|;WF<^f;^F~|z!OLv1Z^5Kqr3wygK?|GvIBO2Dcr#`Fh2|( z&m&IAaQ1xfq{R}8Dd)~_raU<>mxo7|7io%*LNtzp1Kz zJX$=d8D_pb%8v`hqu^paMsdMZfF+vX{^od8SljID+$XVSGMx5`6<2FbdmCOx<;Rui z{D$Ez@;k41YV2Z7RR%32uGm@I$l&8l5G95s#d-dI6h00G`T3!xy{)gd_AY!}CCR?z zw4YZiEfFtL4anxB?g8huK^&KIirjIR|Bj^O$AbeX;oQ@qq@y)5aMH%`B54)Yr#Ec5 z6;tZ4Ey9^w(RAL|uF4&EevO;UWtuNsFMhHgG!Kbf=7FV#0V!cyS>I|E^;cWnsu$QK zz5+dZVh`3)SJ~#C-?84l8*H@W1{-g>2s5OWlq$1(^VUC=eD3+wOfI*_@|4)sWD-mN zYEM0A?T0^Y)l+?#Wxw94vAlqw|?%&ZFy%K_Fc#9b3-lmm&aFXu^+!T$ZQF`y*#}M?|#-{2N`auNO9ztLoz}M zKX%$`uok^?Xs_0#*9`C03#Dascrmo0-Ksd7zV0I2WHimHt9U@nv-KC{Cp^|Fu1SQF z2afwBboAh3JD9ETf-M`Z8g=ANCZGDMuE6K4e)C1Iz=YD{R{7L#+Tqpjw6U%mT!)}! zty!LjnI)flf6^!dzu*|$UAOs}pGT>S4K4!5J#W(JMC=oMJXy}qDr-RHIK|k^E|_=F z=rU)^FY@wu-tGK+CWo14l1v?#axVlwAItjB zV_rJ!wY9bLI_}|okK;*6^RkO=^zfs&`MLye`PbOU!ACUHUbX64t!d|$kK8_yj&IP- zmg_uHV2nAFF2fCl&?xF6Lx)^3p1Y=a;irF~zxA58#A2bydb$=~MD<;84m)p>%_P2aal1K*8Q>?s ze7_xkvIo1O^jj5X*UK`#MGYfP{gi}ES4xk3JdHq`mG||^f!}}Uqjq@f)3)UYUTh5= z&Gw~lK43rd>wlba>S}MX5B~f+Y~3m>J!*x+ijQP*rd<3DI?-F&`SmjTW72HKGm|UpIbOzMJ1z>xz1Rs-_myGv zvB52!QnRx+MbAK#xO`|7^reG?$YI9FA>s}_#k;gq2J7HcQk+V3gX!IxFDsJXA^FV? z!E1I7Q)ZS>$}ic5+y>4>SG~WT_1BhXXbc+3;;*!ByfBPo@89W(Da=l6wAytySnmTL zv!SsG>uGqA)xv3CzI?fMVWHE`ZntTT!?5|dn{fl3vNUe7MV_#^ zD@QhNojI3N(rF(&{G{2b9oD$%WpKLi0%;o4*$p@cGXi~MmP<$!+BWW%wPIO&=D6?K ze?&OK6A>^Iz4#$bHiUT$DUZ=)KB9Q9ArdQ_;KbwCu=Z*^z3a7x{=?X9@|!ks1j|4s zMr^XC!zL=*ZM+(=ss)Z^9d^CJnta0YTEsD3jNc@7ADK?SsWP|;GFU8?_6$xrc4RBV zUf^X@qgF97i0uY@u|IgPwjHb)?y>68LsrhTdMx#5#=7haUusow`n)Z?0!5t-p6s)& zpZEq|?!^pye;-N#CUuvLX5jiG5Z5ud6hd(JJkn(}z#^{-wkwubPGWuh10Va6^{r{Q zcYgRU^pY?cH8?Z1;(4Tv0x+EU;4>D^I6|x~4qwWFCGYPs9VewEjev zDgnuR96QR~RaoD?Va;-D#a5R!)qMZU$4u}jmRE(o)pGIoSlNy*sJ;8vzEw+)Q!^!r zLK@0PUJl78;)D#|YF?hFL_BK7)I6TkIrd^bE&|8h@8H`r-Zq~wsDGI%hRXFNI#`n+2PuPh(vb6sS>K%tbsr{>2fn~>o2uz&~0oW-z$ShBNJ)wSTUi_Gmq|tlEXGml_Dxj0VkH>G~-THaD5EEXsQ4Pv@_fRX6WEHY@gR-jThen-xf|UnWt-qhPJkkOk_w` z8hPVaffK%nnby{Dd3De@xuPOtw=UQ#dtisRylK+pF{!LRA2~T z6*bLRZqbajft_6{I$E&x;96}*C>&CNMD+`Cldf3z-a{ws zQ8?{f7E_6J>&Ih0=6Yc5ZEqEXS@*j)P%Ua;X+}YQRdB8{v;r-^WcQN%MTns zVekLg$88APKQh$fW!4Y;pFd_>E?j3_m}zgpE;7}aeU2rHE_?6*K?snb>X+(NS<{G> zQ`mYD;sjd62^oCLQHMNd^0VFkv6Yv=Tg_3c++eS00QHEOVxOOgUQs1@u$Xb&<7R z^CMRF&_7$*@EWXVU-fNp+96v~9p#p>@k2V1Xepib0CK3h04GQB!47=YbzkF4t*sj$ z?7)sX@sTA`Erlfw{AXz(9GCyB@}FtB9FJj30y?k4z$-APlgBTo@);|)%A^0sKqqG$ z&h7A_$E ztjn;R@VN*vJF6QG6^HAd$FTRvN!$9rzi#E2Wv{?f)OiD&4Ya&g@{iX< z)|YGer*N||30~C2)D)lA5}CRoH*arOhch`kG-SW_&3mmsIqke@`sH{3uw8e>MR*al z2}>)vwGf@*UU3F3Nnz{h@#WW?oLea$Yhlpr#wvv z!1J-__>tz2xQL0~Q(`tfW)*y#11ZQggFqTj2w5DGM`D-}IN$sjvb$8J_@&h3`=46> zT<~WQbLYL~1wU+k5B)9npT%dJ*SsK`F?j|I7e=BEk*@NfU|)+~zy^06GYuC3=@+1+-Jrk-3)0cUv9^GcFeS6 zJ-bh)I|bG1*CMYo_0wP5ZU;LqvLo#m+8~zQyz6^^3o_8T#tW_Z7Rx;&r>yhT)3);D zPHV${`jt^x@S-GqrgCy*%zow0Z`rZo5lP0BLfW42`rx4&yZbR8AV2wXop`b^W&A|Fsz3!tgnCoYs30 z>3o0%y?5BGI7N_`CCrH8B*~K;b@7ihmQ=#uizh$t!ak5`pyrdub@82B|Ah}lt`VGM zJ3*J4AxmoI*9x1jdb9Q2`=>~Mw>7Q1`Z;#qQ=93+5Pb(}sSkOLh=U~JKwhooXFmRi z4{gV1r^jt*^~-GqoOW*gINxqM&THwoq=6+3oJ|9?l|_4bCY#4R+7pdWEdQ9s@(^0l zaW0R+ZY>r4yRGq}o3ZMnUJfVA7VpARf{=&2P7O}mzwKzWou_JTY`V(Erm$XIUTI*7 zfo#_VHnSb(re>Hee)1IF(dnzPw%Q4M{YChqXe$=)qx)YFDCJJhz(p4{+qIP0BA!xqk5i?4|gPJAR4R7_)cu~zG?Uu8Y3uE)Cj z5!o!x}pWBMQ4ZDm_7u&OS+h{~Dv z=dk`N8oH2FN&nI{J}Q)PGdOu%^`u$QkNvf3&>k@CJq$|$Q6-x+AS;3ix3^xxQ? z+LhSJXg#jCd~%g=C3KQC6{(~Yf2KYU$LC@Wi@AYVF=l%XagOmIo zYsAbu@r#Bh=XSm^wiPv=+L&#|b|G8#H|6pqrAtVUPYxWn;UkaR{?0epvd(2%B15PB zxslUlSUN6gU@>Tbc7Q*P!30@v*+h;@Wir9-r{iKxG#z&=z2F|#!@WnX{P;F&*z|I3 ziB}xVvYF!Jww^YC+X_E+q{=?=WSfmnRoUot4a!M&frS@CRz8Je;Ea5g^-WaT(7?ET z^pOfUP5t&g8-`)Ke0@AgXg(NZV-P3w5HUZOnE^qDuXv%}v{1=0CcSIHAR&{dyL@0kj!8JsyYJqHedzHL>hMrj1E&{zIqiehP4@65KVW_MTI@(!1DsB{ zE0QCGVf|PSR}9`1uraH%u`+yhyK2H7zu;zjdhLyN`Swp*Grm6i+i=>SICrnqGyYoP~Z9P_JZ@c`Yt!U)ZB!9YG zZ2!u}POHO9xvTd5vF*F?w;>VUE#p?03T!KefPNN74G)gN$c}hQaIuh|RH1&4JYV<| zToxakr!}8Hitz3nt=X2*Y$548n*S*2W;}tAsZUPECF_d znahctz>8G;*hYB;9SSj5~>$!yk5>kaVy~x9_dBukLNJ!3pfp!WTEm*t2KR z)2;}(WwKd`U9@Utg#-mT$_pPUF#Lu_>S%=6Zi3lse zI;Dz)E|Y$xR#r2s=iKc-g^9V7*x~V`TRZLaXpId`VVM?nN7m$?>J;@TZD+^wBcBot zOf=vobj<$p!3ulZWhZQ7`=p*um9~E^Hi#JOJ8UfnKW@FN--2C!)qT`?>9li3e0u1(b?^N%tHgRahUiK$ zE=u|f@^izIjevnc_>Z`8L$p@lX_EK2Vz@pOt2OxeLhY2zz zIWdAsCmg@}!?>Kl%;X3^+H>l0t7%zb)$Qvury`sSlDUiNkYC=sV{f&b_JIj3r-8$h z=X)U>`fjKEoV@rBW$-MR1|iVelGm=&W%f^xO<~3pTPDX-BG1ES5jc~yE&H)*wasYz2%E5s-Qm+!ER7b)&XPSq~nyveW+w4`i0r<+t zKVp}(wc496-Ux@i${O&i#w>CbnGW>Z;Rkoxipw`xLlfSS!c*E}os03PDBj49y5kaL zVirrxB_PF2ijq(|<+x)ASQ_^ZOnldySBRAt*`hr>?;UpyjMV0gbNi`@NqjbW2aHkW339%33&o5Qos>6S_?bnZb}31~Jb zo4F0kX8ywZ%YNFLu)i@4a$Eo%`FxHv#Wcz^yeL*P?kt=qZq`^0DVRdxfArW+8y+}q z$FcR}Dts9=_L`Ste?Fot(JpCVNdv9{ngRS|XEipNkU}9k!^R_f6pmii>BCk#IcW8( zUm6s%Abl*3`$@d`_tf!n`;WbCHi+c~Gyoj$v+R2bhZ*Ke_--P?oJRg2mO1S1o3=0Q z9J5z%8o{=aaI`R+Dcd5OC+m{zAM8zL|5BWc$Y0KHf}SF)#abv3ufWd}=ReNLFP?6I zOl~SFBgk_&?cDjL_wXqjJaiH}e+|!A?m(xNA|7p7JHu(0E|KoFa2?^)%_$)D<&Gy) ziN`V>^4x2OiITDwyX5-svpQ^rdBf5BtPwNnjd-G4&6#1;T~xRME5p;49?ZI5^s@h- zy*B~2^tjH$&Rgz&@7tHz2ZLp>10)4NB*7IFNsuxXQIu0Dr=*HqQ7oyLl$}&riCrlt zRf%FJw##FY<&4c&mP^B| z09H_@VFVp{PSh8tA2ndx{Pv+SxzNaBZACEJfuC;UM!dM!+Rn;v zzc?xX?GMe!NCB%1bS;3Tqolb8B){Vp*?sg9fnu<=So){XV%4bwtfd|Fav>{#a>i`A(guJwK z1eh#q$TXDo+AM!=9CP09`UzQk{8wc8#8bklHk=1|Rb0?`5UoB;yv^9XHafO0{iCeg zI246j%bgg0K*pzb$|wx?-qw#-r6zhrdJ>2vkkdtrmaa@$0=GK@QvtOI|w(U$-3m2pgu$-*#&?TdNtywng4kbdfg7}B4ECs$O^<=$IT#u4%B*x&t1|cEaj9eX89paAEj#6L zndMjcpp0A95Ba6Mop1LO+7)C<0GF(PHWD5V_N~`*a;}=kBGk;=&<;D&!cH_bOyT3NX$h6_O&Mo7}cG3e!XxG6oSO~~u7 zx%^q2usU?t=hb*$I{y8uXS}m$Ds+>UjheGAp^Spe7|&{9+gm;Hgq+IVCFLClb&3o} z+uy#0vsz`npPmG+SOQNx@q|45@WXnYKY#v;>HNSGMtTQ1tNsDhPN*H$!Ze&plDd(* zaA0=%I!*yCjZ0=|Dx?d#ix|{lK~K`C=IN7vbrf@e+t^wU3;O7o1ur^>dc(Zv2$>&^ zo&eR)D27EY;3Ge16E&E=bYvV``!$i5ZC6QtsAEje#1NE}b_7G7%rCl9`dGs~jCk%R zfR|@jt zxdDCAWtd*%6Mp7goW5Ch%>S7*nwWA1S^8!j%vRAIB8iqGKx@bT+$7hyNY-J}&#-Om zu6@H@E7uzVYo3jPEhQ*P2I(z-7|h13o(RUcn=4jJ$c&7JH8#6ts5))f@M+%cctLRk znx9b&BMwspS{{%LOZuaD1~SbSm&Zl;i9DD@%WUvkEx#J?cYam^y6yjO;Yhj9o?LFe3Gy=gm^eebs?fvY8f zS6_Wqo_p>&y_RrL%e%*Irt{e+gZ|eCKXKScv6~DH3~V7dnXgI#=P9TTWY6~JLGV+@ zQcu(1S;Bncr_T>#L>z`YN7qTi4bBAYFpgu&4t0E)T7%=v;ZNh>C&Wd59Ma!t_hGSN zR$iIqfkTK3&kypWJW+p>)+YQK4ohk}iN45d$t*ylV>%b5r_VFs`tybaJxqxk9m@zM zMcn-H8Q7!<2amj`nMcUQsB=TRWf6lb^$rYA;%s$678*y0i(fq)svqeb63^wNbq1NW zAjcNAlRUlu{mP~B0TMwZgzFr2f8`VgciJ{;oRF=TM|tovlUMgE)$X!e8lY$#Asun( zQfap;cRHNuLiyM}zI~#s=LG0MRFsRhA#Y0tqxPq5jcw1{pTp4p5VKV~xt zHYQdwE-DIZ1R;QRt_qM_C;|_DLSF3X4z2UMvZUgO{jLx;LUb0zBTV`|ZLCzR14035 zM>Kr*6DIveE5_X_KF)jd{H)woW-qrxFSkR|N|rh4-&k(Tue;3eobqSKua}`ae>|Et z)5K|gZ_RkO@pL!Pl@@P!v3Z5&*H}CyYZuPQ>5-4g`1rVP{Yb-|qwPJ{lR!@bS6Ko< z#ku}gZ;!YCkMB1 z{JqFULzD4@yr@$h(og3w=?}V>`B$6$^1U;|7|Fy`KhDchJp6#H@)n@+>3kcSA|(@~ zc_r%%BMfvLdbu5vmyH+GtDxvnz2}fSo$w3y;_>sc&g~yjCd~$rhI0$ovFko6YcSfW zKkf48NDAsGGr|aQF-wQ)r@R=(JoF+x`i(9NSHr0O>FYm&#hVCIZVlh~1#YvBL$?;x za3{W(+u_Etd$}Ewo)IQAKl9fd@Duyvc*nDJAyZ@pnwD`4oeUt`L9O1D=jMjdKKLwQ zCKtKhayw+(7c73w%1LY@(dKI6sNSvod=ciN?veR#N|Sr^qmW)ss|S{qVYpDX2qyg@ zo@Hme%fbB$k@I$S*Zktt?uun2qw|LCeuhw;tEs9O=8yM8Cx$gkqv3voF&(3=B=F<; z`F#?0)3cZ)zWX=Nk8jJ)Bh$K{aC5g=elg~DD!&=;*dG{k-aDPQzS|6CXEi`zipHkQ zFcdo*G^$d4;~6ed|UC0@M#fF^Ug<8`-xk zyM9TAwz-kRNOd0$auRuCwInNRXC;fh#|LzJPAHHfdRdXqy33r9koD94tLUo>EyF3&CD z6xDGo2*%<{{23?C7f=)KV&yk9!*ISrX`D+nP8B1xH3IWS5N1Jtn8?HOXdk(EwHYQ=9V^ zXwnXWQ7rEr)1l*RFSrd1cQk#rcRDIk^|R8g;*)P~Y><)T+dOXxH*xG=st-1g-Xz7v zgPay2-KFRg&6SDb;=N7*>P0a z3_p9=xe@3lkL!O)QLv~^#k#}_IIHhV)%Bj;}kmYZ08(tRhi1Lzm&T((8)^WHM zhHKSXZWP}cQBVeC$rOc z$ga89z&U6`wnPc3F#jE=S}!ci`pz*ksx|%Pm*44mzHUy??oeu>J!~j9II-LU1>z}# zc29i1Fmh4q(F+Ax!%Gia(+{d0u3;&_(d7z8{8y1D>I2U@{(1fP7lu`agDbB~yL6Wf zppR*tfp6nc1seG^45?WU7ma5fQ3OG5OtKx_HZa^#t=2}g1JchibwpD{nudhXC_&6_@tedC!5*!RoSzuAu>)5-Ivcekx}P-Z z5T4v_in(;rS0tbX$T+IPka`3Eo{hthECQOIMG7IFpO%qy1AorH7L2?Gf{yR}cs>r( z={GBXM3eb3A?W)~KO5*L_@tM7XE4e$bjOcHhcsQ0@s51?X)t6D7}^wh!Qa}vC=xBi?3>X}IFJGUJW=17temM=|xsQFwy^Q%$2#1Mm~Vc&ll_ zY!i0xIULH=Y}Ii(Wjxc>*>o6}!E)JH9ukc?xHP^qV>uXp{5&%HCO2c(@Tmj&99vuA z6d4%tT>P1EoJT%CdH{xW2JxKIVv!Dcx0!#lumsuI>qSh`1#3;*FJn4KLyb;s4;-F= zs9{aJju*M=;);%Fo17?X#$A5-jTt@{OeVayb`{Gj4l``SWp0+A4IskrmcOE*JwSeE zszq%Vuy`$TC~H9J3RAg(Oqcu&!_WhZmuiQ6{aVDflXZMS$zp5Gei-iFhDeBf50u6v zTRnm=0k=teV9+*UKwK|GF)}c=juje+GKoRST?%3HUAGs+b#XOo8v{G3$wzwfd~ z-c{KLzN}q=Htw6t=Vfi~n7lsxV=_K5sZ(UQi_GBQAfi)>DDd0^Jqh$A@YW?j-NGxW z^POsU-?MEB``ath!MPsZEt6@Pj^R#Ick!t4Wu1A1ew@-@LA@ZM=L#4pvqYGCJ~{yOa5uYUiy{Kl{T4h(nfIiJA`1h+KAHHhsE33<=8GXpTDcu~xbwSU(piL&kn#HL%PaE6U1H z7IGfYK{~!b$PzAlXQ zd;(3T*Z6QxqrCyc`L@shqRLO32!E`xvh>z`Tm}|$aWT7`9B$DLIV#S!ztZSPsR+J+ zPZ{d7Fl0NJ$G+|hXJrBN+;jN?*y1yv>=`t6n2}eS$63E+{8sqvd)wJ<1H(Pp2x3JZ zRlL8YMvcb|%12&p$X~wHmNo3p+rsh5*0=)jwzgTzNq3wLINnGJ>73XW4beLxp`ruN z!z`rx*W<1lev04JZ8Gyd?6Povr6G?WuglMUWL+jJ$>!zdbEt)hlWXxo6Hi1J@Hgl>$;Gpa@*gK<;Q;+pZtD4I;1H@VT)U1n7^h=(8f@w>#c^Y zo_a=(=6+ZTqw#+2G}`_1_7((t!95A|B=8m`K;ys+gjnZqMpBJ$7)VX*HUbOE)G-j% zv{aR2{hN3Ucn(w|-ez zZOZ_zKvBO#-+J1E-u3YBe*72IAf-xHxuNa|)v02I&#J-X6zXErf_oh}B+_Pngwgo? zpPqP0s>}16l9E5)Joud!dho~bVu5xTJ^B*meY@HrWq{W_S;aG7^MDcWZJKfP4-Dib z*I7Y7u|3~Acu{1ZqzeFP&F|H%k%=G2bLjX9lx^uvT2|Mn078#P>EcEM_2?}jlv?Kq zB8(m2$T=GA*!HMh#B>riHYyZHOhMHO%p)M+t9TmWVF&^{(+8bqyW8PcIU@L zhHnp}xGZUW%gWV1^9~sWn8r7{m!eNa8u(4aMEQBt_v(5OWf!G1b$EwaDC&Dy`SCG)fg2U{jGtr z^sU#S|66!@r|x#F(7{z3QMJ=h;l*w4kka)R{I0W*SkvS6T_Nu*mlzs#rZH!_?qc!A z5wmQkrkey3?^@75$iu5RnjB=|&j#MAA5x0K?70x6p>j>jol^Ap*v~LJlRO~}_rYx5 zg5SlzI>(&y(f=PzJB+1?ma7$K`I@{bOkil^*0(X#Ei9?14r5`fU4ZbYe+ECF?{w*S z$8#7{+FrhGV7PMzg*O88vr%}%@P=k~t$n_F ze84B*y(f2nf=0-InP~pmaT@PGD$AE+yt6!fSqqNkYhZdz4M%#3r8Yu3wmGx8bL{6JhPXoHY9i*!5&Z`tw7;bH(A1`U*-F{hRtZ z)7Mm;!F{UT#hA>+@*&*J!Sq)gLU*xmliU%>Svk~uyT|hK(?9+(z2P(qrXx>n@W=Ad zNFpS^HdO;*QCx^eISkKmm|ho=YI={eup~|QbMJM}M`7yTr-gOmXc+OBCg3rC18x)+r*(X%@AsT?!s$)QLozR&l4RkO3m1fH zTkOY)zu11J&0M;DT>Fsc^JWbJ;rPejRyjLH_*t73?kR z&IuV*qnBwy8JM4X0~$)B2_5pzS=5Mt7tlPDZs<4+b-%-I%$PVcp(9>bh8DJt&uAy- zs&d^J_H@l=h#AvXe@GPOh(>aW$`g}TbmYTwNB!0FB!t_jppZamTS2JPG`;DznCB?H z<(D1X599qaFy8+S@V}_jW~flNb}=K&X?<;3YO{ysweqKBY|KX5Ij_C9i_F#%?8WpX zaJ3~seW|(=3d14ua2-NOr+phYoD)8`J}JYi^U|NA;huKTP*f@S;or~z3_hfz;be}` zb%E&;hHoBAOboc-kI-Cq`k0o-8!;{&IxwA;{d@0&PQh9k1V_&Xr=&{KdExVpw8^zm zC7SYu9}8tj8lPcF2TV<5VUBHt5at1ZE+dwW&r~j8oS$Eo`6rJ^KgBs)}* z(t#baKTF)c#OxGNv0a$Cskz&a|@3 z`muJXx6zNw$jeI{4Bvx@klX@Fe0*+`$9m@|g!3^5|EO$Zx!i z?Xpr0*AOT8n_H(|)L<7*xX1>h3OVwZ!; zt)G?U@BR`fUCMY@E-XKMW?wz^oSZ7WPfC;5%LI0jDVNK5amlJ&m9gGWPXav&{J1E3rGVYr`N-z7VjPDyTjKXeUsA@y0zD(^{$#@bNEMrye|qm3vMFGe=4 z5Qk8=c$i^QzcI|yc&y#K?jwxxVJu<5XL4{=#)pTn$G91k8Xw4bxtL5}DlB=#NE8nQ z)0HtovdmU)aF?b_o)H%F&HI3)oLW4Q7)l=2FhIpco6TZg>UE6H!q0eqD`>*Mx^PMc zcJD-(mIOcK$!K~$F-*qNe+7mosB=WSuXaHynSP8K+bA}3cSD?uDTgPAWMrraIddO! z8@yo~aOZ7`_v9lPM>;G4(TPVt-wUi%mQNH9EcGX716^H|2ja9XD6}}XyDIFJ@x=w1 z%TCBz-AAQ77t4=iJ;eQlbd;-wIfingF^p%PhGQw*&jrdO1Is$9w0(ZJf#J^PGYzeuA>rrAmq&(Hl-0G7faS7t@c$-*9dD^Z#^6{&2oe3QUt7l)Dd3 z$Ye}eSOWekPa!??(Wq!- z1Q8)HLB<%DDx_CgON?@P)Fu=RHz|-{gRV zWQNWeOOQ=?H^|iz#lK$8m>i!ocmcvzGMRI)%1ACNISecf0>N~&VHV3JncBBQcJ3O- z4rw;T7B}Kbd&UrxV1UP(1f*wZUe)eH5#>ols}H4G9p%+`#!?Y-2yhBN5y92jfr0P zHF3y8TNT11=!z913#*t9hQ77RID9_*&QmAlJKsMiKmRj#$`1_>$cOJAmf!jPSxXzp zDV;=inzi>nkO7m%1ED%cHUgWmm$chA^yENgYruRw4|?I>wER%W2=_UW19(8j57LBq zqKp_D)1{KKj&9}fEC|hA3g48TDuyV9rzP8$qK|j0Q5f&HbQ|w6h1f?W-neiEdlk>i z*~y=j9b;pJ8H!F=S8(bI2g{(M%>Q zo#M32FAYoS!YTCqy-GFKH&k<`d^p;e#lXvT+0!ytx=jst*Hw-Ohth^7v8Qz$5)EX`DW-?>6d3(n<2q z`>otY{fV5sj7UU&$qEO6%9zfxUnQls7Uv%r z*D{cFV1&V4<-)y(DT+rX5|Ecj5Q@~1;$TM#Z#*j{9E4ZGOW0sAE*ag2won{~;XXYg zVX;U22hD_o&91X~f!co>3w`^U(*HFd_j zg-pM9q0oT5JNW2$Fq|FJvHCCvEOzkwbd4}YqtjWF-}&C6{LrVy@QJo8qn#OwBFB)4ZO>#2H@u7CC)0vZsu=!NLm@LFD=|4HMi}Ng;sIX1 z<-us=J-_o#VkaD0Ok-7Q=U$fAD?cKmle=^=YNb+9qh06FCa`Qllwkocj$jq&uGasQ|<_bFE&wbj-OScR6f#1k5QaQkI3VPKF zXZdJ)CX3mL{gJu_g3FQ#+#l%0CRmoY>b%T^bd;xuZAu$xH01Mmfh^{8azo~b)JnI=*?Qw` zZHH{1L*=|o4a~?$k$d;+>x8z4xDD{s*)wvYa}&Ne=V9dIi@OzSV^LV%gj^$kXjhSp zgItt=r`bqt+l2UV@U>lF_G4UpQ*@|)Ewx7`p&7h7T8t5a2-1%^dHj@WFpMfAjEzU_Q+T^h2w0-rR3pJ7>MO$UXNg8{4tuR^A`qo6HM@)Gio(sNWB`EP+wo|H!5 z%ZzEy!}y5!Qkq83M#?Si6_?+7D+)&8C*-Ss27AHbcY~gAvkNE@PSWqV%tL8}d^~SL zUd+p3k{71C1b^Al{c60gV%p5g$*1+;46;B+MOHoYqMXa$A%z{c>LE)s+Iw3+ntXb{ zJqi4^NWi;wO6ODWcU`I%Fvl4?jO1VtpUd7Mjry9jR(X=DDiLEp=qiSAbT-S8S{S3Z zW?#j8bF7Ktzy&RLLh54T2WZa)svzQk05ASHRDXz29L9BDqm`UY^`Di=LJKD^<4_4%}& zkI{9Gk*EU>Uz|!W{5j-_aYP81&zm3(m6`gHr#{R0tb;##ej-wze}H6*Bm6N;`YoQg z!+F&8^AGW?R-1o~{520SM3^gK8q&TAyeQe+cAe^oa4ioZE_KXkE9Xdrdf_L6^e{X? zc(rW=sD4B_j53VTjTfY6JEY-$V`)}$185s)7jI`f6osu;QF=n%VP(- z$+VVE%S`qz$)Zp7W5k^=J|-=bbr6OLoCOROpg8ymx9agQ2I@eYL&|WQx@|Dx&W55@ zkLF>1w*R;fSX|sx!abERM+=CH4(eD;r(vPN=o>%$@f&}L2q4ZKQ!kK`ol$9oPEY^+ zw^tNE(Bf%NHNqFE>wNOF>n6c;9N&<<(bC%CCH7MizkI8K0Ja`cv0Q z>$TV9AAM;y<~Q)(hb^1F__KG*-cyI=7rrzf)2GscGLycN{QaN3Q*NxEm;de8kILD= z-<6hM<;A?iMX8ZwtgBLzZStv{uUt(c&G{uuVI&r56XM`Z$AKMoHM_vW=K8{7^!v{Z31Y*INMk0D21_*LG z(wUK4M;7IEd~sWaVSQn#+J){PS3PwXr0yZrppe|}xkbtk0o>JuqwRMVUXeA!n1>cN;g$b4c+?3<>8Rr`FbZyW#u8DdqdkiOt0JZ+t(<;QhQ@D_3oBUo8`5hX>T}ov()F2?6Z!OKOqW~{Zc+gc znD#Q*X-etvw_sG-%a6ASK65iX;S;0qrKur3iPq;lkgGOsgjg`~6Y=y?UfbAh zj}iA!!QNO1-S2hJ21ge`zV*|fg=Oe4>1UXSJ7hdw|22(5;sm;+-Dm`Q#KSc@QIWs< zU;dE%=pJsDxGqmWeqPgS8q&da*ZsH1FaPvGxm}HR9K$?Qm6??$-U4!R@Xj0MU;O>s z3RW^EJ=N%;cwK9E(wVMthKb z5A-DP4v~QCc|!F=2|1)fQk4X#1|)Q9p)@2X`)-$7wJMFp(@<5^Rk-K^q_N6TcP_ZB z6mxP<;aM3fW^`(WfB95B!e+vwZ!O3}RXmhG4bKq)8Q>$GU;y6J z_zA7NF(hrHVUPyqa5*QrM}J-Nn8M0w9xg>1?W9>89g%_SupkxFWU?5&<_i+(sB_4m z%P7AkCLuaPTuL%P`bj&2Nk8Lo(K*x`qsOlSY!6)Tu)k81U2;UGG38dv4jz+jiGC7s zNWW1rNI&BocBKWav7r*SL0y%*hfbi~W6?TZf)rO}Vf9bDpf-C<4)@)Isb@ueX~8)z zVMiXa9eSKX^@D&zC1yzc4bBT~yfeR;?|`xm!8RCiXN6mJTm91S7n%T9KZg+dsn+Ru zGM?J4+PH(VS3L83161UT{uq71hq=G`^Y53>HjF+OIOSkyn`_CFfBAj+)mOV$g^%5G zll=D|!U7MtnHLVrFaGgq@xz(N_V18?@Soo-yRswl-~8B!^Ixqc&G}hMS+>X!6$n-uL85t2SM(yp_o)CJ^+mpawn*?GVzmbJl8Bss7 zck`b+$rLf_UK$#cM^EJC6WQk^Q=Wj~!GRT~Ffa_OLd1~Cm{dSXbfm*+5_enp1yHGn&u@TGxI zAth-)mP~b)C1w?|v87&Vp%s7qSdt~4*04*ltp{q4R!pK>kM9l*1&{41;M|WAwl-{h<5y+$fiKGJVjBZp7@$fy*U5zLyokJ~0nWJO z7i)&oXrCCyW&zKBSxR$9VVLK@Q`k5F_@t4=?n!+x)`#|u>r0p#_GY;wmiZAYyLh~$B>fQXJ8nY@d8<|6y&bjQCX{&q=SK^ zGc$_;T`4Q!D`|(!my2|V3+r-U{s|eyR-~m8b_wE(jT_~m{A=fr%4~aF>gAiIQYh=% zkpWCYbGI>n%0A{VmNCQVSi(5oaq)VlvD@{>EyruN{9gSmpc`TOCK);an}s$F8h*sl zw@0L6JR4&dOozoI+B5|9qM=H^8?2<^4s+YNg%j3&CyaKH<6nDR{?+67%!<5pq*`HC z{?4a&Ys#5tpOgRLkKT|u&jWDo_0#eT|MEFG&G&-x4*5GDD9P6!IVm*P_rG_)>gca8+oI-d`I)&GR}WAApz-7j)5+$%%la<>1V%-}0#_003^^rl+^ zR6PSf4W<&d#T&s6F9$~3a_``GWooP}Bg17Rfo=u$=(*s2LiJm&{wyL*^%LUaHk>;q zN917Nb8_3zj2ccA7)yLv&^g7(Umc-dN;D9+D9y;d0YAiQ-1mDQo>@%9Om8g^@q$U zr+_8w1~zchq?GY8MmckE?P`>tJehN7&lHC7nc~kChpZPZV}Qy7k$LaV{eXP8D1hJL zazVEVy|?h3Ob?uu-8;r~yHKXO(ssy~pdEO5oWREXN#qa89o=Id>MY_;oE zS(`m7hva_AW1YYtM%%e|M5pQmzP$v>Q!0$N@l%@hLW#s-dK1E830mlO`@QB3cUG_~ ztK_I_Jd%r!9y)uG={}b<%pO{S19~qS!@x@DIQGWAJF@*+nlvVjqkT!eh?(BaPjkn%9qML;rDn~T7*fz$EP9Cj2=C7p$-*#o{9X^>C^W{!qv(~ zdj3MyX;fwD`9G0!ogFg2_Y2WT`ydwSyqos9W_f$ACxL5K0^Ywj=zMf>)&1=1-rWPF zk-{A?N)?=(I(my7Ker^c3o}w*JgLSf6%$bOqVddq)iW>vIodgfgL?LlH06Vthh;~p zA$xX?>(mf`F5q-Os3z)g8lJq|Pl$^nh=AOeeaF-Ywv^4wd-}gEw+x+^kxCI)0mc#+ z0 z26`TI^(!dLz)d)u4r_ue5e$38E%D~rtNyad(Wf-`eOT#(ayhzWX(e^wD zap|)qK==!V9Hx#8N&honlG4+EEG5XWfI71=U{0&C4I0WY+A*Xo!+XYHxMM9D>Op0l zDEC__KV?aNUA|s7USE_Q{e-;O797&A_yMXv;44pPi{!O{uQ8)&E2H?T@k6B-bud7cMYDA59VOBk7jk+RtYlAsC>|q68u)ry&^}OH{zhJ z>+od=qwRw>Sg3~`M$crk$JCnV+6BraK>D@l0Ve%c*RefevV@$rsk`P4cUC?XHUCiU z5u~4vp}g3k95S3nxf$kE6zn+OXx#6eiQ!bbY=E(!VZ3wzp9JAD8CO7n^=0|Z-};t3 zbh?F@ynOy|-6~@d4bXxiNMpKw?5sS4@U8fH;mI>P%AP}K%|o$$`I~2%mE8M5jMghZ zgYskV<#d~MdGf2T%0sVWWFPr`{Qipa*U3!EC-hj?CHe9r%TZak%FoJW<@9npOqQQK z(7AlIgHy)d(Sx6I=fyvV^fR49uNQ~JW4$^|`Wg0SE~iD9^n2PgKdk?Dvnm%}_)|IC zo{$UE56I-?q#Eu#pr&WEYlZZFdJ=d?NMLK7&)!ZQVNV7#01&3S64O&S>Ygvk@W>8X zFYK0=&-Y0UyGFHf%BCg@r6ax?`WVgS=81?3ygg-d#LB zxd_9Gk6P$)JP`CkdWTes9AV!%Jtn)R$K|@r>vI3#U&#Ty2uzL*V-$4=i&6`^lT7kl z2cxcg2S+@*7~lL_W=C@v@Vf9rp041PpTLXr#NA~mt09zS zWG|jc6ZkqYIi+j>Rk9eJ<(}#l;FK|PJMbs}tCW^c%Fc;VDdDgwD{Xkz>Qsq@KPs1n z8#lMi^;YUC>D`iYS%3>s47o9RwT={ZEGFbFmxNRr4It-)jKD~ow z_HsMCg7Q;VDieC(yygg#evMRw5E^()%hOOiCBJBJ>Mf@Epsh`z&5Yx#&#lAf?tnFo?TNiH8C7c@!q^0a(WR5poWJlvUm5C>=_ZPhx?Yiuk^Cco1YvRGWlTszn8_^ zOJnJb)Rq_JwfrY^fwwO5g$&v56Jm>s=N|S`Eejz3NYA0QNeeS684msY+s^KqpX1I( z0~1>7SQ~`;S?N~s);MsB2BUse0CrESn}2aSYcgo|QNw|0u6{x~;(C|{Fu%}p8?@r- zXC9&5{l>q0MsE3^@4>>7UGfW`Iw}9)FP4CUMKV~W!%=~{?hkBykRFp6PrQS}hyhK1 ze20`#u<*w7N9Bbd+aUZBHyYJj5<9~ig z_I~_2;VhX8FCUZddD$%!gnC}+JouwCSzNmOT1J&Ai;L8Jxe44WA^44E5QNf?Gq&8A zf9y8%Zx*I0sPI(Ifxi0*;aIEuHK;*dd0EaeKkLn!EWP+;nOUEb>h2%GLwQmS_fn~( z%FrnEetQzQwj}V#Bag_13ou&Y?z`_k*|~G)yP|-m^G$ctPrap1yEJ~nt&aS87!fdf znaAAR((s5Zm2Q>7+S77)X-Ezpe*z0DKPUrPJ4Kf2MYRZE22()zOc?0L9BQQJXk;!s zFMC%X!c@iyIfwnpFO0xIWeRMS=i3+3cr`UAgFwRF7z6WDSw`spQE+`Ax4D{PW54c08ZP{sjr;|(ft|dax^^V z$ykz0;UmI%vUuZf5HC6fPA|fXPW6^)sbb&uVaRClJm$MYhJ#2)nBwo_GI#B zdE?bqc+OslOdtOfA6w~?1{j!E9ejj$$(JiK7lW@yU zeUa(M^dM>CESb|Xy?#t~6joK1BlrR}ic$L#rZ%Z;fN##~e;r?;SFt0|cUlihVSEyc zQt?d00%Y=F3?>-WUchi`$8Hsv^aSbkhkg(^*>2#PH{97kT-jJ-@kV16?;Vpp zS^ZEzjNy^JJlT!d-niH}xP_JC5^4a_g)rVat~W#+Osr^z+CpcExzI2d5~|ZWMI2ykv`@o}>qR61dhRaO%`4nZ?$9 zba&iw$GfI@?AxaIUH_Y|aTP-txrCo{t-0VXpD)R|(w#EYdQ?^hcFP+pZP|PLap>b8 zmi|F#RM+`IFK6|TnKnj~?X=_^Mt8aEMzOjidm2yTpzM|`w+m8(iQeqvt||Da+SitR zyDCG2RqVe_V}DReg`yhk+?$(I5_kxZ^W-{;d~aCHNWG@z2pY`&p=@qMcNvZ}+5-QP zDh~K6JNQk8RGo(F^WoFwC!WdH@^`<>+2d&-<}m7A$hBb%mZg5{v~&)w3kG*(fyO(Y zxisFn%M2Z-*9^=ak;X&+Od5p|$?d&c@_Wz@M|R=EF-GnA;=_k3j7O|?mGvVpiJUql zgYzdOgS_%EUW*t-F7OLB>lR_;o%62s%gV&KtRI|^aU8hBIpPJ>n=V=n^1RyR--@2T zBqi#@@f@;R=nvJiW&QI7hI_qG&bs3aWWBOlL)%)DvHCHz!`Gw+Ghk)C0{bE-?S8zZ z;8~dMG^Dh?f|s18E;{A5nWX|>WD0r}beA2rMeqPVz5cn0$I^*scqs0pH z!q*YjA8_f6clF{Yw9f#oM2V6inr%BmV?{9up}J&b2-=2V8yN0h;jCTv6PiD?Eex(x zF|m=cVZc%2;Iex*FAuk9zct(troeGMuaJ&$Za9**XAJM$GtbDEet1egw>vLC@n790 zfA-H`km~XhreuiRke!sD{%}G5@i%Rk7^dSqK1-jMpZNq77~JCQiqS{>Qx6}LIT-HK z@4ruePGhU`%}4!H@0$GC6La!`Pmasye&&9SZJj$I|M9FDu#}6JnUU_l?D9wAVoH{m z;!Hj$EKKFhd>F^PDH}LS&%*SUj4&iX0^sXyz^7cg=m!y5G}5wsq-y2V2NvqXA6%X` z%0JNVG%(Ws(pT`(L!5^z0iu={Q!-_$Gp7y-}2 zfG&*el-0ExWw?J8vz*XtPhUWXp+v>VcHaio?T7(h}m+H`+zPKm(IJDg_G z52I}$SC;WSxDR?CgA8Q>=m ze+?TuH0n#zeDxdBe&rD@E1wuP@CNw+4@(Zzu%v-bS#nBD9u<THWRVRgMzM_tS3;sAC^Lu<6rs-{$ z`)y#jvwK+U4^4y3BU}WDICh3m^K3X~AP1Z%0{8S=vVffg-3Fo?ii!b+wBEMFo|8B1 z-}=9wkq`dkd$9Opzx<<*zb^mqYxDAJUz?Ht{ZmtN^Cv$d|6}Jn@_&3|L9;aqL-P0j zyZhxmMNn?f%Wwa&9lFGkWNDw4uO7Ws{)_AFvu^#=arv_)>!9TQweK8}b1>Ye1KUr3 z`=}~nyY+$vsF}e}NXKAfKyNqxSou}vDo^Wbl%*;fzI4Db)H|u(hkRni1RR8g>VY2= z$%ej;11cYoBB46>TY%G!6o4wW*ierMhYYlM%5R_sqrTHx3r71?WVDZA5few+dq#T@ zdk^#^@D7v!^$-7CRcZfLKD$E<2f7G2ZT_KT$9Kv3d2E?` z;k@j6{V^%-yGI6LkY6nK#30pV`n~rLb`i(o6(5TE| zM3bYWG}akLqn@XD?ffj}GMPa~Ezw1B4iS?)H(g@#i0l5py`WlQ+7@}>!CUyr| zUt7lLcSGv^6^zc7Wucju#d<;3)<^Mu79+KoRDsDsFos*W@Z|s#F!KF1nar)qME0x{ zI~XX+Vf3{;CfT7KGEjgaA9&Si94IqSo^4~w41;IxfFb850cqidp$*;4L0>`^n(0ZCyTs7IWfQr8H<&d&P#iB4qr~Pvd|upbG4GJw9B&A z$wLz1YYHXQ*N|dgO-8bH8PA=PvMk~7CwxcGk6=d~%&W(Uc0ax%^wX#Z-`<8iza-q; zzjghSVH)k+(voeli0MfU80~C-O}t>REpm+oTMNrZ`REC^d|||wA1=D~wy9q!$N;d3 zOUE^E)iW>1>`F(T>c^JI_<~=;83^3Boi73Iv<ZxupcAFjWhB`-(S;YjNi~vP;L#o%<{8*`bPtp3O3M*GNQ#D zc-4HvgG_ZX`GkzjM7pZ7>p0~yinv`!F22*(kovEn}~Y9Jqf&nCBTl1X>QaBP$H)Lt$VB5an4nD6V*Dt z>2sKRFz{%+&lmQ~^7>90Y0P3?_PCsyACw?ePx+#4a*tq0lv`QDU0hPQWb1qpd-iN5mQnqlYSV1gB(d$ zXEI~19Bppn1>mJMOb4w(S+`)s_s_`n#j|q5{L{K8ePMKuWH800kJB?ii*n(7bWTa- zi_OAKeX@4;f{Z~X&DxqQV2T#$LXp;n$QoZ1*6Xk(Y8j0 zf#SGii!gjkWB6KuMQJPp@{g*JoA__g^cUvA3iEnV9zow}Xc3Slp*b&IwPAU7wbZPL$*`pWG>b@H?mE zcYpn>a`;1c%g_AC0gT3v$sL>w7~F-kC*;?D?J4=ra@-Nfp4!C656k1rH^~Ppv+@-f z=E%kz7@+@4-#9H__~`?3>aoM}8hG|%d0DqAPlOl}Y*l^=v0vqB3W$QE&pV{&=TPYo zCopOcP$~`2h_YqjtQZ7|Tg78i$5(oxE|_AYLNI!fTu7LL1LU0c*l2%1r^j$kd!>RA zButNS9!a<7dJ^bK;2kIdc4b`JR~?c~)qkoejA`Q0Iiwx{PCtyxEDVv#(5O7Oe82q2 z+8^Ph>q($hWNBqko}b4$qRumt8`~$j$s3SQJjzINprX@qIoe5`)Zr8iLPo-Ae1NK} z;8(*D(6JmBL0S&gVUfaeU75rpra9_b-3<*>UCc;d!ZdJ$M~69`$oK|VD3=<7%ldL8gMfHZdQJJ)|K}gczgA>#$s$jD<0*OKn@`E!og*?` zV#ivS3ul+)@G?HTZpaVOnnz83<(I!Ef0?bxAt;X6kr6Cmef}BwU;h0OIrG{Dd~!rs z#IL6hDK5%yOf~WaezH)&T<8LI0g>_T-V{S52QtZgty>xn(kgBs=5@ncRV=`Fg`mno z@q=JVj_+3F7fwkdGlLeOm?6OYz*Fpy0BLQc{UJGrIqlr?k#pL)?W6A=qe{oq^k7c{ zJqf%+CBUAI%R6|01io;cjJv_&dPVgIbQPz|3}UWsu{6G(BA>yyg|<#^{7$-sCnV2^YzTF&Byf-iNH-2iyz%S<%2 ziH0xh5X4j-FxClaz;nc%JK?li4V0mQmzxHrub?c=NttQxlT)~FU1&%FhWyS8XXLuV ztnA1f)fa*c4EStmR1Nul?vKuT*AY4`e#lP+p#k1b9BSbhZI!3fhm=G}PCsfi*_`0# z^VS@Tw2ry_D<_|m<7?yccZw!#E%d}~UTg$g53a=%fgY*=@M zbPTmHy$t~&f_a$kVpJER`tC9M8IY(w*ONd`0`F)EP;;pE2@VMjq29uD4sJ6! z+Z>XUwNW|WzD^c8_hIIF35Fl`rqAVZS}C?l#0a!59)$6ifuTMKBUcSjjuvyo6na|? zb{g#n>%%B5^Tm@9>J+8H^O0)W<*9lZH(DMX ztIK(LxHc`7rIYgE_8}R?p@GHeo3Je-&lKo#j-A1=1Wgj=W^5F=hp`xr6!^7 zdIC477YdAW;Y$s-u;kPrF7%~*+K@|44SBwh@M<)Mqj%%`L#jQBoiX71EWWU`Cdi<%5bQF15a=}9}-pro&~G(XKv+2TYO=A%xP8GM9ro|ceKJ9C1bOjkNof76xLKkh&e_)bfy4Q9C zJG*fA5*q55pU#D&Tu@qD&&oVb!k$~K%9*(f@>u45WuY>Mk($F!DurfQ zWmqZ?%j(JnslfpM^70RpoJXO>TcJ8!-mlYi;M|n}t>lpQ3 zn>{9VjB-D-a-+P|`Y7g`kH`cTThdtPh%y&l@&!j_$F_lbWm;<{4k@^e^06kMf>HTX zE?LwCr!vuTS`m%;7QRw&6rM&szgF-?ha>aN#*`ebACOm;>bhaVWdAwYlRYQbmtMz$ z*F`LLh3vrVU=btoYRL1sYi*v8j`S@o7y=ASHbCNO$1r@xvMjCDS$ussB1`o_`K!eT zuqEd{3}#^`9*njZxfm6)w3j>Pve9kc|4bIdru>+#$&r54Ep9BGV&KPk;hnMwzT$X> zZa=pz4EMMxdNXE72(^}=O@lArTtrmF9;WvXza&q0c1mW~UL6tF#uK-nWLxU?r`wRH zA$&U=i{W{YzWX=J&vCDW4^K--=lLZ39qclY!{VERnP=ttAuPnnj)y&QfzA#Yk}(Ng zx8iVy19arB{1E48k;ek~>q$sEtq>x9qx8fwr$+nXuSG`t6m}ip)ic^%LOs}%z`IKV z)CHywOy`^S*;ocM0Fs7)YZ0TsInr4eo|R!3?=Y+$n)?y?QTaLEMU*| zrInn_SBr8A2K*@YWk0a;66O>)v3tq548oWiz=Fd*jNWtG#6BE!W6VJVa_%!1Fk-94 zP7B-faU`8nUa&9uIt+I1Y{i{QR$Bw0-;w1;zpS(h(&UaSoP)lOsTND8&V_*(j?VMB2MYR3B=?-AM3}=fQa8oW^6`KvTPx7Si$$Z3x$&46-^85% zDyG)F(6~vSUcgkM^;MZ3n2|lX^RmBi4BuC{n-50gxzH6m*Yp>Lu_Yz9p!6C>Mw>Wt zPaYJaJvLTmWNrQwULNZ5?ZsQE8y1I~VD&Nm>VTe_Wj*4gu`YTuj81$A-%} zxvO?VMK{si1)Z~8%1yB(Y%4I`4K6Z5PquuOkWctJZ(F)p)#s3K)N*);-RKAZIny} z0&0YGs3TQyp{@uR6K*5Fr&oR$&$Mug8zAq~=@}Lw!|V7RjzktKN(Mhm!qyWm0`DhW#vKn zMCW1bH1mp-rw+mxEWtR$=zG2{7qD=3rJ9q+XRuBR+bND_S7aoAK}HJLp0B?RJ&BRx zgd(E;;($jBL&j_Eek|zh;Oc{M#=Xl2rHbtaX-rd(nn8-)QSv+`6vjB{x8{$_q1?Sv znR>5G?$|3MV>r1I1}8_>ITeN@=roMX=rrw_G$iBl+emx5{OO7}y_t9oaz$#SH<6#c z6!ddsAN9dSt(-s45$6o2?_d;r6{FctFPG%ixs&oh`zaa1c8kU78}R(V*G@c7^0@)b zna{$I;4aOOy|)=ILH{1;;K%HB+m5VeMXSkci{i<`&bjyfP@ zfFl%w&J&G)rb3*WFx(M0JU=bk!E%7vtdN`N1o6J*4Xj6~R#$Y?DPv|@XZ5!-8XhNf zlF^7DDx})usxGz(I!Ie3DsANErx|^myJWA>ktr;F^rD7L1J;0G%<{z8Bk*rH=HN>v z@tZp?G81n`gLlLAdR$`#dnWgg!{_6cfkuPrEkypR)5i~?P>8iy+iz2oLsL5w+llN@ z9yLvO!u?=-gkoov5aNvjSdQn&(}28|^)KgHEvJ*dx}O{Qf<n6BG(06P^5Jk!$-zj@s8d!j-k@?p5)Hw#CIdk7Da`pYOQ%7|e;^qJY^4*pGx zt>4qarE)1iAHmM2aG9~op0`Rrb(o<%h}%X$ET!UzwWw=&agG}OeiaW$yfe5G?0kEe zl>L*f;M-N@S*rK3L^ze?JNp>Z@;2BjjK50+eS{BFFeS4JGd05rUdC7SMv$_qow={x z-HIw%9O5ezI$!F2eW1-?c@r2N&E#NmY|;wc%?VWtSa4lL8LrQaPhmkhtg}lw)E%L- zkrP2&SBpWVy8Z0Qfj<(u%^boZ->r|!MI)1!QqJTCan-Ju$?^jpCT{L91*4b{rw@>` zP8!$7ucg02l^f#(`SX@(lsG}34^E>45c9F#H`G9|?%Cq-`*HK~)>rP5si{zR;N1diocCzY=wet1`d zmiFt{80?<5pxY%e1G7s5k+;g}=aun*1mWK8_pAA0-g+ANCI@u{Ki7}foMy)hlfy~q zrs#DJKK8-=*{^1G_FkR6EWah$U%Yv{tg@)j7GzK?C1)+}v zj(S0Aq!9+LSW)*yvw_wNdJsiYcCO_mDbe}65b4<6raskpowW1=4s?Lfc;DUN@Szw9 z&7vXarGF?*U_A87@k|fdy$zRa-0qenk{ZIJ!hC3Rn18u@7mo|%d<6?gD_9RAip+0b zJ|qy}#pu~jTkal+p`Lm_^OLtSFDlig@r6(zBLWTm6|;G^F37XMtnz)Erc1g+{y6=J z9yK@i`}!h%cq|oLQge$;ZZbN6PoR?0ag>S>G8+Z!vJ@2R;gc6@zTpM$+9MdO^`E+IhcBEW$qL0r3#ahRYx>$A{30 zHV@_FqKQZ4En?Z;@7FR1nQM&PxiWH#0^XuwVhwG%R~@)u>}=kVUr6oj#Cik!bW9Af z10T0I(StZ*j_M39M<8O`aer~+-z}%dV5Q>>CjNG5jxm%=xz)nHL5;-F$H-ntvxZU?AK-b~2`Qws2EJPlhOb+9c z!XCyxcAmXowTp6Y0KTEiN>$CQwxrUpZ<9P-iS&F4?I*(zm8NcaS+ESM`>)k2E+j$<+mj_=0)m%0Up$0O;E$5VuQeAT+FB8y78Szy(}9Vqb_0 zNt9h)*+5VYsH9O>@=bwEw7c3fi#?Ln$#SRj_+9Lx0X+A^i8#H!(BStzk2^+;H2U4$ z%3U~*a%O_p`Se8h>$OyIy|5#(^6xuF>x+|Kg;I*~SqpHoHeG*ZL(3K7uC0qv`pBb@>mBO7BqHjc#*;bEkyahJse) zFaHK45t`WNx+7*h1;0;8(xiFEjtYKlQzZuAX-m|CvZM>)r#yQaKYK=iAOgOjiy#PS3IbnjiRs8oXp$-wboqs;!lPdcx$@)J2ME z;`s3BaQUFQD6Q@Uy`u+}aGl3>Yz!rS3+;W5Vv!zcLl;qt%QZulb9IGsntq#-*~g1< zv4DuUl_P9ZJn1z*4_dx2=x*$o?vt(JOB!V}GA;EvlcHxlkSV}P!-kTd2$RYuQhpSj z;1JWn8*=VVT;rvrVE*RaK*3w>Zin1WhCMuu3}GJ8#Y);c;Q-p0VMUNVF@k&pL^siI z$*cUcrtrfqG);D;!!+<4-@}8ERKcWP=Qh*6{e$jqcB3`4D6&>vY9X^6tk?M)7&OMK zqWK5UX5#D(vENFuA~${kk6u1AxVU=%!7?*}Ay+^cBh?GA&mrtduyY0yfAE<^s)Jlk zfMVfL*&W0wrp4?o`e`#AKT=y!XUd^mI|=H^IW$m`#}6VOPng=DhD?Gtc7lGwuYw&h zI@oF!jr;aRuaMu_gYX8*e()A-l5D(RO73VT{BTJ-_Y9-Se1|+OwPT?!=AHBQIx^ zw|lCLkt|D{da9NR-`FIh5^jG*Uf8j5XO%w~S6osJcQ;RQOc-#i%O}bsib(K1OT+}cKv5GlnWhh$TL{W5;nChJwx z4C=$;hc6PyszP?CsMcpS!`v%nuD%!{axWiee3rwi*^5A8A6 zylX2`@vyz8Oi`eRcUMU1=Hph!x#W%x9nhJS?!rBs28pT#Ow{BU-dR-a19HQgFzI?EQd&%}yOaRhEF z7ROo+cfrFx&IO-^zJq>M(g2((YMJY}JLL{#@rqA~SG*CGI2^f)qeI>lXQ<%B>$m^T zLoA8k!o-_3aiJIYIQ*I|hXtE{Z$m*L&2iNo`zMZc0y1cxwgT=S<_%1Z4Mxew0jXV?)+KggQUkwW)pY97PFjpZ>B9E)XU270Nz6mm;Q& z!rOcC*_RCs|HLj&JmIsKmkL=De{bB`29on4|KXT>iWSqQny;F1-WZQYRK%I$D-%zm zj(|RslWqecfexOM@oy!u$oMTg)+rBt=bMs=+-~@EJM4cBZM8-IEHBU}m9Asm9aa6-vxhP%ULA|%Xl#DJpP9teKadA zU8a3|A2Ze7uIPhgA1q6lT7HKxLH>et#2IcL=T{}&xQND!C~-s1JUTj_j0#?V_tWnR z*`BGxP#&8<`Pt4BHso!;HIL1lV%c>4UyuYm$0!*ypp6r!0N+PwX+e(xoy$9FMPlo% z__!8Xp_nHocYrSOqg}|b9YxDP=?7x}(}VSQf zE}6n_#Kejk^*|V2cqH!8FQfzZbYTll`<5yEOe6_$6C(FTf@7UQ2CK9_Vm2}IQ>uKu zw?n-jcLJD>%0Dw3{$?vl;J5Q*98aqEh}~euF{GuBRb>XbIlgMdO1LKf*>V+L>PQnr zM4`OX@Put}UXnX$VY{?|W%^B@7!Lnvk_@8`$YF`UZ};Us+rESDwAS$5x9v4w;GAEjcQl+zrbJYdBHOeGpSx^N5s)IcJ#pT&@JN%|9@-_ljqR3131Yj+ zq>KAj^qoPWa<|JmzE|xET3SiBgm{uRT8`=UvNt%LDkwvbpg25^U3fIRkpqOY?#Yj5 zbtA@NuA?ERapzgCa&eSsoLr{lP_~5|(5I4l-vMhKAf@2$q%Pg5ltuR_;0D#~rFJ~l zPM89a72SnQ&1O)K?zKiNv0in|X#3qV0-i$>>>`)H_&OdeL>Re>=APeUE zfrQ{Ra}?-8i(R_!@lwDk#B}QDW4_cTu6H7|Y?~~|aNxtmtNxD> zZMhtzRQfMOfvx?3xb*p%q8)}f{Ount=Mm`O5FNfbPty9pEHK!T_+bc!{zndZh@dp45QBPNCY0OuwkCPN(34GR|@!I>Mzal>ksFspbN|zH5|0Eb(>yz zmjsF)>_r$MnmOuOfe}Dzz}SWfk^<50ZH@!jtJmT`O19^244>;SxXI&3BM%)i*Z{mB zpm)9Je!sFby01nJ-b3k%{nNS^{5scIY&P zz01XC(2ZFXY3u)#M@!!b{T#+4L>|F~$Ql)9vzACFcwK&+X4vm>FPBOsr@f@mT`?u^ z_L`<%!K7z3zSWgvY3!t;%`aH8as(M%QBlG4!i63d07mZ{sOGVcOog8R9SR}#E~i+) zYyCVLQmWMjQ&@kp`9w04)Z0V=bOyo|mNw@RDW07;-PF7v}uM@1+NXG2&7M zs8Z8pWd4`fbnORos4kIFY;eROGtA*SIY|(e z?jZT)c6CvyM%_qUaMjer>~uWm{r%H%oZ}UXSu!^P4_pA-Z*g>}2HJqDA2iwukTQ=`HtK2@jfSzmRsz@lYVJ+xME; z_{ZF93$!ke&9I~tveqix9s@*p$Hq=Akozh6I~%MVE~+>`Mvr$ za}ldfMmaIdPpkksh|c>d?$WX{#oPPi>Ek?4i8K~;)8GfyyJejd%I@z+kR42Qr`6gy zM2EfS$~dMS@O^ZjefTDxXa`En?K(YlQ|M)HEaEOQj(zSupLs31MoJotjvmY8_592_ zed?BNe?ZW#9$R;&5Rnva^TA_Wnd|B=YZl$*w=mcME-1FQ?A6P6^) zk+J1`0cXjZQOe>)K`_pp;CaMH`tjslrw8w1rpbhwP}kH<46zB z*IVJ#R-m7+-Qn@cn++iy$XrHX{Kd@7jBP|mqh7cdwhkwE3S4Q4!=NC^<=@x!&O}$j zuY5dWuFHJ9`FWpnSY#hLVus&{@j#9)E=_^Gwk6rOQDhP|7Ef7xF>D1qT8qb$J=I*? z#mXZ64Jk-+9n)rhg?XGX77L2FpFXv-+3%D zwKr3JG%F34b9g?0oXj1kFYoTU?7rlYq1G1MZEv5i>7o1W+t8QoggyrzitvtGsJqZ| zxd`4=Ys|^11vjhiaH@dqIFIUm(#Pb(Q|Auq?MY(M^!Kd~-@^*It&*gZL|rZ|i}sTF zNAe?``R6UZp0nv4;zh4lAKf@TV3BL>g*0QU&sXUU(x~6xJ9uD7{hUDBlw5`;^~&y| zg}kJs;*wGSz(9&;v_~AEd?5@AD5jS*-2V*gp#9^``%C)qx?r+ z;sKZQVBUaDG4fP17tyR)T=nquYq1-JbR>tUaO&wX6CFTX)nM3ve4qwh`d zHSD>+E>OjF4H)PiwYjoDQJw1 zo4mir)Vo^6!BxDNF6{xs z9rdE!m9EV4w!1A_s0lsWRd($?qDZb7u~h^io^QJ4>-XvG!?phWTxt`;My~Rr!SHqJ-(HLI(bhw@kY`G+?5y z=Lv80nCJN)*D@S&X(*VBFzwsYIjtR~QnPDI{F^d5xtv8dX*21i-#0md<0gx-7Fjv3BC{8{i}M_8-P?`Zq$QMUohuTVsr3TdTTDJ z5p1j(8xa+y46Vp&(evx7-L^zbf9HGL2`OeG!{dK=H~W-Ott6+xY16ioeDky8O{DJ# z>M;#$L8#ymaKD;&ukcJNxp624^(rk7F%k*sVs37(0OvKSxaQHTp}q?J)3N3tC!&yt zi>qB;>)h1SZQP_5eAY3VJoltA>e3DOAN|@F;=f8B?ztg5AE>CRT0|osF%AB2fiyS} zqc&kbr^blsn`gqaA47D2KE(1sV47T$~9#H^IF{ z04@_AVj}}_@FvM+&?V><96oLdB1SX>ga!Ka=n=Ti&R?ey2K3M0hHT*r|5$LncfR7GZ5w=bUGjh z9d^F&f1khRy4I%DT_us++C=;umtX52XG5X|{B(tq8udn(|Ew1y&Ol^wgA{Hve0gTp zal6sZv08wz{4|3O_`C^jPj z=DmMJ?b4mHS^iTm(m%wKWX*6XJ-_7spN77bp8vm<{vm4$@#kjdC5L#*x#j;V`DZ{s z3G*VnzU=49{(oi1s6etC)6^gBj{jF<|9&O#g)lSN<;QWV^8a-7PjmN?{$y9jsJ=RF z|EK@|Hk{`NVZOJ;pXE&Tznkmz5EVdvzK80m*8b-6ACmtMEi?QeEDqQBa$G6@ySbfC zBKt8fIo_I+vvJmc815B~?v?fw@Xo0J z(}5W%P{};xYf@gebuMbS%b)*z)rc@t&Qt=LCRh3Y((x^R{%`VqeeIW<{*n0q!?zst zx1i*q>9T+QZ}RPZzXb*O*r~Q!{J**Shsi9opPmk97JWs?R{v?Qw-iw93FXoT4EpHbE)@NDTJ?$s`)_WAj-Tk| z@ezzq7o5L=S|Mrg-p$Y|70pa%0UP` zX&au5>}toYde?j#Et0nap_2gtqjhj|8ohLFqoA$JPf;$Cy9!$gL_}F&CLuDZL?VrtTSObmd&XI!qD`&X)FKtNHbm#Gr15^p*I>E;78ARtSI zfNJ2l+b#p4g?-&0aiSn}Cw24qrF+7+i-Ozf8AUBL79$WaV;t@A3+M%|rfG&bv9pdW%W z@V`smpZ{n5OnqIQqUgxr;7e{dNgOamUYp~o5RGknj%9xBF`w*%kVh%{=WKgmy9n z4AtIMzFhlBrCDK-!Byo1ibYLn`7m@%cZ6AvA8N>J$S7V$G`ForEU0?4xT_kGVZj7~ z>E_?lX|mqS16eY~lB*5`SJ4fNZ9qT^tD{~i-xLgz0G;I9PTH((pQ>~p#Nx$ zWAFe$(T$;3A+1bc<`UVXy;%kn&zl=!}$4P zsm>Pt06E+pG=V7>T>rBIhhqFWj5D}gkodM7e$X~D5%-zMifQ>EXJlp~qM;pi34qfk zb6R9i!X&C!W=T@e&TYetD3Qw+a&ckpajO~^ zRFRRPF;){8XEszAM^M8!DmGGS$;r{li92Pxf7P0KG`fX@_skA~ z+V-evVPPS!swjP!Adrws%dm``nz}++2SBS{G7q7B@y|%o9bqXeCrXYFC~E2kh~xR8 z$@2q{kdXRscL41B<)X?7DCmNlE~h+_gF1f?PFyz71L?l6fBCS}Mxtw^Qv}!bB@&(S zo}ciI?Fl(^v6(fn!a|jxe!lf!ar^)#D(Z@xUiO~Igf>nwG(g3Fx3@HypO~j*5SuPI zy5p9R{;U+*W#(^4&whZRX~pH0K}@YjlfG8O0f+0BtC4$oo>;*96Jg5-22qBk0wQ8q zQv}HdqGIb&amsHt8%^}eLAlp<7ppph0n5fj8hG420hM3K{lYrt&ER?Y3@EwYD1mGp zDQrh(hGTOkB{oL7Dl*$%U&7XF51?yRv$f#H`HQqm&Vl;S6yo9U(7)EPYmIMz(r7lu zpR(;9rs*BE%o&lTZx4szc|r(Xc|q@uxo&2#uGGFECqAMXv!oJNRwMm+GF>^nC?}x8 z!xt^-$3uzMXgJm%t2YUsVP5SLET7gf{8!M#pT9DK5GR=EJ3rBKd_PSFheCG6;WwiX zrHd8AveEQ|-7}F~67qxKuu(L~4b=PLhBbmarQ`4Mh-#WQjssPr7~B%NlQu@gT8w6S zuooToZdaU|+yR^L=5qC5@QIz%>5Qr=`F{UBC+sK*_2z7-*YrB zX;FGvpsaY#Yb1iOYG&Kn{<2}V{RV9x++YX(2#4LQZ086?xv-q|cf+@^j%gKNj1q{7 zXsek4Z2_E7mpv{AOA|QlgY{zH;xkDh6Scm2G5y(^w&$noj7Dx23j73BObAo)bmJb@ z4EGz>Wkc`6-L1!~Zl}}RI?Dh${)#g1DqOeTzO93A-%0ihN+RJrT`?g`4Cjb>%n$ZJh z*}x-^W}v(rQb-vZmT6@pJj!9t+ksmN3AYt+hLlB7`mK(5BwaG7q(;y)SKBwQRvW^$ zCyo_x9no=`8F4LMe4y=@six~}*Xq!h6+#UFf&(c4)VLz2nh!f`FZ#;@KQryOz}6lsgkFc)saRiKc)@c=dM8CxQ}}3jzZqemLD*_*!Lz1$ zesgLDe!JcC!R&gm71v?g1F1Y5r`q-y_dg0f3u2V39GCkkpY{gr8Wcnc3Kuh5!pb-_ z7vUPR2bX>9O-X7kq>&vwB#(g%SI&ySu#7Hg;a3DO=Kk$Y=j)a>QMT_@8@a>uV+clu zfkeHQCUC8*6;AVY+);ZK0aQr;tPrJ-urRb3Ow;~2C#lM&v*V>I^;HimzMsz_Zed}e z;l;{T4}Vcn5y|EBoYWNi4$L^?+0lid`+7;Zy|-Ln@^3LXaVKG38o@nxOFT`Rh8c|d ztZk7DX|XRHL_AJM5PCiUiS5P{)!dnsDlct7-%18I^!xk>QGDtK(ST42=W*zg+Q9;< z!PKX`0?9j<PB=XlRUpv=3}P&FeqsFuPKKZDmFS=|M)bBx2c>b#R#`Dk8KH zQY+06-XVJ0F5MYiUAQ$<|L3pU-Dm3wnhfT0Y#|8s(RA#f__^inl-Ko6?;oR=rRA$$ z)`?Vt(FTq&ikf|OK8VMKFl$>|18(Be$0k2qLvct?`TrjEr^tTd`@S$@T^iVj!al1Z zU-0C7ccn*U85rHpJlBX)NmgPSBc7V&R*mH9zF<(Np&7~8G9#VF86T>K zB(Lzt`n|036GuYJ00#Dx8<*EpJtaHbQ)JCOe!Gb67Gw7u+QfP127d=*o@teuuSAu{ z*W?o(V14bi^QMAU%JV)^99;ueq}QnX_ivY?Q!p4En0(zYU2wD&;*TI7A0Jl)d_Ipk zYuhZ3*GJ2^LSBxrXf_|=a(pL4a{)bmoHhWT2J5ax2yGY$?YTi664;8ZFNhw%J~4+! z=MSgj*uE-u5(9TvTHZWOM46PQc!9dhD=`IsEm{iIHOBn?t77&}f2#7SnbYu))E{fI z6HG#LujERuMdwpKMaXtYo=Kg|nnM$>t}B(75Yc&sf$t79jv}$B%4DKDL2;MvhnGGt zi#7NG1E>9esjJDx*o)qWiAh|K*uA?^<$2x{=ccE}umX#sjt#~F+rslug#N3%*boo` zDGFr*e}TN#qe)JQVbcdQ4V{g3T>FIEa6kQ_y70o4wX_p%H6lLJz10`jX7`${g8lug z6Q7%ze1E4E^yL)ZH@uKUI8M8=0*BB0cj6@+{YHhlUu1Htrwq0k$B3mlR4=lfvEgY7 z`yDrXjWldRLc*_RYfiCPoEUMuSKuqi32`+|>`=2NLlrsPrOHnX>gt{{?f3N+FF-oAmA$2LFQsMc`ysp=^kkH@ooX%g9d14R*o8vmg;!|tACCZ% zM&+8#Rk65oS~(W3jatM61gS${iE&^70540v4x)wRH2*L)C3jm>>COT>7iBIu5%IzP z5#jZP`8qF>2k|(;z0gQZ=Vf?plaUGQK@oJ5AtXIiPUTk4<`8SkF(A3#-emxyZ?X<0)#hfRyDIosijaq zm$jY#<~a~K-|Ps!WH3F9D33ZzOJiG}V=4YpiC)W~;t(p%qx%?Hc)a@6fU@|6=NNYA zPCEa2_=2}ywpQ=)V(tCav3;ND%1KW~5Q37~H)OXI`ev9Bg>Ne}{Bq0OPI|c| zvVqu;$iC%Ht>$yb)9rN(>2elHsp@`EG;P0FZ^e~v^{cao+ORlmD+Ad6VB6i`=*eK^>D?svzZQEUB`!Yu(KHM8^c8mkdAHbkSL~p`Q!0EAb)et z{|SnEs14$IjKuR$p*K(G#1@4}gYl+d4Ty~v1pP9p4z^hQcalxdnm^x!b-jX4$= zBNO9#IGaP0RH&5rnP9eK`3ov@d~#G{&a0|$(jTgT0s z7O4AYNJ~7oH%4`b<89H4^_kUwaD!u<0I_qG3$CVIzeb8BVT5&qONbzDf&kYwvp!5& zR5WR`d~;%l7|ZR5nhih_p42Ih50x#1k2eW%4BSsIXz@6_!wi}dbx|?2yenRuP`}q@ z)Gm|zoJ>X|uq}^wR+N<0_#A6-rOqHfx}q~C-w*c`Ia5tJ9Bvq|hf6U&-iTY!GkW)w@V0v?%TIR_qU=&3i0bf9aAKtHLL_RI~bvi)*p|*gxIb$W_zsCIi1M1 zJOHZUma>M*DQ{7>R{kenahUrJ@k=!^ZwkcrjKube$ZmvVi>ytf)HMAONhS8tNGJJw zN?bM+2I8(LswSu}-9O^3%l6d4Tch-hQ# zxmCU4A&yprVU-|6NUpJFiSKB-sj8$c^=Dkyox_$422&M?;pj$!EV#7bO6+pRMA6aq z1J)9%1h%|~B%GL(ED=CEkf*#$0JvI384|89BSkRVxt1Eur47n9OEdw*-2 zS;HNx`+&D!U0d4sm0Cl-yoDjF3Jzj3cd!>PKX(VhB-YNmj$;o(loE4ROBS)1gAlG~ z3r6rv$r`mNc^})ZJFi%HBS}{rsZ(hDL^6; zWvnaF5f=T8=+~%ZzauTw#k;|$qZ3W*LpJsvT8fjdFU}YbJy05~_>|(&cRl8e*PEQv zdDmG`lf^ddcXJOYXrPB9zshO%koe<0|ExU|=BCBCI3_p%DYmR&fl>w({Cl^Y;{18> zTvbgJ%<)9r{|bs8O186A^HG{K9_5#rEHFY|&%_9wRNvrZ*uG6m!x0<7iCc_MT^>lO zP;Yp-*0gDMLe^YY20KMYcxdQ@J1S$-R}2te=>9-KMV)1PN4R##W5&l6ev_{iF(D&$ zfv~i^w-B1cfN_VuU!~;m#3(3-?qpO=fV5XIIdQHoN>AOrsMGG$Y}HCh($onj4uksv zNM7@zUgidl^`9^h|2x)Y&%!VTf1HLZG!-Ln!ZZxQ4(Qr$(tqagvfGyNq*L$bMxksorY3K9Qc$kTO5re^gahi(MGkIhqDvM5D1Z zUxQBC@jay z-jr~e($LY0qR4P&5eAcVTR(z#2=)WO)(_J(d=#>K1b0PjpTt_-=ge>ehYDEPgb;1h zE*6pO#aQBs3FO644YNtj6(HoWfb2qIlAN=VME_nh>Jqw`c(U!Si_7VAsO}im@njd% z_Q@SzHX`%+iO%C-XKC3ta?ZX}rHWq)nx2xf2tn?E%V`KvEIW1R{0pr+8j>rPjz&tOG!Ro;JrRTm^m3&Z>aV6B0{_Bazq0%bo-+JfMYWU9g}n zIUpIo@<$PFN_>Ds`l$O46tc`9Sv2IXl(&+%=vw*|$x)+>kpVrC@#dl!b~6vYhnOJY`RPEW|Rc4{q_@zi2uQf$on~}ZM z-jxLFANOM7jq{?e3|o3SVIw)di1>c(OL?BS7aJUEi(};dX%JE*LGsBL<3y#11CF-? z*FMmvl(^Z=QWbvPEI%`wZAU8Xx})wQp$s=Wa?EQ|ZatujLn7YWAgsrO>zprK`g(@D zbYuC-5L@FBAe`_A3lUEm2^M;M@@@my=PxM&kUX{tWdTyZi_J=ev=f|WWNpIup7RuL~Q zO4&=O;+72VJgJ+fo%&+gSmyJy;DE#Y5gzAwu}iNUL4;-okmnFn_+TK3ONcnAayE+g zu@CB7gj9^E;;hwgg^q~lLkVY=3{c0c(-GbqDWewjW}D;co`>`jN9_hDuRrk!XKziJ zx*mn|fNgq^kKbn`=fl0=afg$pJ4KY*;ZRLfMHk~3xae5`>}l$x!jD$etl5U_t1 zv=dwjtH~tOD$-(SFK}Yq8NYrbY{yQJX-br0nw1YRm~cya(SF;7V!0Jn3S z{E2<9rGWdzmG5!G|%q*$*F2 zO7WwQ#9wiCwlUu5gw-`yd?W|*r)#<0<*FNkUC$R`xfLg`&85u7AzqtD&BC}@!%rvu z!?r4>TBx1T*}sje>6qj5HBE6LhNg<1|A3MY_yHFe){{B1l4BY$pZzH4J;ngWmeaj= zd>Ok^Q*)b!c;~2%*GXZg3RD{#+pl<>W3w+$fQh<~af{J>XI?g{7%wIW0W_#+x)pVn7J+_C}ZsaP6F42lwjRv|f1dBR< z;{qncUs98DRJy;HZ3Pk2^Y_t`oZ`3tK(!Bq8SvU7(JJM>f0jzQS}Uz3qKyhihX)e7 za++>%WrVf3lp{jXV^y-XI~3cyB;LA^Fhel3TYi#KM2L1li>F1z@k&f+hUEe7iX``w zU{=?teusIBZWxegUUfB)2WW_PNw0-V8@ge}D6;VmMrM5UL**_6T-=U(#eP=0*j4KM zY+zE@Q(kIR^}Mq3NqE#2;`@mR_uV<~KSmKH0Rj>d2pc#;c`#0&K)JN>3Tuc$`9ykT zrFkwD0}*%R9or|gks2a!@T#r-&kGi|h=p#qG7E;D{ij}n;P2;vZ{AE={;=_&1$->| zq{nAiuk?N~nZPV{+Li!&oa>2?RX0gwcm7Da1{~WK9hw2>B1m0_)Y|zRT-O~Y;O|G` z^NZ1;d9#(bmrn3h4C+`njz4IjY|>>4lAMy?Lvy4oMnV>dQ!!cQB-xNYKI3_OXz&0w zs2eeqgG8ap_ptReLLv7X5b$xQ9_BbPX;x$*U$+1rFS&X>!R?`BA0OA({r#^=h{lw~ zy6=R=1uWUP0zTP6T!(7W__(1brEI~Wkyy~UL=le|%Cw;X5#>x|aybR0oKvC3k*N*^ z9fu7X!O<=nUD#bGlHuM$T)ET+s~X_1*LO7Uy3GJIq)URtH~(ka(J`2j#rxBcl-IV@ zMZHlepJ6s*vO~qHP-wZdb_F(49Qy9n&@(D5toLD)OVq&SG{(YgirC% zqyzGMwB~o`m7GsGN3kJNgDY7#qtV~D_MFj)2V9*2E3G(H9IYPNL{~y9;;}n<;z@$^ zBEU_`QjfpAHTfMmUaViF);;8DvqZ-*`9ak;?AeL-Ps&=b_we+a@{*5M$EBeZv@`+` z2b{RO)PWb}bYlO3dJTvX(9i-?mty(&v!Kpfwmiv;Wi#3IVdIWk*z&rB7`%zR(Rb!sm;bTwOL; z681YO{OROsd7)`ZKszP#wd7|sK;IMKL;nSaeEktt9RcPssX=%p#6T_C{*2EhR?nj( zNE7VCZ`0iUaz=Hr(S^ON+XaI>;>?i}&oh)v@_F|RzyGMdRV?L?z1{W2wbA5=$HcTJ z85)w8C_vqW7Q}XZn6}UF3!sWUO4$*I-wHpkNe`J3;Ff%sOVoA+At|;krmwJ9Qo;{? z!9aI`z^#q^Y*Qv#Ir@nY2O<%Sow1=Z?Ld%V$uc{8fw38yys1BJ$UFSYzC0K}vVU;W zXu02;i)S}fKyCKZ7|4)2Hp_BI+AeB^52C>7l!gd6*}UNu>{*O?{bp4SoXy7W zm;elCQekb|QrxC1v;7(^)d0cg+bbn`vv7-Pi0UH?z+Q!>@V6+l!<-Z|I!abV^9m!_ zX{bwL0}uogsEK_((;vc+gYP&DGmCOu_R9b`rj^4-_D%oy?Q=gdLJOP3XW*gSWfbsh>ZF@s& z;;LR07z&5!s~q!c_)_=?5h&hVsqiv7K}Ep; zva7Cqwj1)ywow(D&fuY<-Fk4p0vT2~Z(0C7BLfKzb2mV`67bw%?r?-Pc=%wQUiwv} z%SGAo$J@-;!H~7U|3CySxq9`5lP;H0RuVK*Q}bres^W&E=!6_vPHB?_Wq&yKNsT19 zu`?g)>pqiz=P+$c$Xkiy^Qwco&U@9FWH20^%b-JuN0fIso=tsTdGi|27h0=_LqgtW z!hYEX8nKUZ(DfM+=?Ynfgi=SU-nIpNw#f>z?%rtJtE8M)~{KWDHgL{^KXTXe8x(pN9qySqbx1joxJ-$C?cJLIw z;6Y$AJfc~aiI=zF(%BF-iDC{bW-SFJ#Q9k0T=I83Mkrb@b$e~WzrM2oYV~1(u>~oq_=ZLW zj5ORyz%BDyG)p>dTyo#H`PNp1QA{?ll1MoHIf?-iV_Q?=t?kyNC3w4&=Md|)Mo7D^ zSE{|ed<6X5@Zdp~Xbi^0r_Gq_diP&8c3wbv5yo|`JN3nX(i_&kjRg|BZbdZ;0(Ukg zc8V*0TTI_~cmvxraQMc}MeV#D4ijj+3MVZO zucWzO{Sn;~pzwKf!eX?V|EQeqkQp7^7>qZ+;p6*Y(f52%IXapE$slBtG|kF7BPL{5 z&s5de^?n!rtm*88f*ebDd6mEJuOCt_U6;*95ltOqAXPZ7-c*W2^8(uPbmQOBZNib zXKl?KgN9OFBb0(x=I8TF&JG3JzW!A;I1-wL3UIvk>B!wSAopkDHPdbEyan=KQE3+q zAYefcWsWOZ0@M@m@H;{p&=<6GQ?d5}Y#9bp6M->m@Y*UGXLO^yR<@hH&NmsH1n2?QD`B`*@U=nb{JubyYwQ=X{BM8qZ_I+4VkQzuUTGwxzX zg)%eZXlX)KaX#OlW(*%Tf-^0N9JmeG>)TP8-qQ$Z1TJUkc?#9^53>7E1WJled5L>$Y%CJE?xwDBFYLjS_-P45 zt8E$`{;kG6U%8`Cd_JJ(^5&o!%-6ZU^UpYp4xP5AaF+Mb8g(j*Hr+q1uA(!j8^E{{lh;aH0Huv7cg}0 z&B*`kEmR%fjrd^`ySakF8P)q+bsCt6D$K^MI}}kbqUX|OOXe%-;iFGJk7u~Zbncum zeCw_pE)EUG_%YE~_3lR`tZgtBqJBW@F)t%B%7&l*=vFLWI+diI>{>BYHqlM*Yp&fS z*Q=?MO`D%bcH;48cVg(URDAz|Yw4~zisv0XPdm2mLbh5+ENL&>@CI`gF{nKVSVNBy=oZJiE2((tATdq~UG$U>rMT zSYM%0Ff5u`h8JIWiUa$;%^(Psym8&_Y~H*X`wyN#lWlQ#r#r>TK%*rz0tGQ?D&ER(>}7{H;Dsk2 zhuv=Pl(xHrwe%_;Of`?uWk--%Q-tv)2jGr~!qnMwx=XI(d=O}-D)8}-pTlL}`RxTs z+NG(&aYkSN0P@zp>eF()+XJgRGhkfp;tAA^N#n;NE+&T3_wD%GQ_o}Sv~c|3fzj+# zfMfY}sH$v2QE3UK1g%sJIe|z1`UFXQGrsoKtEp~s8N$R^I?JN6I=S*8lJeE$&)di} zPm%o8;>9-#X#&xWZ`^w&rPOrGEbP3!m}&Nq{0P~WKhYkh`U{ox!YOGF4P_^w$hzpM z%Xx#CEB$e022zNAJJUPq?1<--keXXz&t%&&2#^xxzXT~tv>rt29ui`wLHt6b#vR60 z^IDM3fvc4|^JZGb`4w|#u`Du>c}5MRRGsB;0XI8PF)zs$QSaW$J2E7lYBEe?VQ8|MG#55)ldCGr1%lhv4~I2R*PEA^uAlsl6TS!S0$ym4b6>+iH*T%E_l>o z;1MV(Ekj{`G4b}B`QQz;1=|ppoQCk2INo1*F1rR=j7DuK9gcC~Hb;UOF><&z+1V8) z9di~C7%^f5CQh8F{zSK0hx3ns7!O#u=t`{K!nI4DZ^E>3jYv-{$DjWA8#K^aVNYT2 z-o1G5-8XU5l0pg*j8rEILMT8A#iU7;towwiQ>P*_xCyql?JBlCARN^pU&E918}k19 zA1DvyKFQUf;f*&};YH4zSA~9^HWT_b>nP?g7c99FTSm<@KbM$Lv=6a)`|+pW{EF(U zhB}aMJ?8#yu9f?6)mvDy<83nvr(%boDJlUICQR_g>7DKoC_lIbL3EZB#DUYf4&q6* zzae?t9JJJwQE1!uptQebUhL_}kOw9Nl8s(**%ZA0$r{X>F@-N&GhTh`T@u5B&V~TI>C0XjwUHm zX@#&Gy`4za)iPIgNHdOHSSo8c%gSitkv!ijE!2uh_10nQrw7o7`4)xS!kBFd(OeTjCY!+1lRsA^`#^PI(uuGMy>9WHh zHH+k4TF*YbGjZq105&Lw4nd+4lR2nSjaB(N{jk?|v^JqU|D>|@o|2r1;X|`KCD8{1 zgMjF_mz0-#Ka^&v@|=f;h9j2dI)mAH_C(y}nXW9zLrZ-MY-+AVcha9NGI{F|2+~nkFnGrkE2IzwZ?{iKEBxe z84e%bkGrocLP{bfMhxP?`_0v9xaQg$`Mpy*q$*y1g1Ct4p1W_upB{c3m4UPALJ(4ayCiV9D{WgCf_@Ey>#;q1DyHQbgM_Pgtit-XCj84km1Ew*MHoANJip_y>RO76i?MnAS{yxa0M~7Q z9r4Y^^|#TS@#=}oaNWvVRoi>1QlBCMr#8HS*z>gLQ=Bs;jh)?Z52;y~uRz7Ytw@=0 zxxRNqp!YZGG$0Zcfb^6kb`-J5%1BoWRQDe`NQrn9#*K_YeS@2hXW9@E6^(oEz8%HI zWq9wCwRr9A_mPpAga_}t8p%n~B#~sE?n;vIE!-B6%!rhdz)N5p&xy(cop>>= z8-=BIn%fG6El|o9n(PU(>wD~p4kWRPCt(k!2}$46vM>30T5Tmet6`^Nlqr^9QfU(y@|^#{2`{D<=oI6$K|?Z*HQE|72xrKO~rXRXS({Gy;PX0U_1* z-FH8J_uF66EU*J2ahL98`Mh$@;MQQvo;p0cZ8ho}`K91DCn1)zyhE$;!+Yuw7Hae} z4u=KLzng)~tZ|q*b7r?ViYm*Z1()N~^$(*mm*5PIAZ5=KIisy||}fH$ptln;bm%&a@kmJ7pR~y-SDQ5ojpQ zQ_hv9fAhb4Psp>vDV!~h2nriG_00I;Q?UQF-(%d$AN0%QWGcO@9{cbqP24?qYru%R zlo!kwFefXE9!W!${<^q>cJZc*6_uuDJN6!|MoxMf?p%2To_PLc{Ps_epo!CZcduNI z=~G9NOOs8ytI4A`k%*JDE77(PYtm1iI&rcEAFrv!)Ttvd|B8tokDw$(&TUM~X%y8b zkc2a>lNrQvj9QQ?dMzx&2p*Le<`9{+{2Hn=lzxW0xmX7%DWyrqI&k49Pny|e>n(%( zW?-CjMnc-nKFLgGl76!Y$)dVUPlO0KGRTz!X&uJR0gGT25f*@yc(%;9$;@YD;RSL> zY}3oJQ&f{AAFtJ{TM`7(ykJ7cP+mV5JBnS&Zi0A4Ya*GfsVG*s9XNahzxa<|dg|#W z!zB=zOpI4*LF31b;rfU;b!BaQNb^qRW6O?R*mv+SN@%{Zo@O##qTBQ(f{^h7 zuKB2^rKRrd<44n?Lu5h<@1Kx*^>lQqO}qj(H&&y%tbk7|{UT7M>@QC~i&x%!``q6v zK4dmNQ}LAIiM{_R6UHGqF~PeoqNt=4dk-AK;bX^@l_)#inR6)H>3ZZr?fU6v5_zuW zr>FCsOYuDK-IZk4*WEe4LeBWGa~evBcj_@A<}XL zE}HaH9~Vyo@tM+-XV|o~G(7MSujT(NsEA3$tfQaPpp6|#jg?rm?<3zTy6jhk$K&HE z*CJ)&blh=I@7K~L^B|4b|LX4$91)A;@qJ6Mw>|n(<&0_c^$%kBm0uYU#B3BQrcGIh zFaGpHOuXx7eR@a@q&HS8tr6;AATXfBJ&3lZN9M``$3T+zFkHQ45gvQ;5v<)LB*=wr zyQ^{K(wTVS)i?3t>#Jy8rWhl$Q*rxEb4dv3sK_Ct{c|!>l6H^YxoRdY!8oW#vT;j2 zD$8AX=z(d7ji$Y82d&Jw#b!G@D6(~f&dNtsB0Y~NeL;{d6GqcoM%;5PhQm^qbL=_A62jC+p9j*{1I~P8zimy5jW$l z0W<|j!KuW{c^gId!-ra^KEWleT{s-eQ|XMYo21?8w4jk%L~XPQZ+4JE0z#uA6mj>j z&#YHikJH(Ps*-%v)z-kl4t39g!#H&4xVIcHyON^&3f@a)VwTPS-2`bjhhj8qsMZa1 zo2Hv=0~{5=2P2pZq+>ZVG-mX0S}$6NjPz7&-nt!Y)^EhgQ-x^cV6xTaqy*d`?dtx> z4e?VuGNCaq#rIhD&3G5{n&-TiBN3c&iA+dEL`(vq>5@VvwNMANzPtoARh1m5fzM?f zR=xibvkK9dpy;mg02vWy^*4Yw*Txy0NRDd7YSgvMmSR{=7Pf5Lh0iv9fs=GdP|Im^ z@e3e66pV0g+o*6i!Dj>Hc@%iftqHa9oJzv(vG+}9?t3oij5hLa$}C$5Jpo41r03WX zxwvV?ax7cAh=fbepqvq?1C%Mqag2SoNBzu2l}q$)#&962mA9fWCLLjR!)vIhKAvvK z%*@13|KaCcnEEvS_ix8Ab7Bof4sF7a)HbzV%C|gGi#b&sf}IEBaOh+huDtSUETkGr zcLD|9e6la(eI(b`y!i4PsHu4eErH|EX33y~C6TbxF(C~|MBo?#S`T32go(K2mY?81 z{_9sfCyTW_!QIwH9mhN++wAOY{PgF)z>7~miNB?d!G!!>$SFU8l=?Cyf%i>or&>&L zbSe&HPexvR7M5LeE#@y=*z2{p?3_t0Gv0M)W2ct+7JbR@AOG}ul<(h!u{VFeU(#-8 zoAG?M)|3w@HVGr9lBZd*>30tzbIuA3U2sdU9$IbT)W~2*tOXqds463O)Jm~ZA{RNz z2Qe%(lppma_-M^Kyz@yu7R*T}d3RvVx{Xv7aj;{dPIpTv7R;MO`>{hPZEqo2b?Jblg>>%N67f zXtEd)p0qMEpwde$iFgFR8)2cW(BwN3E$5^|S80uoUEdu@G)Pcd~vXf|d%xXvxJ&iM^5;Gl_h7rLF!==n`jM zkZ)u7Py_{sV94;12w~YBB-b~QpcF~l z83T1Bk8~@oH`P+tzMOs(3d_r|@5mAAxJGh7T}gA74RFZ95&?vb5mg}7Mv_$<+h^l| zUI8{x4eF*GX``KkE{7~q;XSl)wRdM3zdVOlWqMP*51-2*??(e?cHE9y6jxVrfqemX z?mvh^u7UX49k;6W3+J*ux-J8O(WA$qqH?_#4tbB#)6>1jXKQCHZKfaK_tFgM^vfXc z(&+|SH2kgmzl;3*d~Dpf4ljMMm)6}{m4&i!3Mt$9ZE2yCzxqZuQj+78^_Z1+&7zrP zL#?I1^~plzU;NWgR6YCmA3$FI5vr{=vV5*1<5aZ_;3rL5K{Jwu^&xo&+r{dFC$zho zxi&}o{+@&Av48uaA0RI;51+4Hi}&{J}?g?hcA=L&-_xt$Ph8*S&(+q2n-g{>pw`_l$4<3XRMg zm66-8^rw?3K6eu4{N1D2_tLNN>CbP#@MT{`7QMDwDPi{yGD9%XDm(slTr9@}Dsh*K zwu)I42C;0CmYj^sXU@V)uRo4EZWxAd-jPj@Zw^eIkVre`cH|eiaP5_sDM_=-ev)kR zAYV+95%oRw5GZxvqJ+`KX=N+Pz7wYkD{(nHi*_7h+Ir82SuzRbC+RM2M+|k&BdGeK z3R7u?#7Y`Zoa_r>m%#HvD1ynElc=U5&W}n-+$l6<+a!-qkZk@@<}ptQM>7v4S5Kr} zrST+Ixwub0p)#BKjV(`b$0%7!)JqRe)ZBdZfa;$mr~B*L?hM`iH) zk&ZTyYD}YMEkN?n;pAAxJ(K#RNoN|wyi_C278V}ehvR#;(R^WrHV=-*XY@`kyMawTSYBs5NlzO@z|7J5CM}-kF^a?DCEBqfSPjXqmP+gqIg~Xg% zk-%oQlN<+A$31QMC}fSDiloe82;tPDn9drA(1Dgwn)jG(Y-?%eg63m5wsRATPaUJf zwNlp0880T10jt>tHy_IIkT9f;9fizMlW3YWn=@dvUeHGyy>*@Dg?RJvoo;DrMA^wh zIKE>ON=piPe^uiMG1s9`)o1?A_aTYTy-<3|H;}>Gj&2H-!B%5!1@aH>X4^J%pi_er z1%>#(hacl&L2=-=n9rP+ymtfLS&jPEH@|vT%kQc1d%t;qPp8f(UZn4tnRlzaVp*n| zR;XHATb0UW7-zua7l>PL$-GPNbm&>2J(n)RSo*cDt3eL3S=Kb(?z86e2n8 z4TxbM+Mke}osH~!?^Q_$;#!2b|M*9L-=C^4sWh$9-)wpZMcY0`L-BESZ86znG2?+> z_ecO>lw`N ziy1dqw*yYxeSr)mIb1Y%7G8Me1$?kNAJ<<+NjX22)wOoK`$;KAv(Re1G{&sgp-B*zq!h70)?!a|EMEc@tjm$HqX zn#ywIvn*0Sm4=6^JlRUw9sk>bq6Mi%d} zkOKeE6KR(U*&llHN;{3 zQa_uFzPrG}tt2ohStF2~HM~;_pW3{*m~rM6%CYSG#>>7SEu?bntVKAncPlo(`!ezl z?^W%RTNIm;fgvRANjW2Zh`XnHvrswo-S?bU8?ym&yJg=z_D$2rz6|#SuZ+kGD0A-_75;EoK)@@#sY~|52D(kfz z8BjS$lq=}Y+#zcfm`4b4az-|$FIs`zX|oYSRhuBrZpoCRBrvPtfeNY`^MG6bX8WF( zbN>0wtY*B-KsdQLY}}kh*!0G8*t>NdEmKH`YLq0;pbX$LhmFM4#Ve3AX{NGKAIL$F zF(q%LGk%TI{oCYUXMc{(_Q@5>V8=OYKJVQkeD?a2IJ|osXDVouiuXM-IuT=MFGM^C zJ2D|>-U{C~m~HcoGuvQp&HB1aYo0UH%GQXJ^(ZdZzZfT{Z^h~tA4O?#K4fjcV^7nb z^0diZn4adFP48<2&IkdK+@Fc`-nX(x|N11Yz;S`TvkAe}S5He#!*{>)t$tZ-Id$q3 zb&{$7ZZQbI)m6qdWYMQJa4_345Z7XsHCVP4hqmIxnwL zE-D6NMvkBbnA`E(bI)M={z?*XE2_lF5p~rcd|*BofQG?kr{vsKY=ERxW8`xxe-dO% zwE~jri4$$OaYYQCdEqb~ed100-NU zqdxOev`GsXrP^p5yJ=0Pu_chU@SVsjAZhZdE z$qlA@ib&G!B*yhr`zWIHSctNa_KfTioEf+Sqb^&3P^ObnzDP^GZS-$@KkDts>q*?# zHA*{apTD7&3sbp(SP@T>=G-xpam7vdAZz?|eh4Gfwa8$k3!eRH)0w2E-1^Z*fztO$7TXgx zdIHXxFr5Qg8(P`MGAOksF0vXocPXKA$z-)uIt<*VB>RI?OI8bjycp}ui_{A zuUx}&lB%sE`fSXHAFV;7wZpngpz{QVW)JN%pHA@Q5GdZU8b>~O0`&z)R6&txIT$kY z8f474s+08k%hhsrb4R7;V&a`YrQ@Ywi!SQ*##Hc6KYih)O@J#9eTRrqzUd z@4Q8sh}^i9>J!yf2IzjSFiK6<9`jc%&i+M*9f@sdP-*#OL z^^)80?6X@iJSP^nt)S%^uG*`tcH`Ms8!>xYAm|OAA4OA_P0Ewt*$R2^;b6u&{xVt! zbr8vHkj+9VqIg}Ut#hUgHYL2?#0F0i>7pVGK7>Seg-G}Y^R#Rwy>YB^cH&z`J~Q6f zs&ahF6wce`+z=1^#2e)m=95D{_Z~rE8#?MfM z)R1nU8J1^$88S?e=iY+koDmpD>2-155!BK`Q9Iwc5tA;%!dve{`iKb#5N8DnzpnS~ zOzve*dHkOVHIQK&Wg+bl-rJGW<{|&+A+zpKg`SCyh7m7>TFmwp5X~+ux_7YF4OtLLxM82y0BWX9xH43eFBP-pCuil=7|N4Ca z{`6QiEi=R+duR}zd8L6m-ww>aESOW^RGLxb&{KvmbqPY)xGS;7kL-AYiuaIak_{my zE-nBS%Pqk^dhQ14_GBk95b|UOA=PG3ngf)Wi-|`GW-@v>Weo}KrbvsRbCK&W|ZDxFLWlH}P#t&20 zm?=20Z4>NVEPn9Fal>x_7xL<$5jfunOqo0$dk%hp*7o`QkZ?^^@O7+3__8f&rt)!(D= zi}#cl)Ztg&gY5Y?BQ%EQ@h+kuy7+D`J`Ri5s)GHwZD*ahOCv;*E#@KRFOYh|ZLKY| zmmbcSUdT}qNiA(NDSB!WcF83BU9H80hPg3yTrh6EJ^_zDTY|qn(|~!iL+K!*4gdO+ zP}aec96}7`vngeGf{y+-KGR8hWgWm_MMX^j)@*FX zseF2bRU-sBC+b8&h@&ciNVUb3VMlGAih9{d8%`&EFGYDz1#l`Ll)7m)E>eu-!l-C^ zay8UW1fdHf1kgf(v`?hlNf|j9F=Yhfd@| zZtAlSBx#pRJJb6v10PX+3a84B=>LzQUilEJ@`!rMK)3O9Z-IGD@|K22$0}=7(GihI z95U3$lx9b%X8MkpFTdBHG9x+Y5l_eR=7-TFq_Rl$<*U{uXav4A0uv@oK>Rx&(HhNq zw1&)QM?p2$kZ+)&Z5N(@;|orfzs}iiW6^wol>)PnG}N_EL~HoHOv?rL3|40WV%lEC zz4v{CgXnXun5LR=$^M9xef!`3iQg(GM&IzDTFfbH+%Cot5|fD5x=Lj)K6dE%i$<;I zd|`b~h`aB_i6_Zs68G2$8-i_ok=+&iW>HN>v0I^!{8Q!~WnY32DA^auyNe5BEHpi3 z$HL1auz5!WUjMKWyY@LTe@*}fH} z&K5)B&ht*H%c!JgK1Q0mD&8!D$=aFrSg{MAd~W2QHi1r@lsPWR;a84iW-!VU$-NwT zKJ|Y4Mj8$(NZM-}f@lxF4XZaa!Ol!3ahH+;gXwrFoy3Ghz)e-39s@}sHQYO$_SD=d zuDb)Z(AQ{ecc=a0WDvx~vs|ALlRAXNJxz7qQv2BoBFR*LB$a|&pEYrm=AfP00Ircp z8#aRL1BMYw!KN4dr18X1$Uc8qN1wGwj!r?^C>?H3LGDjeLlw%4^Ej}uVZ^jKB%txW zk84j^RlH#-=ycm9&o)u(=lvCu>E_@;5#UPieQK6AyYq!^?c}@~F=I_TH@+(Z< z<^B(<-h0a;A`jjw+NzxnMSP~BojbI2mTu2$HCC!+$BxuA+oQznP`=2*H?30vD% z#J0VGJMUPjr1R(PLN39IkACzt4!`|ZlJ%7sx%@s_iR<0W=EVR(NOS^HC(T#n-Q{RQ z#`G&MhGw6aHgqDq_$MTf4-E$GbAml&)WZNmIIG;#X-^B65$x|c+j801;`6YJ~j$q_h zl!k3&LJu?X6t9Yv==zp+Y~AOke!LA6Mgoa3B-Whu7F~I7JA4|UB5LH~F z>4{X#BX$eAPet+!w$RLCO+~ppSae#Ig)kD(o@}a{S!Hpnn2W5dE=Cj0UphT*x6Sjt zZ;28Isgbetm>QqlBT-Y1m&%GzT~dNbO0^R*vz1vc<7wzxgNeP&AjmT$rr4C%Tt1}g zo_KGJ>_}?kSFP>#L0drgq#3y*F49NJ&Y-kN@ub_}dfDqO4*&YOPn$bLu3@Hsm>H5aESS zu(+EMEA@B7? z$GIr=j~H?F*ElGl8ifv>5IE<=T}JUjIPD}pZjp}Ls3s#{OpBOZG(L=8qOPRvCJPnf zEK+ysa@!pAoN7TrEbU;^%jlAM;E0|bh?2H@u1~(C1_$p-Kva=tT})4@sn|iUX>7+1 zO7G)iY;;r+Nd0uSgwj_CyGcSL(3;Xe+Jg zusxE`69Q;~BqlzQj+chWNWqNmv=#EvMzxFm>p#GOji0d3^|}434V{u3?UE=@qeLAd zE;R#V=PX9%=&1%SK3RO4qlk`Jl&PsGp-Hg@9HiavQ^yYyL*G~?3JGkkqvNI0LRujSrPZ4>LkGVXiLR*n z)RYzAz-J%f=#DRF5u{QLP&&Y3(PsP23Vqrr($z3}MVYZ|B_=JrmPth=%O`;+kIbZ0 z6z40O^Fv0^hCHS164#Roic~3txTfS34sBYEqGS7MO{IyCRoAl3W_8-K03q(ti76OA ze>p9}j8*C-#&g*zrT0KgKvov!p|znN*<&Us3p~ByE$2jis+^8n4s%gv$?+r9?WXr* z-zJ*a`No+w@|<{#jZe$K#0A$Ok=B3Yx$g!(GlzQDD!ER;$w6fDP^J$I_MY?Kv5WLqP1f|Mc&FfIWNmVD0)%IDYg^3Z&>LDu^nDpz0o941T2$ z;ACHDuSGi5(H6{`hbdF0X!V!@zDq|xd>R#pwxPMQ2>W0EKO~Hrj))|F^DnUwl|BqP z3vb2l=l>gHZ~7ipXAGxam)K>V@#Xk1w=*u|v@(ihDTET+mX;>8wY4HLT)d%)KDMYQ z2&0xSr2Gl_G{4>^5#@2alTs;>47SpF3HNCMX4#@3Wx~S`8`OFJ*jV1^91F7P z6?WZLH=cXbj*mB5Fk^xPlg9>O{Kz0K)D`cjEQbVERC5d>MpDlYxhj=E!n>VrE6-ce zVz<&66wBMxid7%gh|7YByXi=JWIzT>oakD1zN8t>@#OQq#SEOgk{JIctl1Y)i`kLu9%ag%sQVItMbd+V~BE)WqINK$TqAK%G za6L#X;;DNti)FiKjz-Bs+KZ3w!G_nK#4*~XZ{mVhDWj)g3FH8pgOJXFGejInR2Ai8 z>DPaRl-_7M$es=-`aNFq(;!Hg;DNa@y{fohgy$vxjw7LAsBRQ$Q%;vm2XVgbGWSKrOi2EL ztsBu)U#$k(ius!fJX>j#;iyZVw$9|+d-rL`FPNw?1rus@$@mUp9 zpBZVXNTE3_|M$%=n?BSCoHqnS=Ux897eNVU0&6&%+R{Rg=luSNnNV@?luW0KHm-5e zVc-#{D?Ezw{hLr z7e*7#J#pv9G>mSKD~m~(=!ZFxUQi6HG4e^4KOs#fapG23u}{1lk=9ZY?Z&~w4Tz6+ zV|X?RkdQ4&EB}o|MmCk6KN2S=Jmg#KwC_4VvcB0#`^oJnD05SK+Ky1N?`adLc0&ny zJ4NnR>Ss%+sIj1+)Pj-O4lb${!Yi#bD^f`iMq%F2G*%&;pPZ?-PRe!CG?oSLeC$GM zniIKMff$)%Rez}|R>V?p5gbCgBpiU!eVH^6`|3LJ*G%#(mP$m`;QzPx9e{CESJzio@4Z@GvSqo0aRb}I zHq~G--LwFq69|FOLr6XddNh|ol^u$_A7r<62 zr1eU&wo42YItB(y)(ZWf+{aX3CCxQe3Y^j=4js4KXSBYynOdc?AQu#gN(qjR)x%z@ zF9fztwUv|J*mF)cXbLqwaXiDY;cuA9Oq|tb78k(AA zPi`&-T(u5BjKj0Q@F*m8idEj{o&niX?b%yqMrzB8$f{Ag{GLc=Hh5!e>+*hcR~b1s zeDE4T=Qm;oaEy(PLZCJREHu}Sd$E6JoOc!a)`HZ#>RB1Qy!hTJ`N%5E%%hPBpl;AZ ztt#0bld~1nN3Sa{RvdiZd!8JK=+xz&r0~=Oq<;JUYm)uLMsY_U;$k8(h;^I>WgojS zD5$8Z0Wbgu-0nH+?rB#EzG_PgBm@+laTtWyRZ*R#cix1-V=d??fuUg%9-D}2)L^oG zZmnm=flOt7jvHz0zPv;lK(%5#O2)(qP+#KpGKyh=L)!ujGL4UqSAU0gPk&myD?tX-y4LKwC!c3-7nMffY=>?DYIGL0VjxREkQij^w)1uDe4bOgc zy}PX2(k(MHyJYT6Po)?`;A$qP2LA&;0mOkTkG_>x_kuG|4nH`yZGHGNOIdA?ba%o-q5CfaK@nWehUEG6aLNWKFKk$13UX3hpc@{}*X zU2i!@>$#~&wB`4VgFLN-IK+afpl3b3!A^T#+Oanptk2`lu z3kCsNFQ8fmsOkoIaaG6>ZUSg;u6G2l3W#C5;*!&LBP~Qvdb@EwH}%Ik&|_;4TvY}2 z4^&&=v1q6+$9)5@cFcJ32mjfbW;@$zs;-oRT|X*s?@=?R%MDjwCi9OuTFovDwkz`8 z<{#wA|9b(Fr&aK(+dg6(Ffl+f5J`ZXfn0iYP4zWWTU`PAi9h<50B{PW7TuG^8KTXS zJwL$Hj)4|H=NZdRmdh?Y55O24dZJFdLXX+w=-!{bhUNJ1!pgpi_SYsq9elr zZXNqb;Qq*8pTV>G4Q6aG5Y?Oqz11t^hT#2RX-5Vt3XFpBg&J!g~ zX39A?%cj5IE%R@B94re)MIYJ47F8SoTX+E3#>d7;do#Rdh1GH#bk`NS1;5)}Wb;y0U1AxckbUN+*;Ajp>mAc@O z-@4T-|Ng)%Yu5FMkDt5Do#i7(PxX?-@c>i)0P5Z#?BLCg8FL(^7pgaAaHg?8jve6J z3mOc7K(nvByShz2`U0Bqv-&`T@vsE40y`*bH1;=vE5AG2j3767>~VSo3*6@A*#~bv zvL0eaO}h$=o7iwb>(I7SbIt+XhzKV>I@w7SWN)=^#Fp*lshRSU4izr zlq8#O@EpGfaBo7}wtyJbu+1CSP7M+uQK(CkK{f8zP*EbL+8PN7i&3^xv=%*s^WMTa zR_5*jXm3=Kf~PE5jAvu61OoKy6#{o>+xV>c`vyyJXaokL@StmK&!^9Nh1ef@&|46g z8b2WeobC|x8yRMG`v}Sk27u*xdz3djFu+fiow!&Q&YJ^|yZ(=G2kvu!DkbpzG{@!n z@EHEkwf7AH?_UkhSi`ZPi%6_hAnm3miJrGWNohLPvaGSpAT)!sUx2}UHn-{nc$y#2}#OSDlv z+%9i_$SkLwa`SaP0L9Rg?SGl_#cGDvdpGDNZ-3M!pRVo0cQ;vdoF6i*`buIPGL8DX zgS!sTBE9AAX7RwNfNBz?t_1jX1Gchv7H@iVM32&t4-E;FEs$hZ+Dq8!3&fcQgVXivP&$o?FnEC1 zGB_|$!a{=~Nf`o}y+Fuc21!X-xs;ZaDl6DPX|(GOdQUi#8wDabm7Z?h0x(Dqb4_uf z@<>NTg+s+6Rkx}MSseFdewTbh)|xE<%kYfD;+ z54evB8PjCKoQ0B*K2-uhUvO7^NDFy^myOr9|MF=MYrD)!sthb>zFAuE!Hj|*Ji$%} zGO&jUy0ML*QTPS~V;~Zv=p4GdaMYzjP9ymGEe&PA07*naRGzPTKH5ymepJ;N9}^48RRU)C4*1WpGxHnk4GS0+IK(Uv z5S1vCPQOO>{O2jmZhB$H(L#w0h)FucdO9}UNkRtI^2fiB@-1IT+^nTzbCAQN)dTvb zpr@~(s8qh+uu;0<4%_;}UTLh4k{~~TEqI(Q-Z}zyf-eG4EBNJi+d{mN9P1``|Jny0 z{7!l5IkSAbsT*E0vm7-vrQb{__a{S#*k=d&cRKo|deD2nQKFZ@$zy z@SsQ23aWa5lB#R#B`1F`z!XLpnnIg-frre*GTwdxYMNiOd2kW+$ZxMI%vJy)IVm3A zcKu+Fre>4u+LNVZAO{=3658O5Nknv<(qwm*q=Y0R;PCc9x1T2ziGAiTI}eH_ z%K*3lctK;Zo}bMzM)eDg$_waNXt&17a!5K>A~qlu=Yxo1Cx!h~d~SHbTj14>0jNy^ z&&=)^se;ECmr-RS^#m8(^8DJ>#aG_5^{`!CO(h18UC=8}QJ~n>_0P`T zi`l3uHQ?j@WK(eB(JASmhlN=BuXBw#SZGJ=LRCpV28=!c?h|2a+kup}P0CnSfV$Fx zpkB^k%$klQAa;XRYyC~zeuNhv@d12ZHCK@5&w*$>GsjY<%1PlhYsAb=4F)FWPW+x^ zOn^cbXDsZ_)5-qEYQqAC1&&M$q@Q%Dm@4w6bmJ$$7VgNF3JK`R^N$E)@b{5?_jQk5 zCy|rqLi)L1lJtl;d)t$A{&umgH!9!S5P|;DXJ1NfO})e?21-F$H3CB$WzjrHFG5?~ z0!4Tz`nU;8bU=gkCEbt5!UM+1m6!NRW_q_g_o`Vw{i+wh7-}$lp(3y}A_yM#IRSF& zVn2AbptB4IvX6Tgrs#X%`R-Npm2oOS78pRiig|p?(XEHFe(!-tyr-8wctH}WUTHLm zBqT*EkJ$^a`~c6n$gi&kcn|WBv=ko+2!L`3bmzMJ@sPTN^2TSPlw;ucRg? zfo|mJ6=Yy)U1OsZ!n?16{xprz9-c5@1o*<<0If>SQ?uQ=$YV7l=#xBc^77)7Cdkju z`vqhqPZUoad!T3yJh1WrEWqYH(C%yk;O)Xa8HV%m3l75otN*65>jF9FZOx5JA*Txg z)iF^Kia)N0vnr~p;BhZkYCYpH6QV7wAB@1%ApG%2`Y;W&E=+^g0@(Q>STQmV0mm4a zX(=n#Hn~5qIRivnaBVr;l98SQ8jR~7wCucm449kM`C8ObtSzEX0F}I#cux&JkAvH} zL^@eBW-Dsoq4wwO5M~f4h3Kq4e{ky%5Zw;`JwsVCCZs#1>R6AIotuwiU^-acBZHPA zIQ9hHC@LWhsx7dKlY2sNE9d+4#;^_zcwB{{PXEEElXM@B{UwF{^bRZAx zA30uPW-OAd51y52=ihoD4}C}-%NALTVb=)S-+1RkdH&^BL6Pv0i_eOXH~&*7tJfBT zkIMt|g-W2#KQ8OPV6j1MQtw|>;j1$cGww?ldCJTw9+I8wCVPva$K37)RgH1*#*YI( z9P|!=bU&X?@%1(<^%r+|&v^(7Nn`McY}fwC)^%nxJdONA&|Xb#FC?)#B_uRlmY;T_ zY}*ziufOwOfcHMR`ciMXYNfY)v7XY80g!t0lr#()cJ67Hf4{*qd3ezysL$Pw2g?VU5|9x9crIx&m{y}+d1F(Z#6?6w#UkAvcBrF~vco|_ zb!{CGnyq)A6}-Vaj!a5d9>rcY!0PX0{|7ygW=KCa!n7}L7%I50&xpu?o2B^^+8oa)3pQ z@oWpu*#qE~TzY>1NoU-90?t+p5}J?{ie7K(s#|7R^liq$MlHO`?NIyiMDQ}bw*fjc z0ou;w?IUOI1<6 z(ro8oguvZZ9|?~03JRnRFC~W=0rW8_AY4(kJUmIY(YFhqi`wEMX|1o5aOp`@ML?{}W9(9F-=R(Y=P!Sfp*%VTELZad&5YLxomO#I6oPwi< zy&2g7-F-b}#+1p2EWdsHc>P&9n-e#_7#1)raKJ1;Py4hBZj;E&d9wZWKg-5vZkI{R zua$|XUU9(sa4-)v@zh_)7r#G8(w1H*e&Nn%$`58A$NJa@1h}_BcJ3cgv+Tv^OYZV`N&sm3Z?1_fCcc!m87GS7qKz~k%R?r+q|zCtV#$R9gd_-!6L^@ z>yZvfDpH4?jxTR`-ieXPW$&YRo$ph$r9Lzk)^oU8!3#gh`sjhnuoJxb+~OW-gs@7_ zxCW_&m-Wuy{1P5O4|(^)k0cAqF=w3UCC45`E%sh1hce8wuXICgqz&|x47vT*8)f>` zOwEy9FOqYfot+0t4F@%vf<94+CJgk8Ab33;w)a;hP&G5zOGzO(#$6H$9&|=px~8GQ z#TKYs(A(LD&vEvH1hm7SC@CN z7h}6bP}dHfZc=%?y~#O`Qyyb|emlC1S%W4FM4B;!;0O9dQv7(Rp*Zr}yKsY4ZM}6^ zTTlEhh^G`S?ovFsySr1IqQxoh4gpFj?p7=m_u_8F-7UDgJ3*H3@9y2*=h=Jyo#bTl zoS8W@@8=awZSmdW?)dc>;z4{cH);}z;P-Ny(o}=%&%)|k7uI2~ZzhhTX!K><6+iq6Ai=uc}1N>emKZa4x4KspmKcSYY zW9ADIx>fc+^58NVdj)c^)i19g6X_B4gU5v-J43v4TJpl0A){HA@`z|aAjz}$3f&zI zIal7tVh;Xnh88$3@~q5{zBdzI2U`Cg-6&bW4b-ZA%2J2bgZWf%c%n`+z9-)=>`WaG&+Z%%y^y!URy-jGjw z2>CkF?{YuswAR7ZbzW2jj*ydk8wdy}xs22Yx%9C5b&2zi1TVd+8%zOn{8tgGKI>2U z&Ny?=U#kM@%FJAO?Oc1v88&uu!VqudtuYHLyXEggK1YQC#M=<-u(|YNxITQm5o`Tw zRq_G+QS76m;9oQx$tUV}FJsyQ2?>r;RUU_>nC+Gk;l?%U&bB87wII0N>Vb^i!dBwl zg+S{#fX-Yn#yJaJ&R+v<;Rk@z={|-aW^~c&>=8uZL2@_urK!5^ZxZ;bT04U*+ZyOj5Jp>rqC5i#UE3SHGQ5v4S_bJw)U{0|Z7d zPc`bu=rIX5dHW1i)mLoG+%_|>u&>hw0KHKNkiNlW5*`;50OFe@ziJjtl2FmtpvuPx zq50G$OhB>J!4*tys0p_wWH;Cy)Nt`k$fRiKtvapX_3aczj1+(o4HKvNty)ccFmItG zvL2zYVhl|^3_*Pre9_z2C(m_&I=XgTo|K}cz>b8!3e#|xmbSpfBw$qXG8s}CsEY#V zBrldG4Zjjss?sGR#P%C2yZ{dHwNS)H4zKoZ{kR>}X6Z$}C+V&BP4fbKJSh3vdK{t z??>;b!9;8(J4mea$(eOk132csp36dygk-Q`OyEzHlWCFsk%C;$N_X&ZHVoo!06`^p zrMz|SQl=cinf$sN$!=UC@Y;f3omnXtli0EF&^uhCy%_RI+I9t2ki%tno~K-%s6I-A7a|^mXD@acTR863oTJ#!222JNBE~y*M3J;;X-*0@@(P z0mIa=pqq2@g&D~mO&!a#k$&(VLOuken;1i<5xexzAsB~b4lq*?4nDcAIuS9nkcS%u zj7sLm)15dv72%JUsGnf$d??(V9DR!Pk8BG{`BV{uzBiz(Kpu@H1qxcmW8Lvyt zGtFqRo2-r$+ptCf3c^yuT)(Aluw4SHHqXdcPsBIAf4>S&w%w+n}zUE^) z8)hBW>O(W&>r*T*Hcum4I^e}~Sa3`!G6unG0xMLo-I*0mNztOQ z52um{6MJw?jR$w1G|5nba&s<#PXCSOc{0qAh!A6~&tr31f4@bw0On2|?#s(o{|4Q= zxPRdxff7{ z++hPAB`tlQ75k2UqBTrv>gsU7W1Sj*sE2e`U9Y{a-Nybt9jDc3QfMSzsa_t{2*;C^ zesmi7YmAKSaVPoBJ&}*S;Tpy?+M_Nj{os!RWzS~GhaY)A0!ItKw!OmZH695d6XpB_ zjpxYkwU)&IQb?s9l^R5ckXvZ0LHn`qt?#fIG<4w9(G?x)D<(u~34voUy6+(9 zTJ08sHU`g}ir<`US)zlUP)O~CH{!7Hyg{C1c+)JAaiGI(R-v7~K=+a9?KVsjKvipU zA#7WbQQ+OVSb&bBJ(5Jk4qnSp9kSXl*!{Uzi^`X^uva&a7WsgZ)ZQPSx?LTG_VZfC z+&MoSNl4t_n(psFD9d&!6Z9I<64;F4>f4J^Uh#lEKvOJ z=Mvs4{W}swm^IF19SvPsT%fKlHc+TnCU4uQfb`{Z{6+~;rn!c1Syb2npQ}WviZcB! z1&EKeQ;i7hkvni+K*z&`E}zZCGsDws7N4M@^Ce2avUnn@@xnjr6qMDhx)~FFHP*_i z_+>u!Lc8^D)Hc?twmfE&b|ACK051n)j(F7WApOU^xga&EZ7u_ggUAMSARNB{m02eM zJQTDE+ZD)pvz!4=kQ7VHjM7|h+Yl&dDMTW4v}HRWTwGR1u~$ooiRhf-e5x(pWe(TP&o{eykdW2$XDy!lbOHE~&5`9S6{${&OVA{O)>o{3R!^`8`Y{Cy(BAvfb2@q_az=YCro3{$rMUS# z11nZgl1IcpW4QDvdVQ6uBnrL7&6M3Ip-?sJv;POkO5!Txt-IPcWF*zhE)vJat}Zf= zacyP;mzt4Jz0ulh6PkQ79J40C3T(I;(%opP6Psa<4ttrYZuti)(-~ONP>z=YhE)uFQ6KsXAWl%IdP_6Mpof{ zAMXojxZX)&>Pn>IB`?sq^s5o#P4Ej7kJAjLA2Jf zS-v$E%j1S=+;B7q`-RN@Y02TgBEvK->Jnhh-W0bI2&1%x0Sfrt>vlF4JN#QlumWI>X$2=5*No9A+AV?u*7ThN{o_b_BG zN0HR+No(MgXxs6tK|hY1mVhd-&$hqUJgMItX_H=vTxqB8fgtZ4gU4aI7#BPrEXwf* z^1dBe4vmhwK?+Rh)0w|E$-UrGz8-DLlBE8i)EgK_-xbg+@}#HfKq49-YcfvR%b3t(gD*m4&D|T zqQcbqiZP<|s`CN+Xcub?iZRDXm_Nz*gHrFt#w}#s&?Y~>gZN3B@Jap7qLap;w}-3K zpEXMOyTskfbY15>zH;_vtogWtJG0wJ?bkv{$wX%SdmH-cVJA`zb7y3pO6ycvFY_w9 z(Skk#unbn{E3-;)-ea^!nz5#o+<7g9czPP$-VMIZwZ=wdlbW%LaLm1-g#u4KN#~kL zm;EMo|ndY@7CML51Kwfdf3P<4}Qs1LUciu`zq~xN4YVAK=ofr~- znPKxBx_yG39HSB z*ys@8Z-n|%xxOODarAP}q)EC)lp0P)eea&4U8MeX*R`>d{`f0tm2FFtA9r#6( z#Ma7H{r9z}q^6%1Weq}}qXYQr`C{9n`MD*-X%*1T)d~qo2MHZqHx?h4*}z1$4~l=v zNc+*j+#N7J=cTzt(@Q;TqTP==Y$(tM?FR2HV*Ihw=j|ZG<730~x-7u|q^d=!DjYtV z=`jw_5tdU>NT4GNlpyokc;|Px4-KPex0E{AaWb(1@ixvaj}lO5pdoJ>({cF|Bn%cx zL4w}1m(^1)-Ah)(J!RQ(`)`MooQ2I)a@Z6f5+dBv>gy=5t*mo0J%4rz-I+HE-KrO@ z1ZI$5*P(6!;E=QMxpnSA|J3ytsNN$7Wi3@gfJj15uugD3KGS)!=frTF%cIkivj9hP z!NSefE;GamBv6QbjUJ*3(%YAXH6Ok{+w&_m^*H`V%cW@} zI#WNnB7V0BFt~8YUG!2=k`Rh&d1#~3-+T;WhG*3OdPHNxVU5FbGpx04VlH|y^3o&W z#7Z$^*Q(Om#L~t!f1K-SPBccTgi237IeK7lPFlemF${e*<4Ve8cOe zoD^>*KHm)2uz)kiAhs>dADabMJZMO-azch*?h{UdqinbDX?c`EN^SYDeUR_ZpG>z3 zd|+Bdyv%yGv0(XbLqz7jO~s`)cIJ<#F?%mJQlw#jXSc9WSFk<2i5&P-)QM5YO7b17 z_HS~1U3M?iCtJzT3k!+iwK#8G9=+~sxp)ptW^S6LdtW!5f8lx{wz%n@4 z)0;H&zWObl=)y=zN+-&((@xK}YPlmf6GEAT+@t60j^W9~6I z47}ee;sJhHMy46SBJH()~5-fJYH3D83`tfm_ zq%`6xA3i)o zVzH!- z!gpo{2Co0T6N5W$ipYSpuCNcEr%ZI(h4JFp?7=4daxRIG03z@0;BpVFtOEWbxkSD3 z@q6d;@*Vr3Vfd^r>DX42#ZVNHW*K#XMLc@43=2(*`qC};=|6#&{=izJUmA}PNuQ4fJ7)cPc<)L@Ij z1U84=h8_geT@<5UMBUEDDn%#cL zrWgQ(c-o2`mPkVR?qN-a5bfdHi~+=Xp1=JrN4+B{m-)#TMz)6LH3`e3Xw)-^N+Ukq z=rP1sJF9IwzfV!$t;w6FiGN^_Gs2Soydt3Z^LL)!J&JkO6Grm2ZV}hFU%D6EvHC!U znY{=Mdf7X8=J|#OF~RvHJgQ6eihL!}j~k>l@m|^|RwkPMLapBm0m~X3MNYbAhg`37 ztcxQ)6q>hEOk${K&4&XywAF{{P3RX5txfCRr;PWOi~+5!RgFpj+)XtMrI)*9$AX3i z?ey)N8C70T>ua>LZWXiaO8>yyi|*^)Lgy*Abz`DM4{?g@bwQ*?7bB*q}V<^G*_sZ_{ zHQdg_K$36m;-IXW`H9tnsdvYr$})!s|f|&P)X*=n#cC z6lHghn6CVmd@=8Qs=T;Q9L(Ne!c(}~T1%^acwNd2DtdT`ges%BmzQY`%0_tCES@D5 z^B4StI!x)?zXnqFt~_}AA*pffl3UY7TS8e)K>It7*HCbDEylP||BiGc?-$BK)O&px zEVTHL^AaS35KoXj)Gl)Ef&w9O+`#4MZ$hW}4#5a~vz|!>S!Yn46MOElMEAcf@XQ;)I zcknI6bC{R-dFNsK*<8q)uGKY8&rKuaM_4pG8sY{D4t`khefe!Wnib!414`2|*~Lo8 zIa-&;^Pfi!+b1sW`d0gWhvw~UQBcDa1M-Kj!uM{T&E|;i+lN82Ba5A~+&mAE#)h%& z29G91vohXdo-=h_2s63=JniA7k;|~D<0hi* za=!Iy6X33PV2!tK!fFHPY|g7tr6t>Dss40fvaHI#+)hl$2AMozK3$}v>%=}c6y3GQ zxVTT00rkE4{r&24caogW>Aw@}`7~IEig6iArPm%J?n#Lz6ZGLm89KiEgy#cM7D>A$ z16-_>%{mYfjb2@&{G%2K{z#p&91MLlDWsr6<4A$Ee-9?}ptqb=Kut2bYjg{aRPM0H zrAC_5{81<_Vv?^R)gMxY-YW1uMEBcp#o~rso-v!^f1^G{C>Z3~phntNefmz17%HuT zgq)4_@oP9M8OLbY;+fscPadV)6uW$cW`wIAN$VH1$uu*AcUa%PoLOgs?=#OQ|GLkA zzma&6LkDV!uIMl3f5S!`+B*U+sF~%xQ|%}7B1ItY_Gk*nHMbheQD2K1#r213PBzaN z3Lf>;KZtno9`Fs%2w_d~Ps)<0y2`N#)K7@D2O$o`*vxx*H7kk)D~?>b_K}+i3E_pH zSeDF@scEcRI)ejQO7&a6z1hsfz2c)A)bN&zn-&uZiTl^!JS{v|{6|O+v4JfejB3C_ z<>ce#;Svz8a6g`x=Un40%JwNOt$l61y_(N(EV;kWmrulW?a||SNk~qnVQs%j_!ykU z_AU6w&mX0PGZo7X!)L3lG$JBK^vZ-pN0}T`u&V2Mu_D0xHQ)0t@s5Wr6z}UX-noF1 z5;-hDZb+7(M`;j(tAK&)Ew?0wLwP64ONDyYel+iz*jlVEckzYn1u6958D*lZyQ_Y| zfH6Eg9NwQ#7sc-lzY2hvRk6yH`*Ys;MetmV z+Y-S*x*d%>D3<%Ynmb|>Rmd-2Md<=JS0 zd>Px&m(;v+rKp5U8xUu4e{Eqm*+LyP0M^C$D{n{$raax)WZ~pB;fI5!r6CD?a`d4| zG`MyLTJw0^au+v<+;AZXAPQj40q*dw9-Id8mkB^koElQ~RR#Cm{PMk6`L;et73gIDTq?d3&tb>aL zp5i%WQ4scqh@i(EmE+wSIhlMR0O9wSviw5Y;SxF)=|ZhyB*MPGUE+0J^A-BLPsf$y zw`4+>z1S+t)J}!68FdX>u@(OW`zYCu05*QTHC2XJ-1082LNxyro12aZrhGEA>KPvvgT8Nq-(ZDaa#to7@>iq zUPggW*Et|hp=3zRn%EDcm$27+%H}J4o#B(E+N?(Ye*K4-wX3c30>5jkBe{;;DJ*{P zY`Bh zf0z7i+|~Emwz~#o48vb5eyg}xBp=fGanLcRko!Gm3)F}QQw!dFkwikiJv552Z7RTiU9@38h}ItjfH{xn3nB96fB2zG1$Tf=?a zJrm$T(hv7|y7WQmH0UWt*kS)h;C5ItSLhEu{qWvUGtFQnGc$s0C+SCul7vHQw&W!} z*Q<|gBPP>?)F#@n*}=t9s92dd`a!1jf4C&R6k>PP!ix^?`qrBNE0+e3{t!6LOWf-> z=eB}@TvBO91}V`M9{fH6=EDUAGjQ&KA~EdUNm;wn+@y-W7?o)cYPf9lFKc_<3BJ&?9cJs`eBu|&UQ^&&F_OKP1 zfZ)9t$=b1K)1H)s<9!n^ySrz+9u?$Q8h7N%HoV4aR15NW9#j>+x0L69^)s9_$Y|H& z5pe}zoofwMHoktYt@Ja|T|EuzvD)Trqf>q{ z`|9GuMn#F9J$#lQJNNu;f1a30XBEwIJ;Bz6j6_h(-;3Jx%(m-|8w7APU0uU)FkSR6 zEXbxX@!<`VbOV>B8w@`*X$mIR38KXz$-STD4 ztLPJztD1Ic3+$Hi)v(~$)qEJO_kN=G2?U}>jljoKJ$_BasVIdf z+fs43!hQ*@Et)a6kU|=y!+LI59LeFhX=g*2{-uo0nZAB$Ktrl zK&+c;FR5A#7FM{vh_~W=$BQbpnxwWa*ESN{?@lwu%5zTwYqx{f$=Dlfs3!SB(wd#C zXRF`+ov8WCfaB*{=MaTpoMaFmY#W7vFget1^xIi)cN#ti4$qwS5Z&8-oZVYMg@IgCIq)6*)QAx4wt*LRe+u;+ zHa~I1fGi;`D8nOK{bNWiqeYqTKqPzCu5V*~ld>F-_vRI5(Gpx3is99f?~Ao9c<0%- z#-m;osiQXytgbnoo|0bm=@=_|x@MTGTuBOee|JiJXZj~b{zxLtQu>SV$~7s;%(rhG zxQ_nV$=3X^f*V0@wHiWpKQt+4`_^vJ$bnQ)+hG zecTEW?|Dsem{b9l9p|Io;PQgp?7zLofab~46ore{RV2g3HUwLyw`{l-j&5>GvOG7iN*tZB+ zYZ!Odco(!vybvJ2h8d4Vz0e|f$y{cqYr(D)-4x%ylSi;0r|v$QUR4c03&WhgrI2^L z7{HpYoib_e+nlluEjS7nxriyhEHMnCq8zF^2*LR8T!*m~ zX``Hh+YKnOyF<0#^PO`$3^zA77eN&Ojwuoz-#7MS>FhkehiIv!E=`yWwzOdR6jhdu7XXwq)1Bk<&e@qrA5X^>bR8Ql#w39&rK6bcE%}I!Zv)$7!hnKgt5$0KYn=9+c*(am{4m8g&H&c1H$IR zccaXWCLOrmMKmlgo+g;F3z(|Fdm1J|pB-Kntb?i{XglLzEH5?XAD2T<} zHmv&0kd*=loIpXm})!pFq#v+wrg2OXy7!&}m5Yc^^9=bSc?QEk@$c_LMAA$ammZPl7&G zNExi#pXu_`8s1}Bn_h4c@kB`tT7YVB+yU?d(X|7CS}F0D{l#*9SOFaruhjCMbppKS z74W^v)46&5onNcqSBi8!dPG6q%{2(TKF@|X_ejl*O5FOocbU>q8Vp^xI7wG&E!QhM z6NrQ!ESCUuW%~$=3;!id!{ypVR*19fhkYeS`uR$;1`G7j;@1Gt64d&&?eXf}%aI|V zV~vUVnj`LsW98pErOO7tpcB|tppI8FlDFw-R^(m=JvKL@XnmtyqT|U)vHws9tqi#C zkFC;f&>(cLUox^7%b)yH@^BptU+Bhff1s{d*>=IqGJp_HlzZ+TfY_a)jlP+hd zklD~>)zG2F1(BXp)?31`pVWv`+$57iE|F+aE(d+gv%^pj44gr`#m`a0xcukV?4P^f z4@qM1;?o(Z8GD+dpVgcg9{=PjNH5aSu-{J{-z_X^ za^w%aH!cf`K4aI?dBS0f?E(GM+bZCxOU4))^0^>AZI2Zqpe9zqlAxoeLJ56guhjoEQ&)7RMZVc$e#x+I~oqA^wU2_^JHiI~Oe9{>fsnJpijN z5C6sR+q%z%)hQuB^7I07w)WJ%bGG>IRDz31TKr8o3}j`h;O4Cr^9K>ZPRL2a z>-5SZP}u9PxK(r#@us?G7=%cbxpW8iG+RIlOnwUs+`z5@XN;qS+SVeRt zNb`A#{OSe6Ts|}K_l09q0iuzI?fawVqhaHbhBFHjjSJ6ifQjK$<&$lNU27bAg}dLp zxpXRboeUC8#$^%MM=hyxjMI6&rM2dqt8or#?WOZo6nLez@Pc>p1$L3*%tm%;n5Rj} z%rP$u7A(z1&JM><>Z8rq3ztx)Rtl-kJ1zo=7yRi01IH8tT zAi%Ot>-f4P&3Cf6@lZ)5<=jt@lI`VP1LDN|t=a9;kCSY{`yH6fjZLRus!d~r;Obzk z{jLFcF(8mbVmcwwtyM7qTA8IcJ+&At9_)H9iT~*gv+rZ}O{ty-c?(**zpQIktq5Gz zACk7$<|7?mOCOL@mS=2gj`XtsDRXY)YE9T_XR)l>aagyTpI;q`u-Q*0V-Kve94FLr z(bTz|Kn9wEi!RTNj@cCq0)S#so&J;c^43MC9@M%MFp=?w!B?`0F(9Vr-Fort!h2eo zZ!b+-B%wZop}H5KOVKUwHEi4tCDAut5^K4dxOpUnPvK^ zusJ=Th2`594Jd_n#7o&MIS2k6e-;(ue z!_~P?Mds~qqrX}S65#YYKjHM#Aodq9&}0<;VTyk&iZwZ>DmoGeocYn`VNO0~7#lGc z<+N#vK(B1y=^%+aJ9|Tcou#RBWyIY%w{dY!tjHKac&+3hU;U~bJa-D>(hL$oN1nQ& z+hl`94|Tut6=(+WE+pZz-iv&J*pXg0yMWRPo#@Z{+-@QSJsEitdzj`a&Bol`~dOQBM> zmlOYiPnKZZ@r|9L6ko!v**n|Hf0Aq@!l2%N?yRWMur=__MqMCY;B5-#sLhIx0v+{( z=4?z}Zkf+x-~LF5BTqv80{2wf~#GO2FouH}AU!$o2C+KH;+}Cao zq1c^)R_FA3al-D;^Phg1sWyb-`Z*wvHjBR2dzDnju<|+R->=F%+KqihZDH5#3YeGh zA-KksidNN5*XA}uvBkZWckoq~C!=N_I(JK)Q58nsC$=R-H5uy)&{d-)eHg&E~#FCIyz|~*3E$0_3P{D&s4+o)rWn|S~0eRd^FLo=AXir>-z>+@m|ri zI-ZXd`L;8C+e>UZszN!(E#3>q`Z|miYR0zWb|}L0T60>%*eVmZ3`Ln+HQwV$C$_mb zH1M{!5!p9+M(vXJq*g{mk#`%8jcLQz$=SZdNeht6g&DMgPZ59w_UO8k$bn3#T5wI? zk~ijs9Q0D7Ebx6*KpReV{$R~8Fq%o5*bUoGv~n|C3S6qD(`j&6rd*Cktgma+bU{&` zhDAgU4xt}J&2HO5cA0^?m{kihCEVw(qzRG&ut^@GDVNufem2nLio{+uoXPlc9+o1sGOn!Fmd=NU>&$i_!dmf0<(Rxkg*U7 zxvI+Qvm!^>EtBNkBqMvF)cJq=iTA_3bvPI7t(%mlnShV4o7c7)7jrd*f$cAznhmSc z-7l|MPxmM5bJLZlcYf3#YstiY4RF&wrq7y~Kh*cCF9z8v?+*K!zpH34LOnB8LgcZ# zg$GO?9JW?~Qmb1T(c}2DYLY!=9sw%ggYfdr?{_1N>?5WUn-UJs zGXr@Q745d$q8w+^#U{~vj&5(pgL0qHPS`MwT%7GSR^!xQZpRV|75jDMS$HvbWB*=BI=P8^R#M$x~x0`>Iv9e$@x2KtG5F^?I*o`7xU0zvsh6oeGi9J(q&)p z{UJQjI0FCu=&sA=BjdY`&Z8PVf>K@qKy>!;0+6I<|6cMip+~uB-d^v{jNiI8*8a#G zfZ)268`$TM5giU-6;<-SxW$QUQpFFpFGIP;oP;nt%bBL6rGZ@(QQco+j9qONA@ zV8k8q;RfYqCQ<5Ea%yQO?>soLMW#JmU0gKb@{cQ9RUA4V_u|hg^>d7%{>)jwjB`)X z@25kbv5s?^zWi2nnJsyc?Wb}*M8?Vvz^REXL!)*h?L=t?ts`E|LEP2~=26*~XAEEA)=OCOteVHj7wDh_x=_86}s2U9~ zd5Ds7p%Imx$Ze49ulXMh;VRrNu9iCLV+jjF5M0PIUQ^$@UbmN+j6YI#ZOT<8;avw3 zJu9C4Q=mWH3H^c8{Cno9Ji$6D3JD*IQV^(TZ!80s+@C8}0XP(u&Ux1=j|3Kd9_+c= z7k%-Tl8^8q?A`m*q@L+qodJ=mxKkT_AV0HM7JZWq^6(toQ)%7ck>~Sx#3UHJymzA~ zb`xv1%r9u4s)^MBZE`tTwo2kPEqkVAf5Y&Cg62!1e-_2jD{#Ei(@_vc3(1(kz(HqDEE4uWp_ zR7V96*WaTpTwtU$I#E*>P@z7-3h& zvPiMuZ!4}inf!iP9y$Y@^=X=L z;yUgpjku=Tm0%OhG>GHACpwY{^qI3=nPuS%dScMuFsS3t=zz||C3#;-9RHV?m(twM zywmoaH`@cB7w)T0KV@db-h4SH)BeBrg3~~vt{1u1t^P+MJU%66wJ-yWK8szLvsH8h zC0e5h5Nf`o6aB1H+=mE>grw}*XLhKqj%;^guI89H4j#edw!ukIlb3S4dyAr$yEdpb z>|Vn@sXWQ2@|9VZhasYFIIJN3M0}riczE>heAkFlqQh?uOs!qB&Ks2gS?NeZAmM4D z!1G*LCy(Eisy{$BaqSNB+odl)CphVNO~&NtoAYV3N=yY}3ac>-LkWY+8~i=v*}Yt> z4C!s_S&O1}X0rrsek>VpI#z6%P8FGlCp}?0W)8)6e3)tR)wN7rN*TeV%{W!}N-h*$k_d*N;j!;f|Oro;|9@N(oU)z6;5hQV(awKWUZPg1QhLqLrB zpM^vcQ(T5IMfB|$dQizp9`Vggpuu(cj^hiN@rj!Mo4k{|r)(?NXVR98*sxJ9I1P+~ z5!;(x3VoAZL`Gb>*WKvOq_WDk@4B#M`3B{Nt=dL@htkcjEyosXZQgl4cqJtzSY&uC ztZfg$2Jg5yI;Z`72s`r3jH)QwR5)4lb&TL{L15btsc8H8v6h@~-@dtwaW4##dahBv z6DL~p^YahpsmNbL@r~4wv(K8LWUB%IQwzg0?Z=BkM5tSFJ|bh1OO(gH%?E{-vv6a1 z-Wu7!>kC%uPo*63>8^bYC5^0IOVILlw3A zP-wzCtIWQZXM|Ed(c;Lvad4bhXXssUt8>~r_7(WL=?v!bQP8(fzaS9eO))v>o9MWk zP^qd_-j=<rK7_q#LCETTxN1;kJ)HzC^dN4uQA~TPX9na zq)OKWV$l$a2Cix_bt0?!27ULuN;um1!sz(Og*220|ySi(-karCNBBHz?(xz72&zFFn z`-I5#pD5Z!6pDDPUt}$uPT(u6&t9^gcP~4Oul1c) z;C8=QLEket>U~qorBn6_6jO(_bPTyCf1dpl$*ye>Z8`m}%^_oD{BiE<35xKtYh@sU z$+P&ORLZo3)8eC*h8VZfSEm#u{__Qir9tE(14?)PWt=o#Es)FNAw7;st!PI+i-@sb z&mY*ex18A!jvi5{#1v?@0Dm}tri$lg`7}E3dKHa#Ok~YLpn|aMzk$id#D;`-09WNd zmpCy3IqyiVRDtmnPRLnhCG2b|2}Ba?twK1vymY=ABtMe?eed0nF&lc9uLi-QOh&dl zZ_6xIn}sDMrw0AB)Ls_K{f9$N&N>Uq5q0cv;P`yD9Ae_qj$l;TfJS0q6-4HC7xs}4 zVQy57B-DV3vv2|v&Iu}^H+*&Faw!>;a&T}EGT-U5maG+9Nu)b~IA2icZ(~#9 zWbCsuGd(>kX35uGe1eT4{wAk@Thq&lQeJCTr9cZ)$MDocyX$sQvh}4`x!3*2#`#V&L2^_CZ#BcLe3C zjoz-&^8Q{uqg5Gl44f4>-*7UwR8Cn;7sbvo*>2P&v>St~oxP5|#Lc)W0 zarI+O;Fb6Pwo3Y+PpdP7c{ly?A^-Rs|Jw!pugCL0D$f75IAP`Q%V)r#5mBDm|F7*o z-!K!HdvSdN#s5Pax*iHa*GpVGm(0Nb_ntvhbugTTmMSiqxrAe81onh@1d1pGr5t89 zhI#h?hYsYK!}e^8DE#F5&Hlf%rT_KnfBzg?3LYB>s{3`pdij5;oBwyifBo;G*Xx2g p%QKhDBm95A@&EiWGn=3{3_$Ue1tIqE5zPA~FRda~DPa=)e*tFbZ-oE= literal 0 HcmV?d00001 diff --git a/docs/img/edge-cut.png b/docs/img/edge-cut.png new file mode 100644 index 0000000000000000000000000000000000000000..698f4ff181e43d2b11aa649d9e0f5cc99494a747 GIT binary patch literal 12563 zcmXw|IT1P?v-bGLJlcAuXS*#@_ROBQi$W&Y$ zEv@Y=prB}R&5VuNo!p#VEU=A@N2eI*P+UCKA|qqfjC*=}`+BFyCXFYJe`M(E@8DzZ z3_!toKY3S>5#suF4EG{ow^gCCqk`kf5*XU@%i+pTF))cria*kDu(A?Dt*#zW!HYT% z^NBXgs&Qv@K@T#cAiE$$lS0=&&5@roka=K-{(~E2hVg+ZKSY)F=g*iLgf^uSTVW^j zkh6gwHAFai4IVF?{2Pl6PYDYP3qO+bTC5SaGyXeiVZ#)Dx^w@vF!;4_FlhmluN(Oy z+KvxLjv_{0YGP$=U4Z7{;YT&1e<2hATBQS%=PO>_#P*A8* z|Gl7j#wMzvpje{iB*iqmp#KOVdx3`LQsMfsJ|jQSJ&>1e(8It`l+(h{=h8+BV|^<0 zA}^0bjDF4yzLXh4mHdDUM@r^Gib6|9MpmBa6eij#!R&Hpw8(qwLDP}Fy2;o6wXBCF z^GD`I%_*ejf%zb^=Df6g|YK`n8g||B}hDv~Le%}mZPg+6S-yTj8Yd&0T`uA%b?gzd-l}$ffNI260=ik~{ z2iNfL9U+U$_7KOiTDR%(R=rH=$?sAal>XmP6GRHE>kVHqA|(11eYd)? zbsAO2$HsO8<#YS{#q)30TO1=cHuMkbd~qXTy7}lagPG|M)ySx*!eGm_e&!m9T1=#Y zw+%YIux@T{cAAzALx7lI2!M;5Ot{A|8ek;9Q`pC-?99Zm-{|5V^K~!^ht+tR{HkvoigRTaDxtyKGMKfXqkB8AXGeNr>5k?f^-890{wuegPCGE9IHhL z?q_Oh8~F*(%JOaVp+gor8$o}x!otEGAg<7GbfS-AN|2Y+#Y(<^(Vb|wePhWq;BI0e zpOjlF0fFrGUpPjm3l&8-QaT_soMOu84w!hm*FP=WUS$T|ethREbrN`H6-6!9^9Vi< z7w8Y0o_X6$)CQ@m8FD(JMb+$qQO$EH3q65>WHCHOa;mq}c_No*?laQ?|L(LhgnYR% zjgE^dX=at6NK$|s1LH_xZ#eD_4OH?#Ozk_r$NA`$D8fKB zWyL6;!v^n`hoJdyyi1x~FYywLwW^n?2!=Dk?L=~k*F(AscpK1p3`@5$lU}oiN%%nC zuO9*&f^=HM+9Ta%Mc=T}HLG=pYFdAzNbn~%-xi$ynz7U)%u>o^51WU^n|}Qrn}sqR z0%0w}DC&`P#}wM~J|YR!fe|)pccU8^U@EG=W)#60g~@|xj>xe|q2Ki-4u?CE7x zdf%fTp1S?%Sd`MNIUO(J1K+19qOzF)Z_U=5?W-I%+v(B5JS+{7M_>^@9E&MG3u2M) zoGn$G9ZaORG~&>afbeodwa`;Bwto-B#JC;nId*|dh}Menyp=ILE;h#GQ|X6p{a^mH z2;V;XM8#;#F@nm=ghKaYIy!z26`@9>A&!U#W!+gi zj&LaKe9QOpt{*EQf}L=+LXk>7Wk*Z5=juyJndMa0>!y9n=HV0l&gEs#yJ7V_-c*&B zv@MDfo{J1BuY)(6X(8^95?=h)ufJsRCBJwXOX8aTz35jh-h|@R-<`I;sBXlF(|@FY z0G2L(buR-YD-&}rpR2UiZNS^pS7}nA5+#A3oJo7!`OKXbll$7xBAyq-1F#@*H;NM2 zq98Ph0Rr+D|yA&8&6MAON%e;`*cE-bo)aTlkhGiWK5Q0C&B70D!6HXcacdaQjc z;`3znedpP9OV-VUkqJG!+8wB-Zlk&)rrX)}fB2F3L-u6!xN`>k5CJw6?MuoN_ij|c zN~g5$8Vsr@iOH`VKJiqt4dU>vs712${+~!Sy7r`4Hovt3n+4%4Wsq`ivB0Q5$e15 z&9Yvc-0Shm1ebKf{VLi#IL(^XX-D+VpH_rce!R|XM4Zk>hv4Q*kyl0&TKVcX#^*$B znDFjyuW`9!RhXx-YYAoAv>8bj%=&jtr`yhWVm>uT+<@C%I!+e%?S_Ea0MFBfF-HM& zzuj`IoL3LuT)+Nbly+-fzC3oT^-}n+f-+9=?8Rk4kJ zSemv7kCbnFwC1WjtO%1;It{TEqu+FwY7EZ6s88Y}Qfxi(_+9LHu9Rv5F=DAbY1O3g zHpxZ(h{?W2I?^bnC=OgfNZrHM3x&0PyhFSGVZYqxX{B@Z`W4HUVA?#zDKnfZ>#GFA zYzm)RagxlWov=Zfq1pTxLvWOrpFK*8i$Av>r0HqwQtOXe7h?4*x)=Ot*XUQ|O`gZN zIo}K2$A_l5BxR=-xGq(_8zypubBWl`tm(SfonEgE*t4#y9bX4)|NKhYaW#*+%m$+* zykP#tV^=X(ZA=34lMd#Ihl74~vD@i79oNymgM@OYzWuZb)-emOXpHgoaW7%*4azFZ zrkq1gU`?x7m0U0pXr>M{+yAFAY>v1E%I}?@Ieb3*sU`Py|5!0~ZC1F6&E)mJR>?rC zpw3_$BBr^nMX$iCK15dU8}n4SecZIeU3rR@8ENXu^{=3d`rp;|J!Wvk7!p?N)I&m2 zca8J~>Jp+8oVdiFlHuIO_M4I%{~|e9yexzhTuCgw72CI)!R6#^Ph?c}XKS9(VI%uU z9hg&O2T>`cT^WDK)wGjN*x@WoXVV$NYp~wKhn8s(vz#AKd9LowhHG`)?*8^k&6{XO zgapq4zWFFPX_m4TTWzocO+YVFWN?Ip2OfV zgwn&oTSmLnBJVk*_^Mk*o}2DhfR?0fG1QuVYMDK4gBh*zpuZ#XPX!jWU5~m3mJMP> zNGgMtCRoX?wYG;&UN&Depx`DJ2!QcK><$uO+hfa_J=rW zyBNlzM#alTxl=y2AyrD=ijd6Bw*e^ z{8`iM;$cKMo7yi)GyU|u)#aP2RjNPMK_7O3dtKyvJH?YdXW7Bvo}$cs&TB)leY~$1 z!`LP%x1SxWkH7oYazcAR{SG~XppVSRgII{eAaSRxDQ<`3SVYL~;td=fbhz7?6^~`| z?Qp`9+NJngL4s)UyzU^z%}y|Cqb2udZu_6N{tV>2;A8LbJu#GZZl6S~s73M^CFOu< zl2hF{gIC*xO!E)z4uXBdw7SthUR8|Mt*qVru-u5u>2_psH?2f z{nD=cc5wcxQ!Vno?|j!HD^jZT!$!d9c(2E4SN!{^DKB$8o>Sq<2x*|*x~vi<&oewv zj&nEtr1m|of{jiv$$3hz%e~!gs#N(`_0lhO_MgiVM*%g77yZeS3#OS%l^Pin>3MQGJOnQ+s$>?zW82D;VtQRwWwvpmh+%Dj;D`mf`2R~ z>4G1xK8VDt`eA^OZMr;jTyQd$!Cg1_t5kcf*Zq5Fhn7%Uw#y7kZFpb);>+PwwjRZP z<7L9zE&71L-)LfsvG8C*RxYabHVG5LCYI)>Rjvl_Z=6!>G;qasrgv-u)YZPwp8=!>*97Mf5DyBj9HMj6Gt z0@};K;jbSP^$iW)cMIP~_pwCkUol7!uOwKzjU#TWFyl06am_7j8|yG?I~OQ-HNNWU zxG7Jb0sx`UZ{X1MRa<|akB-gquIWa${yo8Q)) zHT}tKCO@@;0WXAc(cqWj z=w&%?fie%VL&L%=QPKvx4btX*?h1BG7P*ryKW%$?=)I2cfu86-TF4W)D(#dpDnij zKn`tMST7^o$HmPhP&C2Q0s$%C6}{J5$*WjJwy_3X!LDbk$B^Sge}OKW57J;3UQYxy zM9;8JSp4swOW_tW?r>6U7H2DkR~_^hH$KI;(YH&vEmuPD4$TLmMxG6^9U_C}e+JNZ zIpIVZEN&FWm%^KM&0(%$l6voB^&qgC2gnaIm=B5v9tppvQ$`*@4)5z`!Lv<9g2|U- zgY{A|ABEA4mAcqrm?!^<#EZR#D2s)aZL<>)-Bx;nU2m<4k%B7=5ewsyFGn5Lwjn+z z++ODO=8S`zUU;8{xmMwhm$XhGhfDJtsD=LH66O;38jB3+^aY<7rdFH6aQO$Hx+}BR zw?_2z?Y-?Gl}T@|&sT54F}gZClFLFHz@1=JQrUY5-t z%ei|k*q}W3OM8i?MQkY~Z7V&%uC^zCFtQkS-e}EkT>rvAEB1Kf+3Yyq_~~nrI)ipn zAGVG@JSrmC``qn+MBdA(a5@$ggfDPPz5_Xd-Oj9kk>_SzVYFV|$-F)Y@I_ta6*s)L zoR3fg^H%!d0I^CS_=P++Gl6CDT@v?V;=&8=*`HMz9xt}KT~{`r6j;fd%zTyxVr1-R z(nGjTo5k$RW*2!KUKS;~9>LMm%3+3GKF~FgN+@dOT*eZ}g#s5%+@RvUYA<-!zR-U- ziBF~7UyD|?iF!m-2h1}E8=I;)4>{yKqJFoj!9iqbQvsz31ml%NSt~2`Jbm7`+a5h* z@x0xHIL(KMBCwW{n;Q@5E^ge-n36dURE)n>KaTd2`R~Rma3A6eFxpP=_r%?Ee(Rf6 z=K6N6C;APqMl1&E%T)+)`66!J7j(EUWwDD@sxGQfz1Yv`FFMJ`6ZDK$HQcqhmRFJO z#t$LK)Ht685_-q#LfV{UQoGRTmEW*mk$DP!D}eKOoeRgb!wtJCvm%rP6y%=?RyRiw zRChKeJs0=#PEdvC4Jy$f?Uh+)mSy$?z_lKnw%Xc-O`Pk&t<7Pioj;&1zwg7|oRQR8 zG+5NgGXkD+ zPnOS!4X-%%_V#w#>4jctaU=z(U(SeQsn`0!c

W)7fI>$R3Z@Ps7dUpBZ78%JG3n z?A`xBh2w)_)tB$V*r{>WjBE0{(MY6ow>Rlc6jOe zUmUve!*-xZHb9Xk#>Y_*JDn_CF-TR+Jo7T23wIxxjJh{sYW7AFSYRlbnBo+F@TYFN zOfWGD3vV&>h;(aIX@5s2=1uhfJ1cX3zSgRuW|t>>0^#4aNFzNXrFWj=O1irs9XRF&;=O*ihtgM8w7IQj$nMX6 z_CU&O^xKklk$x@-nDoQl{dgG=fxo}locQE-C8k;Wg&@@$l*}K z-M@$JqthlD^td-g%I8QdN;gI47+fc{MM6%i{D8hV=_M_HiEy;pFu&{lW@;HRON0si z9?*0uNEZh4hj=YL7@|Fn^Bc$EyUv_2LC_2tuW>Jwg7;j#g+h3YJM?8t>VR11%&Tb5cupf^(Rrl*e z93qzD`xJUnS%sUhUe+x+J%DVil!FNbMfc!w2ovD9@J%K9j9x-7(sG7#gFi3u&D-En zfDMNrKU6`4WBlQAdpl;s5Jni_u;kyV#b-=|tzPz@zADuhAI_#P+>&)Rc{%525Y-E@ z7E$7>esyzeRBo!xpq6{*=i}4KpsQjL4gjP-eH<8C1Q3-9va*=X9I6rkhbpCD?oa4! z)bM!THq#k5(5ib?r$LjU6A{VdvR(ROmFsq~v3lhZV@R%p_KeM_n!5-AD;Tt>!4ty{ zBFAxJCB!mQSZ7-XqpX@$OI0Qcc8gGM^bB=~}!%V;mHKs3pw%sY90H&Gy2oa?K{3P`}p zsx6ADzmg5f#L+e&>)D7?0bD0Pfb)9-m?mppbjiv@rBm;$i)f0;x2wmCj!XQ9^3UP@Y+#y;2z1 zpOtcn_-<#uDn*MWyZ*d={ld#UWqOv(!F zq5MJ>BGnluJ26Jb`bV1(^C3NDUkiIZP}}SG>$6ZqQ>ZDeyDRB-K~8vb`jVthcLNkg*(UW7ygd|Tq zeVmU33ncD3JTEP^Wurw@{lDQ7o{ML6p0D77saLv1$(<>O!q}Oc<;ZFFGlGNJuds3s zKjOz|Uq%kBWRR^e4FyTBUj2BYj6|>r90QiZjk; z!HarPZjrUQem7&73c7mdxxN?BvEm^E_uI(R?I`aS6i4rx@Q>=pY8$D3^;9i%0|q=l zo~9>;EHF-m;0yA-HdnYp{{b27SCAKr+T2hK9>J3UP9_b=|1Tg8ju=(pb1p7AZH`CHHitgfzBnGA%B-xT}5+{YH~N}7r@$dh{qucsRB#&p8v;NJlZ z06D^|%jd2$@nf9?3CY5D)W=zYHKO=WPrsRU>WfwU$S<3}@5n8Za#%y7J`4j7^om1Z z!?_V}i~KgfK3gP%8p0bUE!CoCC>!IIT<6!#AN|$-hPoc>F#y1q{a2xfc}9W%ny{vl z+COBmuaTIpyD~vx(BT%8DMQmVDTF@pGTrU<#Zbt`JmzCsdpr&g#|C2^p1-bwKqCpH zETM@#0AMa{R34U>iW27Vdvwa%zopwrN!CRD19H(7GORTYQZAPA&j{q7a$6XVC$X(e z?D==UUfQC{4F%#2g<Hum#i8jn}n-E9>RB>&2d zrZMRql4ka7A7eX>E9hByZa7C+kE&k{RH&CdgSn#ZqsR!Xm;?5kHT8QFHloA%yd0$| znkLU@#XG+J1td(?EJ$WijT~A5cp^KPk)mFlLu#c6hIoMVhJIL81+EKI&8Pi z?#`d}N94jp%a%_JshUbgL73-3s#ZIsU7VOii_qG73T{&;Y%m$vdN|gEkQWchqMZhn zFf}Q9PFraayy(%7J#B30^;?~wRScjxBcX26QhIbBZ(ZTM$LnKBzn_r=6MY|W}l$!0= zhrNSI&41zC1SL=k9m&NJa`7rhmMt52lH`9wELR<2AR#5^QHTWb@?EnDN0SEn8Aoz? z;Q?vnP<6gf`iIM3;KhNWJcAN+f&k?x0CU-$DUj%7 zCUmEhC1(x|Gib?0X}nSeA6?1CI3>4&c8>anq?8>dO|j`A{r=9E9;46J-lORKcOSs5 za^=GBudWlJ*@@JtZ+i(Mn%5&k++M(~zv%lAkq@d>aF6`ldh}J%0zicLOs%s|*T(bF zxD1^it*SJpnp%O&l15>=-9g_DPr9`s5)hIium%m&u=lpWvrPLEnIATH>N~lEktsOQImp7^ zkfb()Msyn__TsJ%Q_i$4rjog)dz?P;Z@Z3-fu?` zsD&7KuiY>_zufC3g9=3gx&Us-DZQJU!QgaJCB_D6nmRxJx^E`r_?#@mh&s`una7u+#C1_3s3T40u-NSZzHh=cxH$#xL zFhr{V=vBC(q!_THuK~0Q*Yn*M7F!9jLNIa(l zdcL3OLegMs-lIfCtrnXA{2Ef+cb=3 z>zi8j&qFKoY6H8E{H4p4<)`@P!2>Q@i;BP;Ljh)%(O|SOb(~k{f=IG>wl^9xF4^o|zciRm>Hl`02d77i_u9v+APP5u189=a)`D*&N zfSUnoXoWg8$n3&fTbkBQ~vpgN2NW`e0odb2Q5e9U%dJD zs%%e+WWYUM*A(a5J)|kxM}WMssk#NWm@DlyO)0&*Aw2Rtom&ztA`}!(y-qJSA}$+{ z?Q9`@Onkg1UT~tdRDc(T)7mP_XIf&z1 zUSiVx1LY;V8Wwsw(Vp=Ltn@z&eNL!Y^i}d#0XeV&&;jiR%c;(r=ijKa1R}+DlNoFq zJqBW8L0bOby>VrfgY-C79nUyYm9)eNy|Rmre^2ddrBzF zZ_L^3OgpX1z3Yg!Y*vu?c4(7TpfuS~}m2 zAhqPg#-r)%>r9A?vls|RHv^JWjk6LgDi)S$tM2RPn|M--i1x*u+v|{J4b!#=j0~8O zNqJT9*K)(kzw%$cn6t65o%*x>eMoQ7s8D;A2)NkbK^90d zfwri>9^>L+Z!%^-?nEv!_kg0A$ZSMI4c;V0kMmQ?S^`Wxji25A`VS)g2U}Kucy3wC z$*A#DP0e$#vwyi1HGfQ6V2nofJ{{=NDM;Xgp8t-7)tkxjRl6tPAL9XPb;1seK_vbJ zn2`eRb*0X%3dkSQJSe0_0Z-VpmAtVBKgSd@E!|zVNQHo@+H478){MXuJ2v=)mEeb?VD3B%GbAzA0z0ij^xBBRv6b9Sd4me{h-_vy9n;ELmB*Lkz`r>AXCv2whZ`yblD$V{bmleQN$7#1R zqcP~x{12%DKQ{xByDAV~Bkv9eNVv058d}GJIyMT4fJd=dNo!SF4~_ahkOx3#QV7uT z-Egj&B?wU$JE@P53p@ZSHft+FzNOeENQz03PHA@cuDu(uj)co!vGZm0%yU4++e6CgbmWL@v)TNeH_;Fw+1!C+xaG+zF6JHMe2_OlmJ#D|K;4Q?)QnurI$ZvuEgv0{AOAm<4z~ zEEF=|brl`%_&!{~3%Y-f<4?`WL32M_;vzB9Z;$BQ7a?oz!zI?dFzR5sLkKCN$nQ%y z`NIhVmHQcvGZu!^maC9|{mf~B_!V%E)&9uuBV)B)sxq#R$Vu<4rSJE@NFGac(PL<> z-DyZXLv7xf%;X4RGSv&%7!F-+I6!|br^+dUd+oUFo^Mt@_N}~Tnj9N0FEOG}#Lw|L zF&WkG8pVFr-u&f6L`Bpcw04G%`w!S@RDcyGYL9zCQW0W>fqs2`owl16$d0Ra`3LQZ zZcL?gLQ&;U2vuo+skJQND^v_KaXV;Cjna%zX?L?Os%-8Q9*Nx%d4Fv<7(~S*s|W!M zn#e6#`Qz#3`XmrVqYnoNf^gt6MkI(GZ6=SItobUoy@kVew*hyf|L-$==yyQd`EjfK zjqunps8^CGe;pse#MK*jyIGkPrX-|d*jj5AknSBL$cv525#Z+=8^_bRUb?3mLE@dt zH2-oeEi2vVzQ@R51myk`_`ttyyQyqr(Jh9$6Kldi%bOuK80Jb5JK(Fw=fH>hW@XLJ zCO@axZ|btd1j$uiXC+Nlf0A(>nT!C#1mJgAYF3PX0qCG?(ARR=aiEKr9y&{(g zm+ixgO~3`*Mw9so7n`2jOz=jdCS~fO*+>_FgNjh{C;`}yKL526BZ<5kd|V85k^JmZ zw%F4qWNElMUh$~IoStT0Yxts`%jPTk3qp!kR-UIkM z2UJh_uzH;-dAXv`BgdeNtX?u2sqTS#v2?K$j*%6#?Jye3Fqvi{9S)za+uVj;K`kYG z4xx;1=FiTX=s6|w8vJxL)~j61)|Ie9MV?JKvyu2F)K2AA!0UTZP~+E~@~>dQaMPVE zp>k22cVncMm%lbHmJwuf*Zp#V6U+kmL0vg#N$^%BC#nHg?tL#T#>!geC;z{y3jGMS z>&!PoYxJy~v~{U0%EC{@qo3K~cmQ9bYLs;B*F68Fv|o}Yv77tZFrWbvPeH~b)S9VU zz~K5KJ#L~*Dc*~ls|qF97}s1Zu^WsH?EG~wYh4KWP|SktjmHh4EMC8|tF`MWptyhe z;HttXRY^!7a07H+U0~k8b90EbJGH?ldaPPbUfvXVJaVP7aDo*S6gb+bOu?kiFg1yR zK>cX)ShiTm_ud@vUaE_>E@RTx0_Zbb06iRqU&>{()!7VbWz^`iDZIYPCXnsoS2q@s z_{#o3nIa@uW8sKZDbzSIM9+sBQYPhhvl>;WR~xbpNWhXu>y_T3C$=8@IKE_IX!r?( zveD zE75JRj2lk}G>tgmPZe>@zbaAs!B21G^Y<7U0BP>8I=(`Up78pJspZLhv*U6y^I^Y! f-;{XWzlv;UvSNd6E;s?JK9ro4l4Omzaq#~E&DwJe literal 0 HcmV?d00001 diff --git a/docs/img/graph_parallel.png b/docs/img/graph_parallel.png new file mode 100644 index 0000000000000000000000000000000000000000..330be5567cf97ff21b175e0853be670e6504d06c GIT binary patch literal 92288 zcmV)tK$pLXP)4Tx04Nn>U|=$Eba8TJ5@2A+%_}Jia(7aQh>TKTzreu3z|A1cV9a2hm|R@o z7!csY00epYd8r^86@0tKzz7n#tP3&=RgNH!E4iSk1gPo{5F4eW7AG?>Fd6{a+2th# zKyk)7K((2~%LAWc3jh!hb) z&=1t_!-imQSSk8}UH^&&1qB7AND+`wLklD%gp^G$+k1Wg-|x&l_uYHn+kLyK8+fyO z-#K^6nVB$-`$EFWayKTl`UC$#gdW1ch~|Nt`_N^GzSLZKw3c$!kn~IngeMLq&X1f0F`ln zf4}s$Y>}=luS@rqrP9|}D}C*C66ah(zW?@>D>6T_QJC(r%S;; zCrja^BP1(3C#|fNVd&MVnUmt37FK|FSG2ntBu~^K#v!=&l{qPfi^qrsO)(R-}S_D5YXc~0R z%R#3^(U5raV8TYlp9}Ns>6VcNYrJP-<|8u`c$N;&sXgs5pO&F+0f^p02A@py8a$Yh zovU^NJK~ub^2^AA9ijgY9C4UQrRY<+8=W*vb6{t2fOWmAWt+6Tbh|3%!YT8mVDeFt zm0zq;(4BH=ZGzz@tE`%=R$4_}2>il=^_Q3zHrCfAT^nAJ_BBtVX=syzS*J_!%#$T6 zH(&jz=k(9pk^^rG1#J~5Si-dH$9z$5XEPv%pgQpoqJqS*$fLgtu%}(0euW*ty$4EI zl_r!`T=7r|QCS(JlA@w>%8ZJQ?wpU(Fo0B)s_;}P!k)@c?WicJ9I3!ip{Xpz6sfbr zQWJ?~pn-rr)HrLqU*XvVun~y$vmwBvEd(JIJ^u;mMrYt^*i-1kkV-$S@Ufsc?C{Z$ zfgXf?H1NYBi0g+1?f7BAj}FE0F+#|9{ZR2fQ23F8ci~406GEQ&${UT5Dsb>GrvN;{ zdk!Dq*?53w<`hasZUG?X?g(>>@D#d5IOH9SHIDk?{Uoysx~K6p2Z9{XwgEEVyz~L- zc=ZnSmJfl#datDuI&2dP;!jv)L_88Lb+4iQizWFGIz&A_be(#dvStsj|d1l~a zc6(^2!|a_L*y|N^Q27#P53{!o^F8g`@H9w&2l09+;6~t03S(5ncxYJZGINWyp=X0{ ziXfAUK_4gvQx;GlnNSvz10_ObLd6;|_C@R~1lG7f1@`d)DlpS;G)PvJFq#8GHAc(A zQ^{JZpaI}ub}mHdH%dR`h!=}{p!D-Yz|s^A%lA~_H5(pJbVPy5TSpZf?8hhsM;B~F zf^Pl%7*anNq2TBQMk@@)dOP5^!zp}x7*Sbgs1j$HQi-En^GYx;SgJ4scoydf@t$2! z4n^-4^Z5DoG8KJKme;i1TayEu(V^z--a@5`V z8u)jx@ZI&Rq;1*#_`dCsqJz(uqDhD0JDsCiR`~~}n|68YaA2=c(5y6Oi1(ql+uO29 zdYkdIZkE2*Ez;k%6^k&osX}JY%oc29<+}ztOB-t7xqPQW#?}Y+%rdFeJnEEvPk~r` zA!u#``H#3qUXxVlh(bo?GbPG+6Dxd}VpJhU{U>B6Y7r)f7Kf4IAxvh>jgThlKjI#F z-DUYlw_FuTRa`bm|#}Xv^g%JR@ns{Lg$6X6+ZV8UQX5x5_KY&nlTs6{5fP1&SV6)HGpd17`Hw2hsMJIi zmahy4sIbxSsooRm!^RS=fx{ZroCs;`;~-uR!}Gs4nq#gC{$~5Tzv{xzr^|$c8~JU{Nw9VTHkm?I#xd|x%-?dr3aiX*|{9SOUquA zXK%IlJ*uFo2<+iJZ`YRPItSBJ|0Wdk8l4NyE*XP{ag1aYR|D2SDUViV%;9AdU_zRx zf=Q_`BKm84=zaaWH_NL_Uiab;SGYuaDcnKDRH7powNP9Chx7{zi{y)+{e(=LGHH+G z-%kvXRQylr4jSS7M+(Sybp@Wt-|z?4&M1HOw0m2(VcKLXMrpQ54@PTxn>Jvoq)D>M zpsYvjBe~W4N?y(W_*g7gcBC85pj3=}oXWXr$^Fu~~O-rPAD>nLLs5-lJEP5srC5H_k=6$G?gY&%hb?T8xqM%u+w9}*U z7;f^LD!c|7bYu~v^e*DYZ1_`lyoGL5IWXZFSKtb;O{NMNBBl7TLPs#`0_g^x1 z&VeERMst_IZ+KwS;YvM@TXT{sy;1t9{O_oHG`{G2kUrVRzqILNY0pp zBp+L4@<-1AuUro&-GD|pIWpea9b9Pwf81UN}e{NVJ3*%x5sBOu@j z@*k;>(OjqWZ?wJm?6q_KOVYCBUdgIHKuQn2NOFt9(=gF^((ijO2X?uF4l3oI=FQSk z`;2t0e?j^hR_icf?x^Y5=sykkBupLW_)3N>4K|2;Pbtv!6fjZ$0VhSH^rP;61*cwdqv|YnJ3rS|Eipj+5N7 zaXN2E`e|jIz~wz+!FH*)w^y29e^9!X-YpgLE|=`$(V?_hRUe@GrxK44Exts1%+Vfo zG(duXq?z)g9~KzdX?^{E$lDSrIPm>aywCC4n@+Fxj&gDDe&1yZnnJ{8V|)N8uGrI8<_L_C*EYUSE${ z));&3yHyTCnuH4Dto`4V{v#D6@|s9L>cNl6Ln7(D@z!4ml{C%V=~$qK)xK1y$;^zg z#q!<-ljN+!$4hCUcNAa}kL2A`%eKj#3)jk{uWXiH-OC*IIHo*b=FJ)42hcU;QNZt!GKW>~~Ay`8Pe^Bx&r)u1(J^-y`30(kSCbVW3fT=$CQpN`XH5w#q zxG7&tPe0BZXj}29bgWt^xwFoY(%EN8c1})Oz5->jx1u{$L32KbJ?7?@@06a^Pe}fR zLvYUQ(Yn%_3^IW-0gtqw3GPv*CVa6*dekZXM+83FVh@-84Y&M4zUL|EGY=mp*I#t7 z)NXB4*aHSy(IU>ae#rVz?XZnAz3x zzN03|G5d{{7gsgNhi-maeNjO#f`X2kg#2u0{(E|Roo1-tiH?lc9>uhRC7OQiLgA7f+xBq`qa zG_1D9s#X)qBs@tF=oAfT0tur{NE5A4&UV1xsCU-W;0JL+1&O3D*n>6&ZISi#_RGWQ z0pAIQ`^@rzd(~$iHeOy^+caoz`Q7s;3ZEMEoXsqHv))(GRh1R;%O8DL)6O&pAN3!y zb6?{0lZl6{Bjg_m5T{Xhj0mvtvVFky4UKZ+EkBn#|Hj_BF&(38`FZm3D=v{Qe&&Cq zpde3s@{S08>CKT;tKiS>DF1cqH_D?=ER=;$KQFJp@usX?y+%JW+S@xc-K8ZZGG)>P znK@&c%sXtZoN(-X*=PDRO(uE3d=k2$1lM%vzZNV3Nr zE~RtNm)rt(9ihdek)?ljItSQ+=xA$^=10GQMST0Ef#*ox*aMUeQ2~#LSG;xeTQnRB)!BuuEbVY} z$5QsTb>mvL91Qj17|Lv1+m@e;Gir0OaGmfzoTm;B}rcgj=GE{X1Y&Ryv^n#Qdd@M^Li-^T3prSMQc?9JW(D|4J#W-Y4Cg9+1;J?x3T4Jv~zY z)XyZlbDIurXJV_S$q-{OoG_W`MBDWHJMNa>{_!u{FIVqB2Xb|uCs)z-#-zjQd(=Ie zs)7CU^u*@Hcj-xF#YcQZ3TtNiAKFnW>GzS#fgM)RJrH{KoZA-OB;|)+qT8hW><%nH z&I{c8`*^Qv8xA9BK?CK^s?OvY%{jPa0jK!%!A~Z>eEYj`>?4kb%;@fwo~};Gttgfb zz|!$!;Lkg9Ga}E?20ozz4%2_MLH+?2h<)#eKh?Yprl5@>hUIq3ohDbs#d0O&>a(6)`8kY!uttYye2)%EC0;SKo1JT)mDc4C zO3u_1q-@Un^njCeZ#u%r5&XJC3YtB)j@Bk={?`|z1j9x7*nH$L@v(RJ?A6t-GCl+bPihm4dKUn)&G8VvrMfnmUl0h zsB_E7ddtbk1r;>iv$HrlKvp?k1tHzz@8m+*PuQJqLH}e@;=5W#e&)_OKyJAH%d+6e zL)B%l@}~!;!~8@@B@G|-Z(#|aUG$P%{)x{eEM`kacer-r$BvO7eeY{>=4mH0(N4s@ z6oxI#(eW&pzGQJ)e!831N#k?BMI(Bmlpp*aT|h>;OPd|%0OelWnubl%`tX;f?9dAW zi^77f72^t*A8np{;bq9xHM>@>-gy$Ef{8RRNz&u|*>5WU&YdUYHDjfBA1(kYKH_pI zsGgDj5HuE!aJbzHnnOn2=+!nn_Z!LS+$!aBFMv1=-r$lKtT1(N)=DlEGw*21F2mW! zg;36UxVV5@Q0NJ?W9V5^Q|OIMXsld&MM!%LO<~Bpzq3<%FsIbt-Yy+joR(Q#B^Bdw zRxNLlQj=KWJO>X9O~B(Va+sbDaSx*h9gcg)-hv8x+?Z-v^w_;1m?}TfI8*r#yTAPE z61n=?ugMb&pA8YJ9$h70yY^G^p$pz8S-7Q(a8hnV7~F_@?V z9*X=A6bCPMX?XTmk~MywR2+I-9)>?1+BwPaWZmjgqQ$-!7k$xLV z9578b>@=~!JL=a-|GKB7_?S?xG=OEcwz(j#%cn*_ip;z6zsZP2$KV5~351{N5 z8}x9BNG@I-;yCTpAa0)qEg}kH`xTKRzw>^XUOXw%8;x(M`QNs!PQLW58|0U_-L7R~ zbaJqj~szuPvhlr>AAm8H)=AY0Zhi#qH3kAC`VS-xVW{OQ-X z;)qqZ1|ptkdfEQ`XBAdR#gUgu!_z;LW?b-6cEFkGUQW_d7@8&94H^zl^Bn7p&Uze= zd44#s$(2F60;f{H>N8*319J6$kgGz}A-e)LnLN3Cg)?Gr=4O!g4(EtW0k-$$j-IK# z>G}u0Bqj4cik03|({t0tz7YtAt)MCRw&0>;$={@K(mcKK$K{Q}Kxq#Oh;w0>hUKeq z)^@!V7L`d(F>jX2!=f-;Cx}y+`Z2excL}3sEZw{}7(&XPv??7?r9#H=bq0oVdDWxj zcnvM=GtlT3l$1(ObF*}<+br#k%~CpT621_5POVeqE*0Tlr-+@Ja4v>>>DQhbaIeRv zIRhTA&v7hFd3V1ty_QacOoQQcPc%cxKb7(|U%ozJd)4WuoG9P>`gJmM`qV&aGoIw- z&)5^Xh_-)Yxgl z=g*a@7L*+!4eB;K5tm-|nLQ#`XTJZ# zkgMP5?gC9ulJxl>;p!psGX($q5G==$rd;XW{Kz+@d*3spnER&Ddr!|zNBB7cULz7a zv=r%_4tmeAt&64b_)i4{9N|?+!gT1r{`HlrJj-xQWEOkPSX|bJhe{aZi?~Gyinpto*==uIC{`wP;JC>cvAKI?ZW2z+C zN8j#{tN%c*F8s(R<&NLniX<_Oc-Wp=g;F`Aon{d~=H^*2j45e)FI<4v_;1}>C#@@1OHn@NnmNzX-KF!!v9>meq4Dc&!yUbtNAByugC4c6*v8=)9O~9< z0olXWci17U`ooZRe<$wz#eUWpU<~t2RQlYCm5&~E>#B9qhIo50&*%i%q}re+F!Ez= zgh4#wZG^ts&={e^sm}1Rqj?!heu92tQe0Goea&v~by;bN`~Zsig(v>$lrwo?Wlb2w z>82)3FP;wC^RcuM@}V(EJQAK;?_l@WzyCAo5v#@JqvZpi`kBl<{@m?W&WzQ}%tKC; zi>~>tj2^#_S^4nq|NNJH2dl-^?Bx9CRnL21#oX7{jO|r;nwljBKQW&74228{2|r$MS~%qGyTKn6HEH!@ zkytP3LBaRK4l93jflV2{1nXa0CXJixq!(Y(`ZdYah6FZJ2%Th~#QpFgtsYFf^AQZA zIKJ#RZQ3fAfAaHj5*}7oHBLTo_0OalL)$yklvLEnxu3jM#-djpHD9^@Mp^XYD~`{W zZ-elhkp6(3qZQ0^5EmTubBe^8YNf4qkv5vqxYO?gb3jM$m|ih`u0t-=%vNskKV9&Xc~ig2;!j(1YUxE4Cm-U(To#xi)_~$ z_eDGUijBAJIDgY355*J?BeI}cweuyU`~27M#5(uf0PuJ7AKiw zvBVR?SlS6`BKdI{wJ;(#VD<3#e)u!>T9}t5qe?M1zMD)}Prqz##Iyj6uHybTS8-d~ zy!@yFG&}Vg`7t-o|1k5PKt2V(*eFnGS>wQ=G=B8?Lu!%aB=|?_FTCP^1=P7Ki+VKG;CfIih@^gF8tSDoowyo{C8}81tU-Yma5}LIdf>c{)O8mw{?w_<2pu_Q-+GjSCG&}kgGq9D7iX77Cz|7RWPB$!P$&i zo*B+?!X$?0OtdX~P_F?l#C_96lMmPFm9*R?Vb*T43Ur53$H>N%2b0K#XZ4(I*F!_N zV1cudg6ZSx$;Kw>TDev#u)jHmIbnv!m^G9$AS#mQRsC=Ok4Yi+Q*(hCU;CXJD8Op( zjvGW6}!R zad^;QwlSR;Rg>DsrI;NEmO((KLbykXaurq1blS6W1zErAUh(D1OKT7?Ta+owp5ac; z0uJVZfrB{T111yeH%j(DLvk_2((=OplWxpS^D+@TiZkUPNcVbR>^K}x zv}_o+egs(}EU$z?+;flSht_82c?kh?BMjmpPrLBt`WtRZpnkyoGi55aQ0)%GRo?SY zyEGxH@7#22nC}Tbw$Fki3q{k9lb$#3Q`t?=4Gzvn@`vkV+E+e~MPcJ{g=Ed(0X*01 zZ@4wV-^h@wABW`1bSohrR%wGts&_)3Qd*#$rpX&M1E;NBC7E6IQh)z-(za=kjUY4 z50Zrs`~?><)M_>?|E9c1%k0!cutvdx%G8E=>7L+U0}cLDgng5T9(;g?H5z^hSKk~n z%+l=eZh1AB5I&OLX#S)AEw8&G{~I=KmcQTkkmW*QHcp*8`mB$I>~{6bTx`R7`N_Ma zxo)E$)oZW6Ay48mj-%!u=9eKI6UsBf&!Lom-k90avgBTA-?~b2#?4fD@uvj*6r>Sy zfNhlCUB2--DVR22nZl-KpcyOLuirTGC!|i5 zji<=#uxdLC^S~U^j&ZJ8jVCb)Az@+CN3V*TC$(_WWmZub3`;kpmFdMNcNh^p)Xlum<-~9?~IG`E}Ymk=aHlPP}klX+AcP&%*=6(-a@Gg9G z#jorPy6q}F3wvn~Klws=_)p&pMZwECj$VL<)bdPM!!3`NZ-btnI3aT|h^l}c3HjsZ zVBhM0q<_o|<$=%Nc)+(kru7}uW}U4KBCYGOg^0Xz3e4EA^~r%@W{I}$lR;Q<1 z$nf1@hBU;`t$*X z;k+WPAO2#Tm!_TKy=vDFJ9P{CO@`AS>Cg=y1-!((4_7|Mu(z7WPxdynU~Z~YKQj7} z&Y)~lNKH@;CeL6BTWbjjgZKc!EUux(Zw)hH5Fdbig0)wVHSc>{{{Q*lqY3#raKYIL zcDuu3zr#+^V-kaL{qwpg zyhoGQ{5}2rOL33DD#uKbs z$7>(b%Cqysdv-Y$6Gb{3q^o&Lx?xlfC;r&{b1yC)$tkIZ2USHrRF;FWJ_O#Sa`j?} z*hrMCWw^&YBpjg>2Vbl<6Q<5TeNo9e-|=?A-btA_4M%c#AyV0#bEORn$m*W@Ic^oi z`4be@wAp1G7?y(W-?{`tN7MZRWvZ+IiBs`$YgJwCI?2YTo+(||BZgvRkG2n$gvvVM z@s#)g_B?N!y41D^U&byZpSCiJS-v3S6*GJ zc7AquQvNe={c29d6zSf&Tt|S?jbZ}pAxo|Yozbxkm+w@maH;IU-Y*XXQ#&vCQYgM| zELshkktkO$L$16k8iLFV?fv|Dd9?;b^EycWU3Z)W;FIk&8R0?2@oJDx2t zw0#KV#mlhEoL8Y!^1vWk=JUGQ3t=#NXe~xo;?ARNne6TjZC&xgA$q=ICi-B$5fC_vBfX z7sHv58BUnYX#U|~`A_C2WEZVqq-%EY$XJXf!x=sxeG5xS6QSoU2mVcn4|FWgj-2Dg zp3eW%&xMB}nYOY~;|I>zZg5E!|9zMz&$6;emWCANl~5j_VRk*u8&R*eO729g0m0R* z>BA$!45o5!=W)R};jpk`xnvbDOsqeV3iZ@pB3BzXZ`I6d-K0!-?U8+am_k*Bj2$N7 z50ZHM9R0u!bI{Rc+h_EhJ0;K=ew?d30+N-5Q;)_1vMJJp^Srycfb7VRr1wl4vvv|3 zl7i;y?Vhd6q_})C8V26?>zPP{!^KxniPF{7D*0Tf#Z})>%uZxbxLiFQ)HXgLBRBxG zFy=o}(fn4-g2Nw$jEl?qapG1Vl{C_`wsyVz{9g}C9gY@Fo4uO@c?HFLGy{1BMG5gg zkK-Bn>{|2at|k9Ar#=$psts#rwBbaX7XF}j4Wh4*J>@!J2ZP~h47xY}SCuo?voY+>qJr#dC-d)3iktkQXW`=F8Hn1K<_O(aN zb~V&6#@j*HJ_h}V^t-R?tsUi*pekuVGhy1ajnlXIdDeO9LGP8$^)Kk2>F#~cmeTZn z=ElMuf`ck(7J#je%xPaQIg{U^I@3u?V!Zs4WASxEDQ0(J^>#jb&=jm#7WO3LL@`K$ z&C3O)!|7Ru3~1a0@^Qg%{ur0}PYla+b5U9^{e1-j5x%-x=>Q?0)FM3S}vqCOJ9jyL+QN zwRWnvWwYdFXJVmPvi@8;NpNRD@kKOt&?G2GRaG1?t)|!yU97 z#i3KaRM>IlC83!hUVuH*+2~O>zjCM4ZCxs5^FM}zs!G$n zX=B!Qz(Ex>=~3`|x;rJaeKWRLasC;W!BPoC%7Ufh%PuICP8_`6-;*bO*;LF3g4JR% zxN!*rO>jOVUfyxcCrc|A#J5T!sc;L2?L&{cKZc`4p`_W9@4@z}nKNg~!6zLrd4&a% zgNrpNrzWfc~3|;;6{`0h7%?;QU6gt z(eSDK4>x?2el&bEKFj|jk1v$(dY6(|n5Mc-GGT_j?S9x?A3sUVVm|>#u3r3s_t@(W zmyhM2u)nuc=lc2&{IHajl}T=HZo1*~I8C|XG+YbTzBH~{Bo)2uuqEhZzp7Ys733#_ zKKuma>SlM@jYmpGqFjxxs?@q|X*8HnU%a-;>VMSTZz8>>QvGRjLe3;XYv_3OFZgJzD>6WGcz@F(ZND+p-i!kJ!g{jW889oOFRnTO(uHdf6!jysvZ2r$z z-QWv6%8j_`0~;dAfdXtokF|$$(2|Xn+)$F}$qMGLgGJ5cda^Qj$_Ma7>o`64X77C% z((c6xW;}MZ8%K_IU|)8BPNtOOsL!(U5-8{#jgyOQm`|Fhz^P^6$MQOG{=*705LD49 z{b&VBQsoRLXp?lD5+psbr_WfqfkTEr*^`kby z5f-V+cQE%D6m(u$p7f5w9lp33Eln1>ShdwGBd?de1{tlsIs@XiCV_-qMYU^qa-^z>qw zT~9E>BC$@~6TJaPeU2~QPwzM`FE5o`3=?u5(aAF@(S+$cyZCRkLPYbgLqwVakDevf zdyOi%Xnaxn@t7SJ4e!bW92p}0D7`o@Jpv;%|79ias!+=})yf3r^xEo@wVnFjOA zqyXou*OupxKNzE~W2Nb(KTFTH&~_SW;4wR?s2tyAurA zL(>JV&H>}WdKQ`~5YrJR_DZnV5_2;bA2>70H#iD|8ps2$>+nZJkY6F!)Aep-gm~CE`Jvd8DT< z4#Q%#hW0iYf5;rE8eOG(NO?u14Y!z*(y%Fh%&(vSh=1b^Ax+eON_PnK?o@!$^hC0g z@EXle)I*eB)PG9%LGpjg1ZCIcR|%(ol(*a;&SCDY#naJaTOyg$7C7Z%?Nx+NERR!9 zJZ8kpRi-z8t5U@F*UyJ3L$hJKLpx&+t)lTZU6q3^v$_Iif0__%P`ZCHpw|K=uw|!a1VxuGds3o<3I0K zMd7HhvskQX46-+hMPe9YD!}bg#p7xuqqs;~+BhI2R7QjDR}EToYF z;e-=mm=g>Uxd(I=Cro@&*&VZb6@Y@V5!^od4c;!OrqRnE32S{`-2`BnAa-vYU{Ax48xR!jBdz z=~(4{!qwb;9rgOjr}~OIAo8$0&U^1!37i{Ia`m2fX&zK9vkjnJ0qVn)6Zed%ZbHS8 zUxPrB=AZWDgFXo7L(-s|2Z_T`?FfT7-PGOl%TVrN!iYOlG-aNY%{^bbUb{o;ah7*C z+GchF8E4ug=fHppnoQFLV$GW*0}Vq~E)Qnlpal(o@hD$C2%L&E8D;Rq-_70 z=)JZ}M-LW^;f(HHJgPvcq_MJEFVBF0@eq-IUZkN<5a;C@?uo^4=!owpnO_WRgKiJv z>gw&%4~x#8PTADdB4ZBNPb#r!qyR(jI!_L1(tL1;h}q%&&U_4am~#V87w4Kf|Ky4P zQGrnUap6%2(omcNuW>RED+gXYNSv`54C1uoLmvj6Z)ZoEd>FUEAa3r=Gx5D4|NG9E zCP&W;pMTxa+9=7CC9|}v%#vsCmo4j-hsci~TO;p0@mN>ZEF1f%WcdJhr4g5%IRA%U z{;@79Xg3!hiahAz1 z&(Urfq()Sw=t1IiGpIN?W6<(#(02%WFq{w%fkDMRddl!*KS58#8JR&ZeZ2UkV%`l2R#g$8yHd(Vakp=YlRo%tcO$Mef0M%hd6!Rqa9 z+@M9gxxG`mOA4fB>SQS{E5Q|k7#_s%b_Q2&Yl-vwkcPhwi$ltm=L$H~{2L$03&$f* zGD$wx_Y=AZcIwJVxEH)zXO{1ypfQNVTpIzvh2Nrd(b9 z-qZZw2^SceQIt53`~O{)AGJ@SIM&mVQLI!$}$q@I8)eGj_0JpOtf7@UC! z^}!EeR$if$&;5WDR85hVN4|=+N{dxLSbb-xv~e65R6%oPBo%ZH6m*=+2bx zi}w8ld(v=YI=iqKjE9kQ;&vKU&R7_>sz| zD52uWw?TXzId2B>mf)Z9kOt#T@+2#Nl?6=&Wg(tMO*`boAZxeb zgo-nrjE8aRV>qE;zVK){cERE9#rx$tgL{8<9p;3aL-srSh1>T2@oU`P)7lUsJ!R5( zx%9$wT^(S)Sr@!N=7qSQzd)XcoBzI!Mkr`^=!1Urgi?`yAqU78&e0cQ5okUd|IS=o zFtQ%2{V7wFG4c|r+N0^VEI>> zO+wFT59F>4Ww=OOChx4pnqEMy(M&IKhC!gBeNL2;{m+p0r+*}ki|^3|e(51^uqI#z zRM2D;d%anm+a$ZVO3OeCpMkx?j24jEqp)(IpbPSHrKGr6$|p^b;wh7)0jsK;pftMx zsX)7QpE8EMJ!Q<^w0qdIw#V#}L#C12sk));yP%jmp`g3*5zy4pE^BdfRUYD=Fl!$v z!3x{_JS?z*g4V57_|m52I3bRIA*!q){R3rhsQEVnDu?(ndd9i{|HG&T7?{8697rDt zs~vy%u%a0B{L#*wnPK9Z@x3(vH-Gy}psU{0%@uRGoFBl7;+zAnkO zcHQ{RFGzktUYPeD!-F(zp6iY533o>EcVa z>uKXSkW@iip;1BSbk|E};V8$~Xai3gZ0HGsI22$u_8Avpu~cP-97mM}b=|$LXKIoMzL~p7Mi`us0GK@x~Arq^%t7=9e z9R<4YIg5(fORKhh9!`-lu`x)0CQF2h4>A7(DL^s_BpkB<9Zd&aT%H2;I7+Yd><0m& zI3W?bdC<8yJN_m!>J<)ze3(~*mT!Y_8j<;*jl)2`dEM1aUdU`*wOH=>;m4)Dr9Nc8 zqc1!~@19>=E6W!BD}mHy7o97opK`p` zbbp9DAu(wSp94G!RriTk&XD%Za_Lz6jN|zr0>3}x>KoVouTR5}Jt0?@Uidya{T;_y z65`M*Ct)xzo;;~X;${pt_h6VHE%P6=3*tj{H-83$G=g}tEWYp9{xJ6 zRo$o~sOIhM470uhlZ*jWnmjr$^UZrP6mbG}N9B2qkKewa0q2klm5OoSQ8IbR)Pq1W zq?ZfFu%f$l%Qn3d&K>xvgQt)){6Q&0S1TRfh~+P`;TIs>_|7S3_GJ1nB&+vM!@e6& z5A3+#xf4Be&UJ5WX_uAgssH@Rg|cw*YjVMR&z2f)ugZ1j7YFw}NAs;c6E9E68dQ~c z{8;`5%72vZfbWciyd&hlM_6C!gn;VB2$)|&8g-0^_i1_V(fp)JZ`6M2i4;+xUW}t!bMqeeMC_UD4$`;X2B#>sQEM zZv7wGu=15)G>RWIdzRe(i<@=sIV%4No(HlRV*cCLJ}XU~ICx{iJl#gbc~;7?me)YE z-=_A)6P5|vu&o_AQn=w^DHwMM7Q5n12CYKc{v2*tP6>wpTHN9VQFVS(6&C$VSkBv;sEjTLqZHnF0x5#iP=icK%( zhH`L%S4%SrfLpK7lV+luR6EY_J!8|3yX=nBZLxlFy7v_b< z6UnP!_44FB zKa~gnc)fJAgy)=%z`ir5%Z2Bjy~`Vr7^Z@n7vC)nRYyq~4&LBCUv4EbMP%toL&pK* zi+zZ<_hv~CcB15V)Ji^vQ@MSQ;~b_p3^Fv^qq9yu33Bz^9+RsUjQ*|PxC!emj&eip z!33r%WY$gkw>oM#As)^3Ao+IzEs#ei@aog19nE0z(VYyXCqfP5;g-Nic-}^wTQM1n zl1EGHi@(#gMY*GA?eaRtq>faRR6+CWlPB`DZd@k0?Q5lQ61KN^y!Ih;6tv@efZdt$Be>C>wX+cUx&*ycviP7JlB(EubKy>=v0w*2uTtj!kDXOFB(JJ zczXD79veO9cBFF~l<_vsZC6&vn zLt`+An|mOCzF)JWh2exW3}<$HH+&#>`U#p;{!_UJX%QdHzvbH?{ToimXRz?}zoYJ? z5j2C)<6@w9pLwc0_u^t%?UivbXx+w@^77+x2H>L zmoJj1@Bfwj>yO`-4Xe;=_C_MiPY|wIyB;5tF*$bL;qeA!ko0QURPMnDh{J#Ts{cq6 zJ~aC#o*-3KcF+#jQdr?c1Et@|8ynr>0@5wrMN+!uaU54XUa|_xot!3P6IjY^qVGNn za`nP1kgN3xzIK&d@$jU64kN<6vSgV&`Sc5N+R4XBF&4b}CMrL^gNINi)DNJ^he3Kt z89v~L{!PRP6(?PDAkG&L2IF?waDFXL2GhW1!Em0Pkx^P$!&Ed1z3Kg=ec64|xp}GN zjy+iG@Y_-5aj8^dE5xCpwzf9e{L;Nr-myZ;=DbhY8{)6CN%^Y_xllGk6rvF=L<5?GUM>$4@k-B=GY)lAX>-^L*IDA~ zy-qZ!&7j!;1)W)0DHG-&f=0L;10|36vuk~stVSAFhl@|!>Y zbM5 zaI#hEo0U|qJ`1_J{jNbH+8J`S;D|%y@{7;IjrWlwr2qgx07*naRBb<(CCgU$@lD6# z+Q0nfN5at>r<|e{AX-8EpdK<@|B>)f@~Oy0-6P~uwkMJJxJ9lRM}+ob4MzE~pOXAC z9DWu2;E2+H3%(~+(ERG8VAnnWdl^*_lj2$L(5$n!%O0-wdSMgcXQGYgfeXNND-AMU(IuKwcn`1orYN=}F6hpTBn zebuG%-m~7RBQhVo`b)B9Tb9uD(M?jUI#c>{Rt-y=?2FXeSV57S{n8 z@`pSB4!QdFE|aUzec~f>;rq^3d8(^#kehz;8+r6USk&ejp7DO$kH04;<0x597qUAQ zPoMd6(A)@VlI15Zl)&JTvNTx!gZ|^V7lDgrkYB3{gft9ic6>KHxx33pOxjjHj_q?eU#ASn-xC_BeP+ zMP625TLoe-RQEc$9e52*?1>CDfSUM!r*8C4-A zP|9PlP-ygiGo>7hDk{p$q>`6mV2GP5whOqW3h_;wI!PXW>?y6N?a0q_FTN`0zxPZ% zO^nJW!Kg&TmXHsPLE_t)e~n7Xk**RYVUM;4gK?Yu)33$JU>evg7|ydJ)K_Q6yjlJX z(x*?$=&NPmCzuAyx4|KW55z(Lnm!LI?$YNRh||)8!MH7bbhhjQb>Yl{;XFIQ1F>l9 z65Pey(9|r8v4Cu6ZPj?!J5Q23|KAVgl;e+fDL^`>LH-_kY+*pI7R&iKij-41mL%s{ ze_cAw!Jy^aU?l%7Oa38^tzFVS;Z!J<(W;y|_slw!HmNy4#bXbB6W`ojB1M}X#k~SG zl2urtb`-Fvzc}>_IS}VKd$A*a+25g5mDg;W&$0_0Yw9NcO}1j6)@|4%_2^;oU;;LDD$kw}_~{O&Uo9;Ya8Nf7 z5|2KNMEwwfv7m%BbTd1?8=lPF=on1pe<<#nVR+O$gD`2oh4NnrSvvF76XZgy+^1Q& zX05bi;oxu$Z#z5h-Dk+JessND^^ptl0f?y+lNT_pv=j@qAis%n#a>Qeb@gE7%R~*} z-nQZqY2N&%Y#4pIj2SmU$`L=e4W);+hgXuOsH{wSW0_d5g1Z7XJRlW?&xj|e|KR{nQ^mG zWsmoSvp?w5zU?R9mwnJ9E<{6>hu$>L`{vo&+7rk2i7f4*yQ1ZAdK*44w$(Mt=B?Z0 z`kQ|y5B>W|zm$FlTJJvd9pT25IYZVcgM>7Ot6Vx8{O)-E9StRz;wQ{+i@UGA4wnIJ z!eszW(%+3Unz7Z4J~2$8=cZ@o6i8+s&TY;sbv*@~rB0KWAGP*>_Lls|`InqO=FZdV z{{7@L^2ig<$iinA$+9YHRUwhUf>^G_4x>YOv)p|a>2qQTP_O$XoGS1*+2)z8YRs5b^Ix5!JczAoSR!L6a;b>6=9n_Ipu9QtN@txl7s;qms$ zVsg;pBcw_6KeYV2(8+#>7k+!v(~mRhn_m8tNWmznJm%w)omYf>Fy6PO8BjrQ*}O?I zAG=1XPWUVueYg5G%Kp>>9S#vx=!UcM5LY49V?fb+hjQkS(%ciy@f9?OgAP7m7KX4f zB#zz-S6k|kBBJ1zlzu2cd2*#T&-vy;F>YDlTp@eTAIE}^T5n5EDQ>lT{N6vvtQk`f zZr}=P^X-H*;oKy%G3D*mfJg|-GogltV40m-_{!h82L+hhZ93OIC++KAfKtWt)Wd@ABp9ufM%Tjz8*f z{n&F~KC{Y>Y-R*nUOs0Q^%>mXi8x>^7W7EbBQnUU}sTez5igDbeE2Fq#?tXkuCqcB~w z_njesL3=oP;y90XB0c&`ZNlkczD;Q}>`?M=ew_a}4_(~Wwz&EkuM0y>Z#aEc-`N(n zympWDpo3C=+@~e4wE8Xk@tUw4lmbUV>Fa1fhYSsaD(5(Z-4KZ}I0^?Z5GUxakusda zcJ8`E+=P{l!|rpq^EoCRn12qMT&(aeEGX1DW^PGv;oTRcYY1KOF${;%IRpvM`eoXK zM)ixIxl%s zTR7u`d*`q5)Kw6^$Iwe|eBpP}+MOXQW3#2bV4UP)K9V0OZVOvHbUVtg9_7a}}c=S>2N_}8K(jK@ZguEZ1euEj~{ z8e2AYpa?0jkXSW%Ng%D9`7zpG`ywH!CjYs(tjRX zcfRngo3T)?NTyDnplNjRW=PXZx67uEJXuqEq7+n45H{Z2a>ISGmDpltu%e!vTbDNwJib`Kl ze!{pbLscM{uH(mA8xCK(<(I!x1f~VBj?;K-!}77HmeVgh;Axq6dR{y5+wb~^Oh%i?V@gRU7>_cDbhS~_H{{RND?uG zmX5s{TG!Ff0~;vqLAN*P$~jj^#^5m2!l3|Yql#aYh{}U2KV_CRqJ9Ly0!;Al6)8xP zjw?_{%!Z=l!-md1o`!JR7&bE#Td~mNDK9J5-qUF(9V5$EtdhI_;m%e5J+|h|oikgm zxcFSH6mCAsr6Hc)IHp;}tHF5OE{)rp|A!uXS}wlo^Qs)~f8;;%$De&)j$6RHk27!z z683O5Z#XNvGHS6mFf)ALWh(b&fyp))31mOm>S z=kD^&k1KNzmJKNc{Toe#UK9Wf?nt012=xHseu$Z$ENlbJM^DGeRUcO3M&#;_zsuYM z_me9=bRI)#BO9j_PixoK71F%q9@!w1rDfDnQe9oG4?nIrtesZQajxwQjLMFDCF7+(OU02N_LQ_!TrNJ76$hy(&^qGR$}s3|xTTbeixNoxZVHMGq=xha zUKI}RWGAFC`WykwK+9Z=1DDvrC`8YnGI!Ob=gZ{rW961#{T_AM4TpdD6W7Yh)wS|@ zy)x5H@o_}brw829YiTF+-2?fFDnM6%=rf!?qi6O8qv0(-zW*WS-}jGr9j7gRmloqb z-wMVasCugT`Cm$Z+cw;eFj3(=JqkI@NTrGCOkJfDn zVnIIUu%VnupTphndDkg24kJF__`#2*6(=ateB*|n$WknT{Miq_iuM7cUS4|;m0y!T zKM$agk9@IBv-~Ohk#@F6<%eCYBaJ;RTcqr;3(-cpDF@%L zC2%i;11jiPXOm=Mn-ay)E5C%SIR?fBm7Nc5;GEpLaAc?-(s1X&*_$0b8BTYPh#!`U z$h{b3%c?kAST#`y@j)1FU*P`e9yE--ST+9T&t554T>WKTM8-Mki~sv`^4LAU!#-+v z5}C!Fm_9EWKQ@xl!k8&Ue25a#{57oqFMXMh-4G$J`l* ztl4-E*0UN&ASu{1gx1S8rER@yQ`A)2>|t{+JQa2ecvOd8cC4pe3OY7OS=&ab!eXZE z-d1dVnBscZ*qg|E!x@WpfR%qNBaK;O_i&e%ME;szD@PZ`g0n1{jp3BPD@%*X^&E_z zgUR9=D1GsefW|OT9!Q^P5mXZ zruYmgts1MnW=>UbWW+wc;xVPMoy`ILxI(X#Q$hS#-PnDC)NS~uWW0QblpXSZOzXI2 z(UgxVv;>6#X~t>~{JOtTIz6Ee*2iSw&5mIWC#2D%iIJ4TEcB!^vUE}L^U^oU*qYHgYGdg(NRL5Wl^txXZBRmi>OjAiCb}7J z>5IC}QRsdrWkrf7g7*5DPR~`-qadVHtAdPU0?v4J#DnCuDWIf$Fzpq*0k^2$70z75jd8H0Q?eBk^uF6WQwG=GLO z-9`f+G?s?)!{3y)#w~tlM9J)hKcPipWG25TIn>#|_6ck?!@MK5=CBZH z*uc30B?i{%zk@;CxhDP^4JC#jqh<7I&or<{$GyS&x{|`L! zq+Ey|I~A~*WA`5|H(hq9{NeG{pdUA@*KW`SUI$lTJ;2nHbQP+lE$(akF;m*6xJjE2 z6*!f*DR3%rHfB^J!6&Z>J&?j0Y0jH0>$CQet+CNqA>Sg>xJr6g|695?y(axoOf*^e z*yDmTD;wGwFO9#n3mWrlFz9A@(9LjzLHa6JkZre^Cm>fiS#HAEuv{@b|AOHeU(il{ zU@;H>I0VVl^7u7hjZ^WWa@F6sM)u2kT{iVp$m){gq_BF1_L`}fgNKl?9MWd@aKL1r z^6uJ`4B6B-S_T#*f2+OI##I+0c-2 znKCK{ox7v#dE ze>WsvgJJP$vIcg%_A$3}qm*FJY4w(eCAX(RvN?avxoZrIb1!x#hV87(G(r_7V1zV? zTjjv}N=qd_Mk|?oC~4Ajl;gh?S+9qIPdHyj>B0F8hB4d;Jd?%rv4Q8%W2_X7HHN#L zTrGuMEz@{@aN9#tHubY|-;|6eLwslbVwwn@ zG3DmMaL79c$~gl~SQZr#wvuw|-!(YTdpu5G{PiF2(y*L9JNL4y<=bDp8Vk!V&^(ZC zVtedl+>edP4W!5Al?k)tc=-wDU(@TkCjiT1{^+lM?kc$iDe(V*nMR= z@V?F#DQsVl72>s0S^I)y^>?YV<_d8h;g?-hrDe|2va&ZkUV?@hBk zGDAP+&mbk83fUS-hND}e{8O%SxHlh1&35NDNmJu%a`ihW$j7m|wy~u{k88Z)M?aUl zPCQ-4jT@(<5LC`o%)7HJnS-1kOi#{2WevE=FKwOtIkKKNd^q#00(?lLZQ`z$ z9E|*3@_~2D#PQ?gyO1-UA;Eln_N(8QrOQ{!jo-LN*RH@T;zE30dOTpSuKvZ7L(k5? z=E4i>+useCg9y zqOo+W;L!7jKXDy;gxwfQaP*jG%eO&4B8wytcRW6k*Ri_twDYbADd(v*Me@f_EeI)R zrun!7sx9du*oC3^3$FR8bhXy&@PEj054^BZqjIrfa4H)%a2&4XJT@CT&SUciCeCFO z)=ZcxS!3tQwoxa_tEK-XuN0gjOBxDg{i@fc{@LHiw#R>n(}Dhu`RiAt9}5(<@_~^C zt=tV-xf`@%Fi5r-PUs#dYhjkrk)A#OzF){7%AMg-l|R$e+ptPnU%gv8U;3T2;8wQQ z@h8hgAG*x{u&KlS=hs|wjnA{-UjA@;(r>$)1E!?8NRKP&E33!L>dMoluBlZI2F=Hkd`0DUtOs}2icE9#qn;Chc)Hm-T`;M1J{p~ zKw4~?AHxY9&H{`VAq$5xX#&f#Qk+n6`lcNq9XwnS;&><^ItSezor69LbA>f- zND#w1j*ihKO&6AQNy8bX=uvj@(q1p^fP$ft;n&pYs&XC9efEXL8mtz__HnNA1f0vv zJ&78bQgOUsF8Ie-3^JfggM-BX@!(_fLF{kl9Fdum4yut`KYF-Sm%E{MvnQ-5&Xs>W zzaEQK+`Yt=c<<=OmN%Hz&R%)t*?-G9=UzCXdS2u=c}QOJ3C>@$!Q=cjd)YRJO~@y= zkn`7Lr9K-6r{wM@+hSEZe->+6jrr@xr32T?^|fuq%JDAmd~uurrko(hU6GK-iWB0| z#sx^S@~0hp`fbVNY8(!fz;Gdl5Ph16Ln}REAf$0+%|_Dw^|aOD_|-+&%Y3i2Zh2Ey zcUH>!(xW9dYJVxKtdS{Grs&G-7hil)!>?VtR;L0^IR(RSlwn4eHY0)qEDMugjuN5U zEL&te4)NJ6*&80mmcog;j)IA`Qd6mbTyeV0$<-eqdoDX%GsA;Q+#Ga5KS({1j%!04 zO~XAB8jsmAfQbvCdzfa_Jq&l}@}dpM6c@oRz&aY<=vLRzsPU}9T9$hrcobJw&eug| zCU=$(!eCk~ehnP>3HqnCrOo^YX+-@7$oOc~|6ueOD4PGEo#Om~@#FR)$i@oqoQg@( z_VVr0(~rIIqo!-!Gd{RbBl9;wL33`gW$PNrX;b$LEk+{oWf}^6mS?X zY)Dn$vBl?>Uj_xHM+s7!8-H=G*xwxPa5US91g z)*;g>WL#x|tgq`($2A+aNgje8Ip5vJl`Q<8w4=cD2`X$hc6_)!jEb2`+ZLMfoywc< zIjBTEIKs3(XNqjh-51A|R_KDW{-%|<_LTR*zUWRo!_*7TpT&OF_@x7uXO^{9aw~U( zCRaS?Jt|lH@M8nVDGj4xG=q@k-`l!HIybz8tx^w4`$6T-B(tuSRu8wwMy!lXP%Km4mm_-&YbDXRWOb;zUMeV*$pab;*FWr($L;7 z1?wMz&c@=&^6@HTmP*Q1d{Jw#z2c%)caj(vr&4xk3Iz{pV`(Miqy9aZ)W2uxWyQ~{ zg{QO84B~3%1q!-*-A;*Q7Z3@}K~vb={BxU* z-7`KijEa{g3Y~5ImprB)D_vXb-Kc=CGc zq-fhFsj6Qh#r=;u;2_2PCU!X$G&|lWH2LG{w7uigS=u84F*S+pUEe2 zp03u@xK_IBS4sCa9D|DOD~*|BWJ|_A(vG83^LRcyWT(>m_#kcvjWAFASoz`?zbKbn za*5ug#CR{e>@uOEo<4oLCWAamn~}i*mKQ%LS&o!fj=I##A+n^dQf97tQhFL{rFg&7 zgg3^ya&yDFiA9KU7lhY83X*36XuM&JJIC$dpY%z?mk$PJL>>_aqvR0kiDNhqoZG8t zLwMcQSIDpbaF_fEikXI2?Vf$%mGZ-HU5h)e|4SE`(cb7=T!ce0(ex#RH!=o6G(_6$ z4KmUI9;L5=2gpBzM*T!XW2Bjof@b2FDK6;A#Nik! zbYPf5b#u5e*@VNI7xLEZNQ)-G@A!*n8THSwxOq)!7)>SO{aCyPH7?ITz0lt5_-?q- zisr{vMkr|9BV1mN#ZC}v9%Ax|Yrdhk7V*lri?Ai?;lKVG$|O%I2IRxftI@aoa6X7M z?JnXP3}+vT71~2FbEj7b6?#htuAl4em)F-fV++>*XYVQi?JAD;Ja>tEh>!#eTAUVl zFAl}s-MzF>v{(zJKxspPQuu+E0xcAZTYx~E5G5`z&t3lSo838g?|t_rFA#aOcaw8x z&hG5&%%1aR&ia;e!o43#Llb9GiyU>-kuq-FmpU7`E5BT(ugOTrI_EoP7oD}fb1;N- z*5QOR+c?F+-f-f>WDXrRW{qbwRY+dd5-F@$DBUx@h667{t2$R5byiVt9JAR&GI6En zcf(u?*V?xC*yIYK%1~P&&DbuGUF%Ji3#F;zTWM{om-_5p*tfj9RAp^1jX35m*DpKk z4#$&m%h5+4E$_VZj?!5Dgz}{Dbyov? ze)g{N`EjmA8Lr98N~C|q$I|QL$FQa30lGmCrR2_3RAJl^ zhIs6eM5Ky1!{fdH^@2Tf1^BbFPMJ{7MFy5 z(kSxcm^3fq#h5t8^MOD3bez(2(%5_)ey|VDVa)&y!^-sM_Iy0>;$@6MGn7>~%C}2v zWv?yz>HUg+Cd{QIj#nw9nQx4->OTViy0KeWJlwu|!9 zNYJR*q%&(>huz^}w>a_K!fw)5&_`->2g%~hVKOgsl$53Qk>z!1QdwRm4QS~!E&56t zFrHmI>m#XMFhOb-Op)dDr^%x)eL}mZ;h;^iOIzJkX`1!6>$)ymD2vN$WnO)u%+1(9 z=H_iB75Sqit=lju>d{}iV~G?e66s_jOo-#eICf=IR?1C1sFUeMsPC0OyyU(2-qUgy zELb4(=g*h@_upS>Ns|g)rv~^u?7if3<;Pb#PElHx)>oF-XG;FUx16m%Y2IXg0HusO% z7zb)_j!iv|vt;;i>^STJF%@=gCtV`QIOE;f@|Y(KC`tn%ul5cZ$!k#mP?ir-@j$iaX3=td8^p+xw?`_p0cgcn^e;F)?}dS`eC{7GXkdPGm* zNRNvX!sz_CamA8lWx~x%7{0=bJbdV2eaUtEJ&kp8cclNQ;X}|0ZbxP+Ry0s@i+k!AIoB-n zqBAC>LuMB!Ww7N^=9oiW?2*t(vuA)a+7~ZgEMI^9wI+=lH%>;37$Kuak5(sb^eMRR zYry`d2+dLRIWp9x7s=e_ev(u5wY1Iu1RG}ekovZ49AXhTXxJ-`LGx59c0cQ%C~SSE(n~JzURFl;p#R&Dk z^1M^zt4Y)3-H+T6m-qf{ifjN!yZxx1PJj-6ciIqp>4Fz~&6dJJPyDt*HW^$hmmRvH zwyr;$?q-F*{PN54$Rm%)8E3?{E$x8i4<_0SBaI>hjUt^N>y^$h>=L)`aE@VH_c*)1 z*$YI-c+wCuANLlo$&KF#CacFVoF%(~I$XBk6F&&7RZkf#&057~uubN~N@hIs(UH@+ zQ;v%+x=1EXnk3VwPginwdvCVcW+^9~Sb1IG-kdWbpEch}eiX8|WEtShiekyDm?Q(n zzl1Z|ZJvr3Jg^0keL#}gPEwIC(+C+159!TTX}a1uXXP}5S~}+S2F>gmV%XT=iEFy| z17~_eb~@}kZoEwHeDEaceYbSm`4Y*>$=7EckgZeH_q{k7 zSdlbn8LjoYv_&fuS`xa7#mk{u^GA^Y+GBOlKO^rty^dj)5ckRcJ>@gi%SSsvdc{L_D-3)(kQe$rwC&ge!s{dvOT)N5ZW>**#rSTqJ#Zbe9`1KU)vkAU(&ExkYDX zRSoiSAf~+niRJN+;^8~5!v+F<$wD3O<$w4Bv$AXTzN4R#g6z1K_S+NOq_s@^f!%$| zmclV@3)fwDoqYcJ=XU#F2l-rhnSnG%X{~9tX6gK_+qn<+Rpz?uL4yYAUgATB43VKj zhblgJ@L+Alnn7e`alLj(OB&vDLlN>KU$&Y#+nkP;PTCnex5-GzD_P}JZpuufPC3bw zy^qQGq~g|H4cIfZ4=p}J_J;7?k-l5Wg#5i^=~8_7IpgRA zjm(1~utRu{83=i)tM#y5W5Uyg^yIHRSdNutFoB*j^U8sfF2cLU16z7_D@C_^FW9wn zmQaC$(w+v zZm{`9qYuHP(CHy-VTjAN}=B`ZY^N zT;qJvW1maCU*F8QWMP+0VmpjI{myLpzYj5C2~FxAMLF{8Q@7MD|Co>E9D3-XQdwD< zjLm;t9RKim(fF97Omy0GestW{EzPYp>1YXgI6@dm$9OJ{VjA;sjN8VP>5$2vWwA^v zKjg78oO#HHeCedQ#OjJGu86WtojO%}LHKc;3SF}X?44te3g0g~SccBJq}KqMTeuyq zStNXGlUsEWMF3Ho;GyDb#`wcXPiZh2eGo@#%4Co@c^V|%-br_p?r_pP*yWs4j>dJ2 z(vSytA7caLjt3u)bJ9#BFO}EhjAJlu2}4{Hll+7{6sPLoy{7cHx*+7mV9?(%wts+`K$sGAuHa| zbW%hAk~QNn@^+KNYz*2j$R|hT?V&5j$%5KVUZ8(S$QVlxanFn0LhK@H1)sjXdg#kL zamqKEXVTZxu>)DYY`di&H|&J|z0n#Uj8^n3|8`@a?6zKj_0D^2=33SG5@x&8l=+L3{6_KTNeu zO-b?-%14~RAn`OOs!mryh5hY=Yz6EZlheZ{*|8 zuq7p4a)xBOo8AV0y8j+>`p*tC7V`4g`SRp{W=8jk$a=jCQ#KR&vbWzA>$@hlCEY_N4OFDdaW@T+6%V&Qj<)1x?C42LsiP;?28fF@db%3o&gD4%#F2*|3GBtm3w(J zbeWzZZPoqs|Ggsn9ecJcT(ndjx$?9!LYN?*>W?@=R~O#is%`dqTRo6ZLVhjB;*z9) zAN{o~Ul&$C$`5f?pPXA%w9!HMJI$1`H-9CKb=8_?^|CHt%=-J%DQp@Rabu?vjp3J8 z6j*5tgdy%?26GNIz=@-?UfyrkpIDj(X>Bi`bSCWMi$ zD4(rpjz^ak7fMfz-&}ps8JG!Kq~wigu^xZ!RXWo<%EJ`OZ4h{({?}c6I{qLwbu{?B z>%}SZ>9l3heIoL+orlQF*Y6?+{HT9i5|>*Y^vk!g$x6Bm@5|YjE;6HTA`C>xBCq@+z@rUvlVvCF~`V~CD`KB-lZ<4LrNT11ALZr()^g`M;GTPgdrb2 zklz&Tg@a=HNck7fO7)a~Vog1}y;tx7Xi0=2Zj9-?7zjh$#kT*ocZd)db8`t+9>%3< z*Lq;j?q0BM8;lqx_uO)YY#cN~J{tR_?0V=)GGWRz?MaNvw6c?wZ||%*JSyShShANr zg6+kPk~$&}%M9@(tTZY?f3?rPE)*vZ-V^EtfcoJYcmADn;uufmGrc-0W=+P&IcN&P zR$&^)ps`j=g=guel;Iso6joZKv?=dvU1>Z{zI-a3<-gJ=m>?M0GZ6AQc-)uHyNILS z1V8)(mNSh#1NZAb{`lkL->a@8eoB&r8sPJ^XBsB) zPXQ-{qp?G9La#XIb{r$5(_egOzG!CFGBI zb&#%KLdZj?cmU(_k}ocH3$Yuk5Sy29cd#9{*#c+%{xlNnjdwnjTkiRtv|#y+=C^WJ z0NJhd=sh>;1_orrO-wjxtg&gu6NE;uS1M0lvV(3J+P7P57I)BLpmOm8>wp6e(76PW zpedwF)qr)6a?%)`^iDhNB&VN#dL-k&{`D`p@x~iKha!nk~i#S5#W03g}t@#N<9BJHly?FG&lHOb+ z%ip~ZPTFno;!9X-pq+zmYo;K0h&*#tzhJ1G|H4aztO&+gg{&%MtuT+)j_mXxb3((VkksToWZPj1?e(lb3?tUY5}Zk^Wv4LTHs@-HNP+RF(}+urq=b7@^R1O6tsWfXjSxpb=H9>9qPg8StZ0R7;fq6glLywU*g7PGWq((a`<-YJ({e~0diAG0e~HX$y( z^^mxh?_r2DO)sM*#HBPIc7`5ozK{OVw}klvmBI4_o`w_6iZ?j~Q{2m>{Da?Mf+>c6 z*K&v)w6!W*u}KLw;)d)<3M+{?gF$x}|4RiK2*1mv0@zQ(Z}hvQpHzczJp4_;nQ8oTO()1(h(cQdfXddfvN=&3QSwoE3moFw;09kiF6 zbksr0gFi2RJXfB1cXo8YA|w|(lAX2B2#infEC+5qASo%A?p$!e1xeFVk?Xz&*y^(J zXgX+KT+^;=Yat%rs}oK*L1xdMz0SJV)_pDga3$ae7$;@fB(i?!?q+LxAg;E4==Lty zU%p*dFJ&J;ic`_>fsb~U*v+~oEFP8F(O0e|2naDDD(P$cU-rB{HhC!9_ zFvK-6Nix}5z9jUkL;twCvAT-%f%V5wZ}DjKnpVx1CiAQ)B+1{3ueCbt1jRFyH`?() z7$?X40D#)|6tQ^O(EtK<4JzBQHQEYL+ zbOmJnG-4?NU(Ur#m&-?=eW_oSwme~5Y@_$;GY?^68P<<;8?a7#v5zc${!n@D@q1+G-~swK<6Q`Rvl~8)-qQs8&R#|~c(k_> zd{#yu`IdF{+6VG;f*()8${4tflx}hK%9o(6rUw(}pt%Nm(ey7Q|C^_!$95PKGw0k{ z!w@*Mr1B;5OG2+NJQmHXgWl<&n1lZFZ_u?oTDxO-K)DNTa{cYUmB*icF==tD63IQj zXN`R&rZY1q$ZN9kV_?zZW%A4mugHUs#nwJ@8P%_Uaiz|bOj>pVKYr{i#)rs0N1rW= zmXv9h9!1&m>=ipn?~>R!b5dSOB47J_f!y@Wc&Vv(YZ^(i)mB@{Yp=bge+)^o{Le&n z#*nTH2h9gyve9TST)0sF{O3Q*efQlLJsU2c7 z@}tbItn#U{e%l1eDeNwVBX*O_f}ZLy&52QK!y^ZnP!)8PM)O_hnTIm+;?9ZXmDSoH zS%96!ZomIA9I)|aR3djKd*F`+_=JFd4kO_?nR%Eb=eLwvFAfU!(NZdI% z2i@PlYb9&mfLDyaJ6Bqlj-e-zb8dQe?=9M(GW0y|f84q9IbE^h(dc4uzW?F3n&6 zT`H!%qkoV5m|2q{?vtECwKZdjl{x6dl3gv?+oC-XBnf*U;=We+yqHMHV7SlVD#X^x zku7j4%yO(Om(xEFJ`DWV2jl2t%tc18S0aOYx;>69+lt^j6BkQ$(Ap$zocsxyb@n6V z%E!{HmeykJ^1kSdpXnB$v*#{Q>hrGprHsaULT(evy}cuyl{tQ_PKfFYktSP?-avkF z^|^B8Ew0t{<*ahK?YW6^!_l!#ER=Z%KXaDU$c5Ov#Ez&W*&v*xbMK{x$%w)kQde0f z`6WHpNf&MhBB!E*8sIy});qTswH6ka%W)pTtXZ>^(l#|2HEL95rUgi~cAx_E@+wy%6PKQ7YrFv#-a^Z_;FbaD?jF+nSuE${0uIZMvJ=2nyl z;1U1;KmbWZK~x>Psj9A(pPhS+{QJ4zV+li{cK?L>b)^5#4%;6`!F?@H{oku9(_cUM zRyM)Xs+H-(kBqDSG)}7OV%^$1Z@-oN_V#O}1m#yx{ijrY^czXv`Erb7cCC|I;wt}t z{3d#@u;~RSjByP5=9_QU-QhTH$-^{`KKf{R=bd-7rOnn@+vXaM+70ddk9z*)nW4j{cqZCAz(L#qqyGoSNR`v7JJ5>Dwz&TOhKCMCgo- z9yf7gz_{ahFyoR1HZb^}F>vc+%6Z7XyGh?ZJ>?hoJR()qHJWzEgMZLQ3eOxUfQ?b1 zm||sj@IhdF$G7l}dTSnz@_aylg6EE_2L0>}r=)>fVi$Egtta^Cv z{n(@j%dPr=@5=Py(kp&pa!>AV=my$w1EAb6!5q)t}O6-8ig0eZYex_Ju*ni_gg* zaR!6#F7~VCqgD<2CmJJI1&ouZ5?jwvcK4>}t(FChP;r$P5rFB~w>1Y#6N-7V4b~9u zyZcUYV>Hja@R~d_=6S7CbBZg|pBo1}a@P&IwI~JPI5hX&=HVJ)p!^p-`nf##4;&aD z)mxjkvR-}TZA|W{bn+q-H(M#*=%mCG=og0GzQHq8zf!33}KumyFyHR z-hCJST|UR$IFeKDfXyYR=P;>$|30a!sahjWI&#ojJZg&@m87Mk=i5mVgGm$)MJC=Z zggipr6Ix{xhIl(Zj7XB-ePJb~USZ@v(6wEc?6^mip)U-}N>-ka)K?q=^N)(>t@238 zZs_hl^Mu2(ukFyN?CXB@fPD7VBzTS`L2)eo&7Ewz9xA**tENdpXa+lCk5@xdvtWoykyMdNw9E;LOvODKALX zoG(&N+A6LKLK;NAR~(z<(UdgS=bUqnY`Ecuk+fG{d8LdUJ9dp2gs6}dves*W&)&wR zZNi|PZQg(2deSgpSD9LJAXbnp#`@{UrGDXfI70k^(|D)>!H+>8ih4*q@uDeW93e0M z8YJ%WJE$l8VH~=UTOXrA(i=8sy+Qrtm)BmPjT6m|pZSZH$eu@@DKEbER=W>Kb@rek z3~|fb9y$q3a_7ul!%lG|8+5`VLbLE^_3BTKiS-3FB!BoW_^`^sFyg~HJAZY5G`Dlm z_%wrq<}xOTqfg6201C-CLWjq?DOgjh(nX-`S-bqtyo$%5t9ce^K?!l=A?Q^GOI`)| z-1~DnpayU9w~MHj=Rl(Vyyt%1+}D=ly*==CdHp44=-4xJHeikRiRWD_%dnh+GaaMu z5?_SV86_m>zs=?w%T1S^NfsJDpRru-d||S@i9<4uz54^1x!8^UFmaC`ZzKPC{t@}* zRam#~!<+AYg5_rqx;lmY6t1ZM|dIX6x7*yQ9CVX9Z(S8{WaZ4w;;>w4;hL8*6(D|5uQ&Lobp1$t7 z3gzN6kC!u!KRi08fJ?NFJNs(6_jhCT7)$czMQIEYCvSr$he4w=sCo4UJOK4V?g%b_2e=6TJ4v_M%UXaSkuSi=%70RX)bQi4_ar>8+<}gBD3mUJXg ze(qdpHcE<%^L0#`Ct)YjI+87HxGUIvV-q5z+6 zP)l=hCcl#)umqF9pAbA^3sF#mR)SqSWg2z;duNEoV_&{m<_C2Yb z@!@K9(vgEkNiN_751dpDeC>1*P*wd3a&Lg*UXpgeoF1X!?TA(q5l#*p&^<4K8bYWX z?vXf=Ts^oLmu$Go4i0wKYB48`p%wj;?YEQ@vEjUh*WP+x?)=Rkw40g}DBR<9dap?T z+paxdMsK*jv56o*1|G+Z$;-|@A(~~$#E0*^UN#;zJfb`O(wk-S*E4YMB<9~p-k_}| zyJUb=VcevlzSez|MLAMPSE>OTXpT!-cjqp<>>{U}a!MrPE3dpFx88bdYCJlsk^eq~ zA8Pg%dR_ZBlp~Vsp8DdFy^l>%Rwhzphxp^p9#l@e4XPr zNDE~+p~E4g(oq*Dufa%8FC%4AT|5kRHG3V#ncfDI^P?Q(%bys|I9^U>2QzbvaZKg@ z(m3vqQnO^%Y8!rvbI@!-r?+A35?$OB*oo$#;vP*VU?{AmlU#8^TuS31amE-VPTC0i zvKYhQ=-fkf+^oV}dS`wUCx;!jL;R8~>g;4B<^(WE{!Tux6OxtI!%S>EpPiMhTagxE zM&=1e9wb|C8XGsf9ZM13{pd3{mJCLwx4}qnq-dRiFaH1@!zLyi^Rcl0&_VK#C-0NJ zciGO%0|`OPt1v%L{_@D3Qi8Ky8ER_l7ak~qaQ`W(m}ud_S>oF7}dysC&C5^$EW$hX6H|Hrauqk=+>*h zEbq08Ovu?)md=|d%Rl|2G%lOz?1Q%r&PJH6AcPIlGzYEWYEWy@%mN`UwbmXI=R{fd z9&-ET61=egVIn_p?XHTI>}b~H_kybcN!dY zI$F@~dxee!%J_e5_Vb^00VsX&L>X^!tI|clt^(~YQ2zM(BcFh;GEMf=8JZwpI6161 zFH`p1yl;@o7m)HholS3prnkXRS2A(SCDPpa(cOwL9$hTgUUa&CT{4{~!klu^jWU1X zVmvs$q+Vx~w<7)f_vQA-+7<-QBaBtT(W~{0fuDcG)5JAvt~a zT-p8bGvp1N{HH^GN$ekGV;LqdOLB3pRA;@MOiuoITzyC&Um`!|VO%V);Tq}VEiS%H zch_G3Se-cDxdXS9?82UC{QOSmimWyVO-8)ZR?x-G$Ltj3uX9}nRRPpj1X*{y^DxBS zSB!gUh&yhCj=%kqLo7J?93O`}Sz;6`!@|j72W~S^aF-J+O&XRmjFkU- z>`uA;x(oCVA$nHesm~0MvKqb*PNLytH4FcE{av~3zQ>%5US6KkQiP+_nrHn-DyF`r z<0eL*f?cZyXq0VzG{>Vk26@v>H))HClJbxZo_xj0V|)iv;lIBItlOI(Zk*}Q@#$z; z)_@<$#NvbG+saIwJv~Nha9UPtE0)iqTGUPfhInMZ%r*da<{xXBbngAcJRI0PPL#D1 zv93D*WH}BSFj*+CtdgIea;ZG_)Qj4ZhA?sEIN4(Ml=&EEfn_mtMmEwU@(Xni@$aj@ z$!D-||u!OGeEq`5kVlyLKtn`BFX3y;&|qAI63Tygzjx6H{OJ}R#ks?Lt%d>tdHix{#>jZ zERfBy@yMAc9uZ0N{-@*QhC3d@3yQi_CyiGnDvyP-T-I|8xAZs}d1Lw~=_0xs=eh2O zwb>Sa@$28qYyWi%)q!w@g@gLaUGdVIc(=ka5d&*40xaMCwp z`R9e#-62=qdO!AgXwv+~Ka@fJ1LzGhFX;@1xJGs&o!nelz4|AXuam|$+*g{X{axyo zO#fcKEUr}^#$m-}K;nyVo>+U;6MVK^(_0dNhxQG0ByDhC4 zdGPV4KJo#it+gO(?R?enji{$Y8U z=;kgMx}(&5{E*bwRDS2b$@UJKE$H+tD7Nmp2<#yyLi?H`w10^-$?g*;haIuw;ACVG zPvgx;Pn^)eXczBBpJ}JcU10KSNuXZnU6Lw ziBdX7C<}whmk2`{h;y>ov%kLsUG8qSdIgS;JM#1^a3(DdL<#=o^ju%^*V|qy-?>Zb zE8Vtr9ZQi)SywecCvAq9UEMu;^w3r_w}rI8rB~CYP5aIU!T6-$532^uQFDCSwuEQN1fJZ3mKQUugtAzmF4L6uE+k~FcAnbOOR+koV5!bC+Rg45w(d1EpM7SX|9#d zMdPGU`tIfm)({rRPOhO}nJ&p+S#P%qQ z=4F$F*pj9Z<|ZcW{$}9$!ra89wzk#{lXNj1QsS)H8nB)g-I^08Wbq@88=Ua!-m9N1 zEZ#xB%GgoLW`8N=?2%U@_(^F^1@B8VmBB2=8Rl)juNZZnhQa$UF?=*zi-a)gA02S=a`?~8Q zaJ3#iN7Sv8#h}R-qv#l0uUA2o*wEA}(-&4p_aQ=_tE-EIUCjJr;ZiBLiAj-cw$Vtr z;Iw0+B6)h%B{$xM@o04M@Tqzoo$l1s#W^gw{;_&+w3avIb@=|f%Oz(WAC>#;-~J`{ z{r=CYE9tD8r(oz#l9^j5$gb|qH{VueXzTAL z^;jQmORwUDSjkPCgLYpJ7?Xg5roXVSB|`oSzp98QUiS4t=sc#H>x+1-e&g%Q)wzp{uUnCyUw*dix?^nF!?m~l zM&9}0GyhzvJVi>A!jXGPT2qabfAom%Ql&g2X;Vm7r~w)jTOZAdL0o!u=%I(o!3Q54 ziTA`4Pso@tW4_Z@d&DG#{Ge;V-V?3>prhslGCFFGPxtFTKpOgNC*!jBkR=P|;1G^K zOH<{1XJfq9_P6hBi5iH6bl2pyWuwgIPV93GtbDF)DTe*)UW!R<-O-u`Jon_I~j}dcWJ7fpP82hi{@S)Ujtb)=ZpsRy_gI43f@pOE+4}Z@dT{9xF3> z`HAT()}K5WvU?1Z?9#zf`Pm;=*`~%hXfjJfKLcCP5XB1}I0VLO1(A}$Jv0Z2Yn6Bi z_oiPj4munZEBK-z$Bu&KGoCEVv0|Ak4@c}cC?euXR<#Y>)h027I;?E;fQ}?U42KXMBleS4?7hZUwj2JN@ zDu}bZ$B*ZM3f6ZRC8d!6p&DQ#fi}oCIN>aBP9USRF6}!MlgM_LubYPAILnu$X2v`4 zUpS9}<^cBHV7#rj%2OiT5XS5=^H@GY+MZ0z@aD=EPMYnB)^rbSzI4bwyU5L#ovl5L z#_`ts9>-+X8!<#vhbZS?V*Fxdusm33Etk;Y%;EBopJ9X{&a`B~$?Rkg<#)N0ap;`> z%%4B($I`ZBvQ*Fhbd~-d+c{_s1aP1~xqzmsgCUK>>D;M04FWE8<{pN)eF1q|<_USl zSBR@h_hEqQ*MSPh-{Ks_ihuXoyq|D$5{BA_mZZl`n(Q6R^WF5Pn%rqDKQ~*tVbWNi zUftw|OHS7{&6KCEzCn(Mv#h{6!IK`cKq{U$1|86o*w!ur{I+_`7>(4U@_cRu<;-uz;YJn@G=NCnQJO>Jbt+Uk0NPTD4o*;?42|NQ4V zNr=+c)z!&SM;(QkvTj$SuBSmtqP0~6d{^kG*_!5TZ}z%aw>P2m+giTL+ea2JTBt`_ zHsSydb%JWQX~E1VIOuFkyaRV_H`*&_`dKD%`+k;}3_m10aE_dH_ zi45rLHr`=c#}U+A7t<3%o)QIcTc{I%u4I1O`;uSFmDK90K7VAurMxbhx8; zl?9y1hv@TCiJ zqcK?s9p0wrw?6QTD;!(=j{R~HTJU$Pex%9A%ySDfstTlW%8S?l{sZkQUGcNATq(FV zYar3+Zm_`ya`n|$MK@=faq zZB5tC;=T#7F=6-jA9I+j$#|PUOV2=PHa$kq-52@3=H}pViGp0+ZLxRH5*fY0aJdIF zyu)(`J{>nvesbvP^3^1sdGGj!FqFm0ON1d!h#Q@GCPKYnW+SMkH_-vd^a^EBmGx)E z`e)BRg=E06kdq|uTvCd7-5J<1PRE>^jBkw@t7hWs)03uyX~~C z9F0?8Ebt7~`yPD;FQoNEthoM6f5H$qe(mMK=LPEZ_#+O$@ov*!nX7})2!^=T zP#8x%@zQ1hhh{Q$B?5O`?18e`<;|hD+pa!O_S$(n<0f-4{IE;vO`?N&N_Pu*-6VR9=aR-jqyT#D5uJZhJ+F;v&o%8_SWDGxsO7reV*pgfpHqeu!5i8IY$h%=5C z2@HmKN9kG?eh3v0AlbDGAyJ5XLPTZcb(7q_n@Y{Nr#dxZEV+ZG7GS16)f^f3f`%{+0=VG0zZiF$}Sf-V0 zFw%P!^=HkbV=L9XoLskw3C4x5yYRGVmz^dY$#%l|H^|~8IIkJofiiTq{J1()U7Q{o zPkoe`gPCZ_KlRWrqod)Pnwl{leYKQg4{;0S*yX2ed4*(mcN>}1zJHI@ESslBzB6Bc z#wP{Wd<~efwcVsR5$GoD*Uc6+LsL_e9COSux^H*t*{%8K@k1(sA9pt4(6$TJZf`j1 zmVrN!$$7g<#jH=IYRaqV@oscZj!uoX&!E~t+Im_UM&cqc=%Xx2qv@ov3@cAJHE}zS z@z~1a4&P6%z3>#>jgcJKdcN|P_sjX%B9!~hM-3JhPrfXY7lXkNkJ3S`=_U+`h+8n+ zr|T_#{B1k<9re#&Z)Yj3y7Iygq471SbavwjR8HW8xRjrV zrnkY6AB`!O9^~U#N-n(`y50b}=7Ljne1^r$U${h0x#VUT)n+(nEksLiS6_^^I&l>7 zzX{^1fQYNF$D}X_)~|OjdFr>fqN~dt<}n|WGA_98PVIs>M&Hbut7vY&(URL|W2t%X zZmF$UnrcZ~1$80l-m9!=640<=!{mk=ZiwEidGqG!p&E47DJR{99$Xg?h3^g>HCxl1 z?agj)c6ZZJmk!!QzAiaX7FRY%`R7kdbJb#R>>#)rIIn>Ah5{$##WaJ&6EANx-FoRP zEXB&#WmtXC<-I$$2fgF^3-#Wr=kcdsl0$IV#{zYz#6HVvs@Y$RZI1}}4&tQGpOFF1IEy!^iXq-0voIKR zcL~3iNN9LKVo{c?N(bqkKocj6#bqI1%5VG)Cp5hblF0Cm(#eN(#?Rjpn3Lu|v_U`uj<*&C5*7Nm~DmZD|Uv@<=m+?%&wO0e? zq`B^ut>>L~+DXnh*>B=zi~YDpo!2KfG1YnsC)wsyLA|3R{-=Uy_S zZMal?^}IAJp6qg{4fL54>`i1#Bl`%tklt{>bgXj8$;QSinC0CaOR(7GP20rnK}QT5 z>%Lv>D&7@crex-PF(|H47@FW=^>@EoB)!a1`V~$u3z>3WshaS$&2} zUaz5&lix=&TT7&AQ4{FuWa-RB(!vc>0{P4a#z9JDG{jdy|J;7qO|5p3)Xa6mnarLn zL=HOskJWnsHTJ+eascflVi6`K(8MnarfFPf`G+F=yv%PGlfw!M$yb1flf~l6z}s}D ztcI^4{khv%ZcdgI!AW<=(t~3U+c$1p`0n35DgS&MN5yd_rRf|(3eS5Bh8s;neb1h^ zK;C-)Q>CUOKK0_8W$3mC%9LrdWWU{ZFfOumS$Tv{XA3&ezhKzT=$6Wr%J=V=`np>G zB^!~akSZ`A&o?RDsR$WyD`_Qu`Ssa_@1}8FL zdzr#6@97^%&Gff@i_>diwwRC?O_M=0QwEvEa6(+pNoV6UEH{a^H=H!*#N36J^iEig zWnmi5xZf3LOTYc$XYubac<{rsZdsMo%v>%li<=}Xuean69wm80Mo8YEA(GRlpJd^~ zHLI`)j=BWf8~4%py!D1i!SIoiQP@WcvPz|5@?xo)y-aEPy;+)=6;2CxDTIwzxDJV|JT{g++-TVHz(AXV3&SNK(TIL!4>JLb5VL zI^$!IX~s9CTM3>xSu7$C%MWoQ2;+#eY=b6;LFO@>FvOYW1C1#c@R!0tbMgL_=beb{ z@Oo$}KblUz^cI=rH#9 zFlY7Wc81Kw>G8X;F`PhF$VLl}tnAqxUG1=k*Ijgq9Ch%XQMHwoSIR*rULt>d_V3Yf z+tS%s!#!obv@UIyoZ{Y+({GUEly-;X&yd!}I%%#%OM3Y-w33%dFH1*sqUVf6-}r8Wx%HMLg-Eo3NP)#z*+z zjVBWgH{M|!@lZzKG}NMulGUT9q{G3tq2=3B zT_w%sXicM6kdC=&@#6T4adgsUaMHXgDy5~SMit1!_N0X+y(O!vRT}1&V=wJGI8rY> z6eC6jEry83SwyIFh)3!8`02ZeG|l*Thc+~cgGQMgCpSS1QV?IGg!V^mkfLf$8ze3B z8U(Hro;*~Xe2j*4s-a0w7z>(A8Az|y0H}D3=G&#UGH%B5PF;OVb0rU(m&65=h0w|| zxCZs7f8~;c!UFfzxW(vEa`uUbN1}}#H%Tu4`Th7(Y{ty)U=oAL7Rie&8t1dRN^{SE zbk*VL%LAyG4&)W0Jz@(0i4is6;Q*oZt zS;DE7bYktT+k5M#q+w+%daJFr(nB;%s!u-oL@vJg;;02;@hO<9fy5fH4?bHZzzJmi z`VW+)z4w&4E&ZhOtG`Nf%`)}LvAnb!49Py*^ErO`a=tm}thiJd}@W z7<9O*!$==iu0O}3dD>M&)l%JO!g3mPdtcfeA&ofQHVP-of(aQ6H({8dctijq&7|b* zeq8k<51BJ){-6LCoAlS`u$+#@B7KfWU!N|#zH=Zyr;xHxRRZzWr2cH(W^)r0v}*Zf zc-a2C$<*nyfimgq8S;l`{w7bo@S3*TJ*m{D623Ax9*5yse z_l4J@jGxG?#rd+{`2UmQjSrNp;=YbWf&s;+z1ko`hD7@m`3~k&0<0TGYnmrwr4{9& z<%}{hnQYrFH<4f8e6ieg_wVI`z4ns*ciK@(OVMhE4Q*>ck5C=*V{K|18Z>R_6>P7OC~nb<%7p9N1mD59)>T zGPJcHi3^3C4UP=5DajI0R(t(vL$iAfm)dD>O3s!?Y7bXvKa;eesT>LbP&@0Pv$?Ua z231cZK}*Ps+8QJq-h1ejVC&0?$^=C{v}l7qk;E>5y~ZW*&&RC4)fF@nE3+231j=~wL{aK)f^RB%^CZk2GGYpgP z*P=stJ+5*AZwDE{c>HWMa;V&i$rv*}dP)9x-_5f1<{N35{FyO(UZi)T{u#L?aMA}! z%hVU8dgh0z-NvF?>pBPxmyIKq;%r{FnuiV@8VSfvO(sp6q%G+vGKHjSpo1EqbG8$* zxKe;!-mQJNkr`QA=$JH5#Ztqf#=;vR;INQt9|;x5uR)86L2yvp#4Oy@1WS`}bZ;pd z3OzAbXV`iJ<@Z-zDEn@^we}8WkeHKJYtX{2uEcmVx~g%uL2E6#t8sKL43ofcI!8@s z&E-_wV1hW0`{hKiW-LF%xdg5Cb>LAWEy&wahpP!k?;@Qt6t<>I!@_c@T7ofal*u05 zm_80y-Dy}msedlUqGjPHYCCy|Q5d36W_o_th3WaSR4UCv2sOk*-YJ<+rRV3)VfF=J z6V4PW4IhZ*^5u{RnJ}Onhm*$2(iC^Ebf{E3Fb48H|CA$iOSrOf9yF0W)MEUWlc5;c zO6}3DG$Q0qH4TkTvhQx#Vop`13|^qMtv{sVsV@cCue$}^I&4YbaPg^f-nDmXOST$g zASYaKgS`3NL)yi|S>UX&j_Ow*4mFL`ak|-aj;h#KHTBNMQimfC+C#6I*h%b(Xa6CU zJj6ra0edD6@|UbLu)k~}#-yvhcmjR(`Ium_g??E_^$40&Sa&tR=Rl*%rB^fnTnBy2 zEw||MtJ>OHsjjZpO-$Z*s06?EvSkg9n=Sdtz3G@2%4N_m}rjG1@%^) zy$l`cpIO{r(y&gucF|PHT`#u#A4Q2Aw8^EAoX8Xv5`#p2oYy2VndvSrP~Fp#W2~}V zdICFjjJi;qOq7w!Wm(m+AgehTl5DZ@NO}F4`yC1Qc60gH zGr#fl$Ur8MjIKf>IaLNv!yE|$Cnm2B{V6hc`C|8TDK;wEV8l?ln!iw%0?-~a?cFhx3PuH7O`F}0Ax(gD+^Xq zGFr$?jt7ww_8s6AV!x6_%DIES4?e{jgb znbEa#Ka=JG8^`_qMW4zv+RkuOFtDP`_%W!6N`M4HT$<*g!!?VmVG5~q1}VMAmA50a z2jX-J9R!Jj0A5Ptp`-V4n&M(SSIww00W;{hfsPTpn8l!vW}GDxAs$6gAEgHbu4x`R z+$BfikRJc7zB;QvXEC}-V`zOrn2w53D7`X7F_%-iNh_vq8%8z$_KIL=Di0auzoY zDwoahPz{brTX_2Er{&>?AJ$fO>bY61q~MBbfbSGr&>TWxYnmT_O+7c0x#{bpyZi50 zLyYcWjt8@66^smqxTQOL;Wp5*_s<|@;MoW?Cp8O}N@i`NI%jR=q6f4UtzEsmlZF#R zVJo|Qi@PH_?Fxs(Z*Ij>GG7Mb(jiAa#=;vhH)q-W#rPm-QN3I}#q1sRVle$eokN`I zS>4u?^s-6qh6p1E?U>_Na}qcQWKiv_8eA<=4Yj&Q#1SeTgsHCn93*bC6Ixz_A#W8L5lkcQs2v}r z2QSmx;Hv0P9vOJ?b6`w4Tajl-pN>N{HXJd;)R1d_^;;YjIT6MbFEn3D$Pvm&TxIc) zxQlUcP3fQ8djrWEu%%SJgKqh<`H+cn@gbzbx~>7f`({iz7R~X)v(7q8Hr_Zki=VCP zPd{~s%BP;&x_)9mjPm(T@xzZ35jh^stESiXvJAU(RZn{ZjuqC2jTIO{gCTC|iIDFf zCgtHC)P)7A6}bq;Z+XO#@V9?WH!&5c>)7>2UHy><>74SK4L zif{2Cyw%jlcX`e|P#6 z%O;&w5*>t9oRiDSBQKLd`35?h-Uc15E8pqt_^v7a`Ova4dr`Ln^_!SnbHT~c@`6Tu zHJ)_Q&t>WI<^|++}@+4@|`=Ru5uavR$}!Z z|5Q_VYr6*27{HiXS2wqEkg9>s)Bs!3I>8lAn(q{^ zMcsCm+NyG?n>SA1CB9D1UJ^R426<=io1=-@TG}37Y>ZScEtl*jY$yWz-o`OvUYxCs zG2=G$D6+A#+AlVO^kNxUTaAld?YZ!kt(CR1WWf?|;}EHV(0y1KjOk7L+g|@PjIC!D z^~2^VQ+*w|3To$|Sre+D)*P9U&@PtlG7@8@84Ph38!v==5Kg#`_*uSWkaD;dp@SiA z50quZD#CHGIOFNQW}NKG6FS85JALh*auV-I-caWdx3V-VbE{m8?Y#b!lYw2`dAT@T z9c!d{X6hB^9*>v9rOlbQP|m*Uc6>Ctm&SCBmFLPC3Q zuG-h5Ns@x=ss`Nqk1uvOX|8Eyt7q`w!Mf9!jU&#UJzGva^;9)PsgJ95^`w4S<=T5? zv%A?tQ&iGjX5{TCwR1kfZ0K@tf1KlWw#ebMi^(t60n?nVYCoGGf6!3==pOZ{@H8Nbi(4-1sL#lcALl#RoceS0hH#PutY|e7huLB4%OIxt4-y29LHwiwYNT$J02Q? z7sTlp)`96y$cqXaROMGef6|9_;&4}2CX3@8d;T1c&dko2@^|i*#`@am{V@6zTsJkq zXUA?yws>erxrxb+JMJhaVf#-Due|b#+3aJq&4X9czLbtkf&Rr<;(Uwzyu@-ae@9#Rf}Z#=TFJJ`3q!1 zd8tfXmLv1$Oqbt)Ytb5AkWJ~zM3v~{0eFm{-N#fsIvK8)f-> z_u~(&{Z^uuHwD*j4bV_b!kqPfD7#^8oAp8&8#P02ivP#KlY{u9#?%@#WB-PQvnb8K* zX|L)9CM(XPykj)#C%mI)i#n~TMHRuIa$FCG8bT&?ip!FwK*v=W@rgGF=Ag z{953kBhQWshH2I;7Dt%uA``|F8bgC*keEM_-trpnFpmn5@RP`IW%QO^eUHCk3%VnD zjou*3Yi+KVs>!d&wAPU_BY#I}>^@rRdu$=!!=3Yd*zm1(Mj{V=9+7yckkXR5x?T;w0VwnoZ(qV z@}`RYVblQs?)mY@Ar!6_C`4EHf~-+eHy>ls{O22ZY|jxV4DrNt5b66@zId5%nG?I9 zTiZBm8;+9~?sf98LgI;+(S-3Xj_UjrD|CkUx3l>1 z;r*uR16@@wr?tzo0{ywG82ZcggIMJPzoac_S3xL@|5+%@C15wIhs4#{c^KlZvYgXa zoRzA=J7ex12Sg`dszB)Z8q~6w?1MR0gX$Deb_6LKc)5J67M8`#28p{s9?AZZSID2T z)Xo_z%cX}jz$x>{wSfZ%>alA)$h^4EQ2C}{SNXbWh*VB^N$RoCT=P?|P`2-=zwsi> zfTQDhbP-;9?lRW5Pfxk_f|IcIo!d`*$+G2g^2N7cGaVkPK|M`x2N~ydbC7uAzhwF| zuA}Fp{Gg}tq0SZ;>W|}Oq;P|MrL}CDEdS(>DJSjU!>$16r0pOCE|KATeG}G3)4(z; z#$LrdRHNPwYv~F?r)22zHDFGfOZxcXSJ!PboFp|{+IaF69JpT8g18046VhpeIL}5f z2`nvgsBqfYlmxCCjbQI&&B3;V$n*#mlriNs9HiVgOT2*GicN4pwS>?SGj!bbx zg%(s+!=g#Dq-bm5=J4jAX~a0Oh9?9OZ|?mgnUJ-URLy)}>gIkP$OArJMklk$?&Mp! z{!Sm&84T!*d8j$pM)yD~daF%0l(UXM)Wnq!KN~OC-}MM~wBw87UM8&*kE?FP+v#m} zK~^#S^;v5SF(->bt#cn2@zaOlW%hcT{u!A$I9%fZX`cV7lz-_C*I;R>ux@LBhSbKR z+1laKD|T~p(-I5szyH2mb=6gBIP`hZ#10B==-zyuB=0wE&lAh9-+g}>tq%9=c z7n%>o5jxyK&LL_>>$L&fc+!Ek0B>z%Yt}hwcuMsfw5MzO(v~e`XhoO0JG`hxG9e!h zw1jMw2lfw_${#@cyD`W*NTxsITbmof0;#y#J7}=x){!iL!C;6pP6hCl$%3>nGQ~d% z$P2#_B(8-TBn=*-3QVtwOUv?5apw0SS$>rPdVhxCBAkHL_{|%bZ1-BKCCy6h0)g)Yoiw{7`ChZdbJS5s$*#NZ8cFxS z0}sdxFT9{%_WCy!rKFHl4J5AtI%pe@rp>I*>Lbn7i`8a08{%v}fB8>sm526@5%SVC z&{dejLMM!gSuJf`mrN%OXUPd)Y{}ZCEo$7a3?|a^o_WG}!-KR|lY|AXz7v%>dfGDC*(ha)SrzfHq7|};BeQErO_b7AF9jlBk0%|M~KkoafP_oO7+mC zyWBi$gMomw{?_r*geWE!M_S9{agtil9L9@sSV)E^#D!Jnbo7~xjX_2SIKTGFvxn;L zIQ$CatXIBp418&bvwM);frLx3bYiIow2T@x z%G(N!PW_~6pu-xVjiPPjdsUI&3oT!)^Hr;cEN0xifd;ACRYC^RIh;l<29SnKJEwj>EbMk44DBqRr zbS2Ih=4mRQE7E0^X9}Fc7vDpSx+YA?S}2>reJSXbaw~(t*ZW{ zS3SrB7Xu;X;GrLRI8=jYrS|RJ1BYsyrpvKNQ;BtXCti57RN~-q{TgIGCXkPZ#8;%h zrmmsqpOM!M6UX+Ix-b5OU4c0J+|7cgXsNL7YCzvByytf2Z@+&1^iU0Mb~&TowTWve+V^Tc9OO-Jv$Gr-$vbIO81e10tQtVHBP_?6E1ciH=|3t8Lic( zR&Kl5gqhr2?!?{0;2d$U7nfXlJleucF;hLc3)V5wm2|Ecc-PN5L!^z);NH)ZG*_!|WKmbWZK~%~R%0xW4TIMJ?_{PWd7 z*+*lPm_kxDkfa9qUa^-k9o^Dt8QGD&AgS6ApT{g6VTdzX?Q9-e(44i6i(}GloFvAs zXP6|ui+ZtZ*n7mNtD3D~rAyX*%{0519@2)m&&x8kT&z_FPtNygL64`8=URNK`f&6{o7fxcnWcSV0%cL=#E(WeN6{twk41&E2`M07&ZMYNg5o-C=e&b{=c4Df9AmUVoF! zq@{es3H3smx)mrVjTPlf&(ab(|KuZN#Lz*Jc$eLBzkE5#Ilq{$#*t)JO2_qQ%#6}< z>vJcE@A3S3%v%27ZKdYDd!%ORoRrbG%3g=i-evY2ab`E2^u-rnEF(vbjHJ5Yf(zt} zFTPOY6-B0yR1GAl0on&Tx!jtWl3Ca%qDZtSOt3iO<_dVaUnVEV;_T%*bWztfU?~!u zbQ3l$aTB)Cy^UGb(FF(U#)v(POJg3p*l?i(W?qBn<__rH#wRYFv$mUY@dJV7H^4cU z*Vf3A>KYwtOh<#swqbU&WY~L(IB&t&x z5#v8X=j~%qS*k|^Yx5YOQ=Bl#<%DxQll?mJ!@N|}g3DnvAulT{j09^(^wqEd0XdGf9PYSp)pww4KH{$AVjba=rxv*F_(5%rUYYhqdr|r9!F( zlGT7Y=ccM9x>-?Hw^-vr-3Kzo?2&gLo{?DqLatunyqn_T1EdD$KXYc z3mTMs7q!ou&^h3ChiF8tXF6rr8XmF{%=otLKFKSVhL$tRw3c%-#;5tAfDMeLXi#tI z-mL_Vm79NdJ|ybTI%+E#tPMRnX@+(VT4$iClFmJ_L>6iV8`Rt@`EhMUR=WE~a*JC@ z)^^F3&!FSw$U1t!CUj`)nZHAQOgX^T`YMsU9$Hy+yu3OS%t@PJ<9@W#A4jic^v53F zucIQZkd<>YT?s?na;n~ti$_zkC7sVnV;GMfiP_s%oc%K+!wm2Fa?Uk(Nt1PTBgLdJ zsKj8^-*;L@7D#AK;1tysIvC=X z?kp}=%iula#oZK(dUcn|I&>+cTe+Dtr+s&GGh5TJZ@T*_E|*=#oyCksFUx**GtVZi207lii@{6v9wv zw(47|7m1V(iSs#;gN~dKH>*a)nL{#|Sn<4fg*zDHG1}0qdrNPYAGeyv!{qrxPL3B_ zb`}gH3~@_mWw~IfD3j-ebr4Kf!q&!WNylvC&U9}lvSWUJ0ahwNpf^70$+$$gLi1s{bSj~j;i zhj_#!0kN7S4DlFQC?RmY^$;LYe?0-KsJB$VdpCCGsYrEq$0~B2VK%>lEoTnh?6uck za{TefNAkS%(o1sp-FK&+Ra7G>#A<*Jh~u@j<%=bA(OC47ZHB*jvlO5*lrY3S1Lwfp z4V_*thVpWIb(5U##c%=@ zVZJ7|p5dSy(Q@XQ)U|NXOEBxYI-?Ey1{7h!ZnpYN>Sy-aa6LC2S}hbQ|_LWj8lj8g%Sk_o#Uo=QKs0KDM>0VkWryYBsY_`$*k+`?~`VaEn zr(da)CZ84RzmA@N{?HvID?3*z-}|-H*STZd!avkVnG~|tYJdiVGrMi4v9r%UTejFD zwmA;ZlYaNzch%6XHK9@kb@>{g!)t^iYnb?~WMi&LW*(Lz(H5`Vv^g4TNLmEiwk*7d zdLF^3AyVDaAa(T(+AGLoDCwl>MCo|xba}0GMz%U-2;K@HvO zb-0#RRqMyXa2#4xfHM-f%OhL)8oYj{D`AKy)4!=~j@0B1*4g0vd$VEH_70khx*!~x zr~~mZ#G?WzjjuqW_I2RC7vM6Grg5Y(I)lt$c$Dt){6Blw0hncVwNDa4_97&Z03x7* zC=T3K#XYOG+SV=ZZPmJPuiBy3T3g%xcKuzpwXJ{uYPG+uo4RLJ6qFq{A$te-pXc0r z-tT^2zK{SR0sGzL{myysIQO1&?s)HJs4gb->m8M8lIcbOvA8X8ER7^#?Y8C_p9bP< zI^3;GnYr;_Gd|j012)pO1&gPU4H-5{78Yas6;^h4HdXh)e;^B=HjOyOic`As5z7FK zGYhNE3$UVmXi2f$dga+@tP3?LuSq-S>bqsxN;hxV9sjBq#1rZ~#LYc1J$<2n(#YH9 zARM2EyI)`a0TyDkrYe3F^uYeG!mvh~i^lG}^G;nn##C|0@AT78m!(UWrgSy1YT9gK zXXDI#(YW9($*7ryg`j()^7ZvPVTgB^>cCGr8m+ZjM_t{4 z{m|^$vZ3aqM{sha&>jw7D{c)i-a|XX3GMM=*)VtD{BYNEZpVQn9a4v(?WXJuDcX9t z3?DTdJrk4YoO`ideB(5uEruG3~}#ljfHPO z!mos|@$|kEXe6WO32KWO3#Ayn)0vF<_|)D9QJh{7cE%@j_3J8G8dQKh>dYT^HfP>q z9fq;+gljw7WNk*a_JHk6CUoyD&batB#GW+wm2!x%V8jlxICnQ`ocRxAhIWuLIWXdo zf8ffihsL)-!)~~E zODhU?00TU#(0xT>?|~nV_Hm9kHyfP~?pPT&3X6C%r5cw5wXgy8!_RD>IrPmT@2G*c z#bQ{U9bHQq*`ZN3oX{RL(AqeI4!2Tq&bbvI0<|qIvaq&JI#9osj~K2;F6RKxz+5z| zW>-xn@R5X{kpIpm+(L-MRtA-Bs~^GPv}#-fO{Ma14(&nfi>ok^YB&Tq-I5q>{9sQQ z;*r;oMuxt z-b}1`W3VOlECKo^Mx8Htp;e{7QNqO$M z=j68AZqpCmu*^jsQpjgs0@OR_om-n5rSLG4^LOUF? zg*q*Z^=Lt!1GT&vp3B-Vh(llAQSZ9kq)infX9Zc*Ma+aX@+Yj z51`YEJM3H0=<_Jg8eG;fuew$Sk0_Vz_Lzh+P$GHQR-1)%MO;|D#-vBK#!n)2e%f*8 zLsQ;Zj1IU1vDjXiiDBM`tGGGu>}aE5(3o9=W(`|+&{g&vZjDv^@eKg!&=@_#)zaIW zo%uC4Kue49Nhh^4KWpXrSKrZ>+4C34Hy`>HaHhxH;oZ}C;+g64=O>?2hd2NIv8=4A zjp(z1UYqVF{bM0F4r5Q6jWieAum@6Ieuy+K&uF9{AZGmmEF8A`g5QAT$G@D@4Z)^eDX>C;7v8sky!pC5bB#B(XFj5 z((uX;G@s;*+8w+nsms)kVxgNE!_7Y{AL4{s5n--(kS@sLyzlIqI{C{xA4n_Q>d>=o zMK4r`qA|42ZCGq%+1RqN=0^h`uF8&vv3-L~J9S50XpM6lM(wy5tPQ>9HVkX8YHF2v z=s6co*h)rD8mC8l=Hq~b7YOMEW6%r81m%c0p*gz{wi$KrM_ZP*)p-Q7PlZ*s_fyGwcL5Kj|>DL55oO}aar#Cn2}=Z!!(bisK%R%}Dc zDx|99P~12DhIBS#KDv+i4|(P0{%r1#=Atp2*?q&MXGjIk?WSR2x^UXJW$wZyPLxc5 z%GARU*Tf}DU}zWOCZtg3A@1G7AR%tS0n-QPO2PI=%j$X0O6?4H16Hah9SCRuQ08ME zZOD_xIBmm4V!YLjhWYC3v(J_}bLJ>M0Le>5IY1I%xokm0S^w&fWl%>Qrl}4_&0*!9 zi87oJkCu54LtOoNKXEwf1GHF~$bU~g_ljI`*Z1ZBKAb6ket(8EXQs(A92ioA`QcVH ztn5X%ciwmRzW0>4QAcc-|1C9dpk-JuJIyoR+C@ zc$+@s8fZqeMu)g<4myD#Lnv`Y;u{qeFzgMg7keR0@`R0m;_zlr!zcbE(xAJ=;qWWZ zJ4r@Wy3=a7Z|(YT{4lPfAde6H>`|GAhKYuAv-jS_yKe(tIah&OQX;{t&yD{fJ?njN zp-o}sc+5vn#{TGMFz?OzXqY(ujkiR;?VSP4Pl(6dsc`K@7Z%7?m@~QcsiERn$0OkSB2dG92YBj=xMOd}y0V70E-id3y0X(68U>w})bjJff zldJFgfi}WT7Mtgdw%p=dzDMhJV7RG zvlZS+p9Yyp8)#;(zBg>7t#ad>^y=YzN&VUaX`1)4ZBUrWf?#a@u4Puzbt|c{nG%^Uga9kN-6D=9_QoB^#_%Q+hI~eD)x9 z%bs%w8|QyLC>f1an3g(PFU8QJi|T|>ar_!o8@6t=A;hEk#6!z7A#B2eu+y<_=)s@= zDH0?{2XDOMESa?XB*`0DE;*PVUY(aMwMb8OOS3d$4!H%x-5nghMkC#U^zvy(x;pT5 zq7LiiI~#B`*v;sDHz4e?`X*V0x##pzrBbr(Xel2vQc5xUScHxa&zj&|I6HTox{Qof zorHLlR}U3W;J>qJ1x8P6WNrDbNqf(vt37BIHZ$%nN}NXu6ADLhFk%o0JRn>Ii8GKv zQt-GY6?h_~fAd3}Urz>!hsLkEj;9X~p*N48aO4!(Z;zd#&`&;##?PA%X3o|X-h3@D zEs>ippX$@?J&*>JMmdCX@veOW*g$irn?r9tguER+XWdYF-peXO`tqlhcXk-@%j#1uJV3bs9X^YcIaO&)#xSvPMO$bG=|pTK`6 z&hgFL=3r^Y;**9&AExFxO@RXhSvgrg=1}*EC!Q!r9fdUm-aP#9!}7b|{Vt^^6Ub)_ zvVP#x)>J1o&wpFeTWj$VeWY;hh*mz@uAmYk3~^sEx%$P?AkOC7pf%<6$`62ri?Kb!D!}tWsnc{3JbuzsamqaP`T@x^K_YfWMSziYSLDW!^p+{~CbqUM3kC*dL7yTZHqv zxsc+rsb7*$=2q!3p!9pi?GNacl{+AV^Rg5g&3?eGu(-I99{j0$Ox*ge)-I9$-TAZh z@hx;#9F~M?0{!tl8ULBZBe4habJFzk1Cn;YG|4N+0SNfuNtuC>04o*FM>8LDTM94L zm^Evb@EA}U9%{ng^^QC4h^!cCdSDWpinUKApmWaXSk%9CpJe8hVmsBL=sBZD8P!XS z4h7s-1FrDL3cS`c$P-rI(B0X^lF3D^?|d*zPP*ha-6Co(lXu)sZn%tVYKCGvX|7JM z&~Hm?yJVpU-8QsXT5B=XjmEj72KOppkyj%+AL$tKPQ$2&aGDtE$Id(~ z3@*u+%)(qL0^R)5A}PcITi$C>fS!6T_F(hGuypv1R$4R7mMGShR)^6}`A*J%XX6U# zs9z>)wmlvPo#g4%XGGKJ0mjWiD^2tqbv_aiX+`W|SUI!8)=JwdZz_b3C615}lWve9 zJWgnZ)u5(H83IOVxH>2$U>d8!INGF(M;kH^ciMiOoc)!fedw-WkDVvT=_elP(&59C zhCWc;rtCp8HRPL>o81_cQM#Qh9W+jwWLlKAINrKhXN`j{mHo+eq$+$vC(BK54B>9gabD4U`G< zF?T3%M2K6hSn1F-H8sh}C!Z`0SZHH;aiG$gins41z{a_G(Z8ksnL8yLr-bE=-5gJ&NGpPP8EFm7A3ij8hr^^wiUme#tA9c{)atAtBY@CPSm`$Gd zP>iitg%}|$!D3i0h|R|qsC-;vQ8Kzx3bz?01tW(^#&DcuR$3_SXmnfhv!yjJOWKNZ zWpzn`WZ>|V!qF8{juqM?CXPdIybS9r3Z(>h8Yx9Xv^G@hFj&M$3h7q~ zu#x8CM?0^3!h{KO%{A9VY53@)kL03@F4Dz)k!1?mVkN+llA3q_EDe}*&KtX*z>&pvHJ5H`R=S#R+ zw-8g5*bRdQnmt_ZR?%ZX(O9$dF$|Y#RFsuUIflB+G4x%u{TM0dgW>97led?Oy?2tz zeJ4rf9uuVkPvtHXq+-HYDX%O?m?2U*tV}AfAgmOpG3uN%rX<6`5)jfVtGf>~48 z{DM%Sdp7{}Lr5fz^NI|luMRQ)jE{xgQ8cRRH5Q3Q%J3dnNN*$^xby188!tZ-E4hE< zi8yd3dd|Dy3{`#(IO8DWc3(hJ$Bh%E;{`S zdI#%(=%4wHM)~fZ|J;$gVoO|`)M5TB?U0)#3rDkB;c)~~zx^Qr%f~j<&4ppSRD%uk zqmMqSN%;Nme=mFOwU=CW*=4CIiT+T^CWwQ*=Jw_~se2haV;Yx8;f}}Rz5;iQutult z1u$QJtu3%RRVy%ca}}6YXDHsjI*Zzl7R(i2e(MAB$p8K`;(`lYufO;-*%yZ{m7r1P zt$pk{XW{rymU(M8xT1PcMh{WA z;lU_$3+Zch?D=!;1v)a;R?yygN$G$NxhsJPm|j`m)#9=de79ySx?9`? zS;-wbS{7#RB#ocGfPswIV!eoOFSn3aOIm1W{2SzAGvie+KgSm*zdSj@aNfzEmmRj< z+Q`Vw_dG0b;qVp~NXjHKhN`SKz5dBFjmtwWzeIBI`6Or5o-$}~j?}&I9jqkfb}>t2 zlBSH1Tc=(W4gT}W(Q3Gv}@`yz} z!;}>r|2-s*RH~gqrj>3U5?5yri7QQm0dq==^fa)C8rpEec%$^-hh)6C)B|xdhMPMZ z^}!f=`_VUI!)f^-e~1lS9hMjsoQzUdOOKq+v~tv{t>TaVyWp zw?WUh@l20~hx`+#9P|&25z;45NK^zH+bJRnnIufbhevkn3F+VoB?TY*Sxw8A_n{U_4#jAz7&?Hc|o|C{wH6_>I)Pc)#S1oGkfcFyG9@#H@p~vSdNk ziYW%^#B%7Nwhtb+@*LU`pYh2&nevsd%FA!XPFLI>9S2TF?l69g3`gV22DKPH=Nugt zM(r_BdN2P3>CQwAG(D(C3~}S+ZpB%d z(~gjTGzN(qVM6m~(EJ$;!_tm_OqW4TBcd<}hdARxIHl9coDC+Y@6ygKMMoaEmrNM9 zm0Wzr7c~wSw~No{xwO*`%HdpjKN-f$VH6u8WW~0)Xo|EfpCui2Jn0ME^Fewd=+o94 z{z>0GPG{uZjsh*fk$G&O$Be4P5Ynk`(>-V}T)bQ^n062LM`MMft|xFYxm38wNt4Ft z(~UTx`7=m=hCA1u^-o?As7L-;kn*wpN}878a*dy)<~k|t0W#D-KQ`@hm~rFA$xS!i zr1_gkTCiY&oORY&sl#RlNI&Zntu6$^^|;l4zgaq$yd#B^j+d;VV^QEyD0Qe2b)-RC zTVxPrSE~mPLmUN~g_=#8LgLR%e^U-Q<%;M4DLNgr&#rR!b?3=eIC!d@JzcD-&cphd z!Gk#$%&rFv;wRL{a6+aF52^XfQ6~gzjPd&Nz|9(iHP!e&MEG9dt?ecA)v~X&1@; zw_dLEsumxGO*~S7^2&FHCf)usjFZDDx%k3k1D#)5Df6;$>(!hWq5qCJT_oe3cI)Dw za#z_J1Z&Kc946w{s}kHFz4xvY<%|=Lh?su*r8nf;Kl+XD?U<|(lc$jgL!CMiQkEbG z|9lK5Gag4@2DCL&)tN1=}4aYgc|OwQoJ52d_koq>z4-0Q0eZY;u^J z(>D9=yRRPqX^D91si)-YU;lb=IF7lb;N~KMPHe@j#rgNO|F}a27gb9Bgrg*5aK090 zUp$di7GkEDg`9=jD?}8DBH6Ob1_|2dE?s56z1vg1uLlbS)oO1l3a`go# zqhT(=ur#jf#V|E{z&yQ+`HMzrc&Ipj3=+}!Jycv1=o1G#oqL%9(>O%#hso)Mbqybe zF&NU)NHijPVoCrtpvV2IhVgXTYZpsP^<1ebo1$f+H@#<4>(W4{VUxL@q{WEwSi)RF zXu)94k?A1r)1zsu6oiC+j!iovdRpL=Y0l$h9G92g_+;;3p zIdJb?A~`Y&Z6G{!G%P%N(71N2zYQ~D zvysN!3x~K*KJF0N9Si8peEUa_$a62hjXc4|lV68);g8e1!BBM;R=q~P9TEh0-pmu< zLL$^1-8>}Tlm62NW#OQN!=!V;zohp4|4sFzorL@S*FHA6_>m1XAKn+uMPg>|zWZ)@ z<{2Du3&rwDnSqx8^}q#wE$B6`eClRtpZ}5+PW(JpX-`6<=!SW<3?ivoG+8!*TYbbA z;%aFsHR$~TME5uELYy~eI2Qz8#@fwUUZ5~{nC*-Z(NSXnV0>r=j>Kd<1(ip z&&1^&#!W+rM_)ZuE5ix#Fxwf_EQoi5Auq-sSyBIhA#Uzr81(>ugzWG(&Ha~D>Ya#pM=G2hp#I@Cf^;Hz4~eUH4~c7d3&S_V(NLywSXTov;{G$% z`{2P3oX1;Iw7*{F-C0+)N%-fNBM+a?LuJaZ)FGIYzW$O^g$09#+a1om?jBjVbh(oU zWduQ)-VM^PiHner$zJ7(a4K^Ti93coG`$=2;Z5hHVK|}UBtn1ohgW{(7fd)p+CF|l z>SsNdYNRRN0Wi!Lb{G#wg!pi%d)P2{PCkcE&p6`@9r8}iM-PC4f+(%Q<08u%j6T); z<8~QbFbw7B7)i%D`1)OrZ*v;rkzqNc@#RtLOW;ONi!3SvLd6l5a1GAFuBmO5FJE+< zJoL-|Aq6#@YP^gG^-UGUz<( zuaDD*C4WZG<7qtRa{zkKOr2G1oR4OuR#;jg^RoBEQ1`!}7A$}Qk5PKzzc2XLI5FIl zk0GRD40n&f9>5!M^A*2XX;$JSwzIDKCa#fdm(@62KGG-S_cEtHufaakztOh%lMUNagQV%hq9nFdrEA�HXOB>YNQ81 zL5PxNin4F4dQYmKyk1r>`!`l-A0PFa-N!XYD%p?)5IDo?iW93pJ-rwwu8u}5bWs`{ zg_~EkSPnbwO5qfmnXy|{%Kf)oB0J-NsWPmlF2{T`7jfm_^h0iSvbqJ4S`ek_UL1Xk z$DpelT{;zKGsEyCrUX46rH|B@PS(SG40mbY5dJ&rmq;_#5-lGx#Z6se0VPLCd(${G zHE|6zEjUcbA#P<-C&0;sdzfJ2HfO5}!aJcpU@n5D5Dyt9(SK;Ccn}7@nVi5lVTA%T zT+^!l87B6)K^cT2T|64Uhs0?N>WhX2rakEXAL4dtV8)1F4VzZJlhmaTm6nCf&`u7_ zK!%4rhxq#N&&rg;OS}x7hnVcL!#KGBThz?F_U;V1`M#gxzEtK7?)~ekPwI^Fw7*cN z$^3_Qp*r80$zFL6}j%Z>r#uxB8l`5!g9*t>^2C;fXss>kk>2VU3_Rx}=ZD z`K(5Vs&UQ>xM^!I938rqPW7<&%reni6X#P2W09C(h&xPtcMNqA2pLw6$+b_shsp@8 z$xwI3+z1W#ew8fjEKbr%d=dqwaUr1jv#^B89{QubUX!+#8`={R(OtSXfjyOIr;Kr^ zduj20(z0rSbk;A5_%Xf=ZVdmNV`7{++#ijjK#Q@L^@Jn#ck}k1zyJE}U*)g=n2yR6 zE4w)~ii{W$$Fp)l;%}#OepGmT0 z8FWte*1Zpmip>_<6c4SG`Pq9&BNiLTn)ZnAzT}^Aa#&mMk47(=i^i_G z@MPWQV3N4(mhZ`|c?;dvD^;Mm#B{+l_2%gh{b%8v;A|X4*fjk+(o&1V=}}RnO#e#2 zU@-IFqXF9FK6lQ%dm{!pr)x>4F^|3mV( zJrp0Y`{D*F*85)Wgm_dB9=i6x*>#7r%+t-vzJ|xwb|>yyzU#qX$@$anm9};qBMiem z%9o#Y0=B&#uZNEe!{rvGSOdcj!eD=bm(qgY?&9NUuEqfv;^sebgbtLDHu7i>Q$pMu z@=ZhN`8H@Y*9AlV+m?SU%?&kDTXvwHPtT824sEYb@0m%78fd<-AgQ9zyrrQLu;FY% z#nqGhN?;Sm#2Q>z!n<<|h3evBB4P?=bY`FmjW2?X8-6tn8YYHYe4)S`&Y@YRn+w3) zBCr??=ppl@@lI${k|m;_A@#xeI(~VvQj@YtSk*mLnijkn@!i+_yUH{Tr;K4fx+qV# zINWl@8M=v|>^I=rwy9TrLmF_IIUBo8sekHP^+nxpg#L4QZg=TuX+%%D6<6b>O#e%O z6%%_8+#k&akX$Tw=bd-zF}F-jeSN(iVnWC~oicqV0m_rZ*{yZUWz|a$O5^nValysb zQn39ol37p|)pM?HjMcwwj6iZzS=04VZNCQ#CT8$_a)P49so^)VO$nJQsL4{P6D<8uh!myAop(nQIEAT-0d9tVX$;1{`3SrCV8Uq}155{TOAp_i9Wfe5 zJuMxl(j>Y1>Z_wPeDJ{sdR8~L-=s!5BC%{R!m>j- zwzpz+HZHeV_0;u}-q9#UlTMWEvI)8>nu~MokM%BbSO22y>4HoTZE>bSdh&R(b2@my z(c)z*<*2FG$zT5OrHF?S!-wKzuuFyW&b-5!!`j98Xv;!rVI5Ra1Z(7pKjf(wy7-v*6ugDL~1=V&=uAYQd|@h9Vd4NhHa_~bcRn1vJ9D#i;h zL1G7MUFU7PlqUa8+CZvw0-6#BmCgar`VzB>^Lr=#Xytf>G*nS!xe;$3}Ir-(Eo{m;^ zYaXE9`%wRu7qn22HpzSF@b%Mw?#R6)gQG#FXVMFB(oC66CjsUu`{3kx-CSF7+;PXr zi6@>IMf%7ikI2t|{`1s)bR>}tL)b9$I+&OG;j zIJR-JaW2ARE3ekg3!|7d{i=l4#1;#Hi4iBac+g0kNSX!{Z&iFFE4Q>?wpvpKiuyBkkyLlIC zAuiUq{H!m^c4J3HVwv`>|B`oRe4;(+HCVTxar=Bg+|ouE;&fB@9`*0?f`w0nVI0JJ z(mxxA+|heR=_iuvsl|z7lfP)b!UrcSExT0Xs;jONUQuc0YhU}Cy#4musYcoavL28h zoNSnzm(P}!Pv0Re@BUiyaO+#)_#-4EuLSiR3m*t3dst3s@5`$K{`e|aOQXIL!yn>` zpyHx)+)q6FsvL=)^P;6Iqm&;pWe>Ug+H++Tu5&EI#TDG&%zDLci_%!xw!kRekhY6O z>m_L+*h31(t<_MGhQLwZ(0V{6|kAKD84}RTWXh+<8 za?n}OQjPH#GRt^qqWIuR?Nn)AC>D_ZJWY!~5t3K_@#1k^D2Hn|HIow;Ifhf zq;=(N>10wxJe2}cXIKmo~g}YZ7fQS z(gQlgl`d&BEe3%z4);&YCDfos=B!+}B>qX8{-gNdN8@9-Fsu@Tn{PXM!+(3td}&_x zu`DP#RECxf(*~NOp6k16+eaONxCWZGOw3wdn1O9Z_O1wiwDI$hc;bV=&JP*W2c9Dc z6o2mDiwJJ|M@WAZn1>dpLCTMQ;{>ea98E`8JrVT}L%48?kv?rmEXiYWr(+0+hPRIx z&uTG%KTI1dH7cFG=%TWbvLt^m45_?~G(uNi3$9q^IO&IZd7OHK@GInpA>4`7rF z78v+3sA0T+ErcQN=`o1XMf|NTZF2c--@{o6KZ6zGf#DPx7nq)T;^BI1=TN=QF-H#{ z;eE?2vl?Dg7E7DP9i$~nAG~Y&NSwyU)2;u2hQ(vBFZo~1MPZn7oL6v=lnfuEjWb6( znZKiO5?fuH{w z^U;{Mr|hUpeGDh0OAP9p>4uR0L;VA17?+=XI8Re`N?g4-7~OUJH5O%@@ld`{H;@G|a_VrOnmbw$(`W?|p-s#>iLV zW(beGCWWLJBm{o10++&G|bU>Rw)Y5U!=1DO}p$#cT}!2y5~p zT^?kT#cKpHZ&X}ZAQzr;tn4yzn<&a}{NOk8(i`vNoL%mrr}akaA2eKEanST`Fp79x zNHYEt{O45eC~fclQks@RhWx;=FbV!ruYE58>eBKv=c73?bm^s+>S<#Z+wHgCF3&&z zy!H}Px^YsW9CSZ()xU8S=4PwE`}!-Ks=h4JtVHy2EBx7*IQ#{ zLut95`^?!3lunTJ_FGhg?GiEE%qkcDP- z8hg-gk(3SzT1=))LdDtWYAA;EmInRT4m@~$s$bP{ufmgV%$VW~s*A=AETZS}C>Ya1 zLzwI_H$wVw{t44Cim;}mb+enFq<(C{7#r!rvaMus?qoOAjiDx|566$=*>Ses{0Fj7 z3XB(-ju)R7;iNHMtZ~~{&y=Ah=*h!$qG3LN+I_NU8RrdoOzOt!AK`tWBQ(7m)JQi- z{~9v-$;cZjIoN{L{OW_!R9%(otwoaQAH?!A7t^qEfzz%;HMtk)xa4nZ)Xl{>+ZJAC*!V z;&clP)57`g=U;hO4n^ZUXTjpgb^kpl$-OsTgkw7^qQzjmCeob@W|8>{TVH__nv+3T zfn%jh7~)Z6#;c9!(!uN^PXS`IyNR#2{F5SO&^Twh)O6-b^ROwpor>!zn2%XM_m%0= zK$CE%_MrV1p12y{h7$>sJq&Grh+AgqP9gBCLNi*4;rZ@OC|srMX~yC7YjK9S`DHxB zlgEc@2J($HK7@6+>~U#;Hh)%SKCAtda}RNOyEtPc$KZ)gkPn0@8#2L!F0VRZqrK z2$@ogGsMjg^`2ci{qehJq_MEL3vSi5; zZKOGz`=6#4^2DYthrb|!P?NcRwAs9~vM#IbzpuSiX#`r_irGaMUzcw8gX80boB}fAqFUBi{rgq&3Fk(Im05V1-Ovd`) z4DaJp+o17nP-UMj)GBGPLv_==o&($PAkT{fgp@0VxeJc#?7(Wa=e!bfQ7G^laYm!&DhUAkNk7dUw~ zz6}~3gAtvO9%+Wg+#LO$__wsREP7X(YL?2PvZH0l&|x~-!_iF6JKINP7|X`saSgP6 zIjqgd2LD{e9nV0VP)!g9>6gMXs5oE4UP}T!Z6FxVKN^;P>;an8cz27NQ1Ji)4TnV> zP8IkxGLChGcSGDQNn_s}EU7E@I=H}wLxY9G#>(>Couz61t5F5EG5k9oDQB8^dGk<# zEL;PZkK;gx6c@;;#~m!6+kHpp6~EuZg0Ux`dsQo!jn}`W!O~%H-THIBT;8J`Oz$QW zx<|(6pWLnXkPPXRx>x^8S9bp=Gz5_hHUq>yI@udxBhCAw4>;g}C}mtQ_QVrUxO^4a ze)f>4VaA9-9qu$=`S{h+iGxP+#~-Gbzocj7q3-e(or)Ry+(Z=Gg?M-FTI}(| ze@rvg*Eh?_mwg>uss6`8=zivJUU!}xbs!Grz+$kW*yCJ;8}WF40!Q0iPuzPaJsPD; z=xE34e{V-WG>QJnEA0r4Z-eUJ2T-~JPB)s6=MZ6x{R@V18zAipxhx~GT_@#Ld99%Q4YSo zQyI|?mLVRwd%xX(r(SqnZTYkH$YDC@j&@4`06+jqL_t*FyeqDz z<=xGE^0IN+1v(D87z|;V5>1n)%j2|DqprX#y|iR{0n|T!3_8Rd=+V1*sQ!G|h#oCz zM>+u>xsym3UqW~~t6xQgd!3umd!Cthu;i9hYQxMCAJ)Olx6HqNZK4L+2y4YZD90UC zVu?3~5@9G5;)zdU_(cCbx?8FU$$Y}og&w@99sR=?PKZYfwTG^NC$36oAg3gr$Gx_* zRl{tqm4WICnSyh7tk6n_mdWChDbj)yZC1B6Y!v@ac1{*urW83hHTCLm>&ZG=HoRO-zDen;zI>b^o~Ue zZ&O)mlFX;ygLN6KC_8|ORjxIRZm9d(POao#I32Ya0hhHWF|v=G=4V~7YxpQXbV&>8B=aAqP)$E+I6hGID^3{V<{m;e&}n`S zT7zw-Kh#+`)SX*4PL^kFFU|A$r5j&BLY_lBtHdF(m0$b+3yy-4f}CcFDqa^-o_xf$oO?D1kss4k7*Y!MT#Z&4JST z@898z`QNFYbSSWXz}XvNp0-AM=bd+!E3UXAiutX#-Vz=U%K6#Pf*~JVoLKYP&!y(+ zTV)V3WzmkuVFB2-7^&)hyH0FH4KbIK4N@*FQ3gX?EtC9MSVs-JdC-&p(~~dCk!Rf? z%T{u2i!;X_vX9()`597KUMl5!hjWphkD%8?qOt1EB+{opvkP&HKeQvBXe?}Z{zINa zJduvQcSo-Sr!_Xtc}eEv?k~9&JLsG^S(;dP?&NuPsKK7)KIcP6AeE;2}`2HIgYWyA-YxG8rMl+;ub`ik^xo3=%0tpxv#3|;?~!QAL$4)Z^U)kaWM#-pHI6>|=qUV8fBIAT?Qeg( zMeBuSq64=d)qe1VE&y9wH(!c&I7V_tU;!8xscHio>m<2GhQIF5HNi3WME~~jNf_dZ z{*t}(qx!y|JSt~hbC2*yUmEV#xcIbV<$_a>!NDW>=sEKMl6>4Bk82`zF&JZsrDuK9 z8-`7WuP?oj=MYauyF0ti#^uuZ>GLu>dv6&$e8;G9=BOp}=%#F(Df74uTiQ6I!WSLl z<`yIO1xAm(G4yRJqJP7ka2z$WOXg0yctJTE_nVk0Hh_ndtDcrYg_V+l9$_-GF8-7GPi9A1u@{|#LRwgu zi*sK%Pza~~`-^`|cW3h|X+YyVCup{d>}mifeQKSW94Gi+*JV7|urzA3j`eyX`jJ zibcja5B-&|d_`(&YquD^us&(QJoCzb-YpHdn>?p{0!{!sEY>)Kuc&-@oV?O#S@6)x zg+VQg>Pd@=W_i+K_KxTQAs$(ngNHn}lV=$mea=mBjdMl$5V`k;3uK>Nx7TAk%Swv1 zoLl2edh9;Y7$mL@s)r#?HySH?gqBW&noigoRR2Bzc{aWcl7KpUNSt8}Djob7RD>_V zqv7#R*x6Epd!3(^d6~P&pvv9!LKm)y;GDA!YY#{RjkMtyYV^3cJ&Jpwky_({Zp0&F zFWy7QpyKdokYQAq9;%QHC&WV~8Z?E%T8+^$NXtZeOq|+vlOBc*A9~}v_pUF*X~`aR z`r6IsvJ57xC&P~`_M&a5yS@xV-HofH4XY1Bz71a&|0)AUNLc`=Tmnd^(2_S_@w6ar zz2a4J+ipElt~%#L^~Rr1<}H%1-tmwhkpNlH)pQsnIYdG`!Vp(~BmzU+%!caU!V_wI zfXW|U4AQ~y+))^^Z>yEMw|~1ueXxamZmGECX&dSuKYqMid+oImJ2Pj_l#4F9XtUP~ zKkPbM8)VgMzmVF0V4fL0u)>MQ$l%h^;Dxy`$Z*VfmiY{tp8&eXkiYH^OJ4PlkPm4Z zBpw?jayZ}Z;fKSASOj*^sn^JB?|zK1&g_U$uX}GeU$)zN1h!EX;WjGFHy7k!VJfy! zq0FaY9f~EFu@fic9}BrbE%5wmW}zia_62Rc8bQ)9JK_nC($*MUJR8G5<)GmK*R-JL zJnI>mn}MyP6?{wIKvxMxh3ceWmU4!NEhbkNGXQF z4%&NXIq}H-BS}5>_v!M(-~1Josk=K`)8loJj2`25`Jk)r>84>kBp&KK#5JD4Uz8vq zN+XH>jc-i@WSWQVSDg!A(Usj?Jd-m0Cjr(=cK$q%0zLH5L*=BCPKpxn=%bIygAYEq zSuW;dK4gBWoB6D)_+Kn+TlFbcW*?1_nH^ErqMr3-$mL($XRrC#4*6l8=0Z4^|9h-I zeI7UeAm+mQ{B3!iFnrdm?v$rrd_xX9^LkmZczGn(LqE5x+;QcZGGh2poDNomqdRe? zHp+JPV2m!Zu#+y*=V}GgWXukRxVe*8{Uh{z8#KNR8s7#p4dP>IWpQfo6QH;Pndu4Kz>EV%DIr`A|XzRa)wRAB8Sk5Kms6 zBD^Zd!4Nn9jDmP%NY@)1PB2Jo^K0Show!d9y}0x$*42NQh7eb}IO;RqH_$%#76GU~ zW+V>9=t@JJI8-~RQd+9s=_0R?=Me8!o;~QFd4fG@4t?>?SI$+Pd-743dmbIh?$+=A zQeMXwx?ZY5Tc0OX=6sEOGP}79Kp5iY#rzWT>8<`PJfYG+8Y~?K6X|7O;bi`}1El%g z-%8W6S(|;2E#z@a$EkbfYn#4dBh6m(q)C$^*0^8#)mQ!DIFaqZB08MCVz#V!>SlML z1r{B0pS>HQWZIBpH3Rw@%Wy(G%$5cpn|v7ukTf*8gPADrGfy~FzIN`Hqzt|0VVJVw?Nr=SnWb~DIBhaW8{>43OOs0^ z8*1|Bpu@?r=b2#)Ck*if+R6(_lW&84z`yce4`4Vc=pNp?fGW;)L$Y$)FRtv$HQ05+kle6$fb#=^s?oiI9K% zkBpNhwG>&!JkX=3YmfwK4C;#!bpACC__x0>4ZIWDde9nl3;LlBId{js(L80WX6O`Y zty?S|^>(Oz&-^>FIej=eLrT5SKgGnmUb*$Eq%coPag)sr7oDub-gIlnIo)Snd#|jl zse>w^OGL!DLfM->sB7Y?bJM#)5~G_M$B)xBa3e@J;*l}`k%b}_Uzi3LES~?2T+Ho{ z+*KO?^`Nx*GrQv{rQSA`1emA84^KAIx88cIeuOg4_V#u;<&;xo#flZ1#m7@;dyCY( z`5UQy_HM~Uz9|@ggbXSeruAe@MwTCa{#6IFL)Hb)G=%OkoY3WdgA7GK2E(vQgg;Ro ziX4x!$BQ?vy6eYs-M4Y3Ja^@lRPOeGl1lZus;$nVqq-tSNVM&I8>zQxGv} zzJBUBu;47x8&W<>R*32qgyHJ^Dt9PuDR<&}=~g$HO>={U(O9458@!db*v zS-d4JA`JQ75^>f~{P1Lth7I)Cv18??n{JBo)xw1f^*(7fJOgd&Ud8@@3}+XPKN9D> zZ|~EHJR6tcm`dg^Glmo5VP-c7TmXgXUXy&4Vlnhva5g&hUsG6F=e3P%+WiPO!X;`B>9d*DZL!X*CTP5l$9VPG0U z!_~rPK;Wbk8N&%f+8DP}O^siyM;Xil8u9DWeO6uPIQ|hEK z6J$R@>YpE;dU75r(S7&bS9tuVnWvwATCb~QBOT={bKFFb@g%NG$Qn9UaxtWxo|PLb zKgenH%UY3th7`aC2a@*Lup zev;5IL}-lHKIA`+KBDNTUn-5WUzAS=;~ai8&J`6n84XvRGJg$p<4h4GZJ?=)j&xj@ zvAWgET&pWc&o5XY^j}g2sT*bngT$SU59{A)XAk`nC$w+|U3j`X;{*#^O9Sa>=I}7Y zeH@AOEgpl3>9x2C(=m^0J?KQGTADU@NIlxQ#{43z9(s_pRnL>vO)HYhJLI3Z>BjW6 zKKf_j;z?sVA6<~Emuj4j271q($46pg1O3d)?`SVHjGKDVKj=DwUhjRO|0n@KEMCTA zI3b_z_@`WXh)Kune@Ii+zf+BLD9}xeQ}!Gk;)kc*AAQzYXUX1s?;XW__uY338|uVH zGxE@f224O)O)8G0~4n1R<%$mChF3#+WkLPbsJ6FcwY;E?OOH1C!+mK}=X@8XD+MoFGFT;GGZEU{F;8B}9-4!JjHbu^3J}rzme{^i+w?&V^$p zie<1+IEB0j`cuCUfoPO@ z0Lj8aTCUb+Ug8$j&82as7`tttS7-3JPc%^Y>M;c|3lJxC4@m?Pwh2f=#RC|G?UtVA zZ-xQc_`HsI{THW)bPnx^Gc5)+U11u+w81Bh^XnjS$E%0jMnHo`dP_DeWL#Xs9tXDz zq+@kr)7o5V#of{C^cX<5Xe=L>YLv*di%-@CCzNY-ZM~d*-FKv= zmD4HAOELXx`k)5Y%_h*l>DB1R>EHM^n1uhq#iL|!QH3Cr=2D(EiK4D%vXJ9_yNEn^!mB4%c>`*VYf#!R`^eekHF$I z2v??=MjJlh%h|;KWx({?0KVo5=-rk_{7TBpDMnu2f zs2|`FZSYEbQ}S;z(Iog69)a9gg*s|clj5Qs7U%+Er^$$k9hVypIh8!fVD?X9VrkY3&#&-l& z(o7#Em|ViW)j#)pvKP(y=n{OPPo6kVE<(j%=FRs%kz2m|D_u0kuYKys=_#f|)4Rcq z(Z9t@*q#23Z-Y(-CJRF5m9>&PVi%FdMN;?C)2Tb8L&5d~ZVMqfU&xQo$&)9`<(FR` z#r)c9ugUe-U$2d6Uo7Thp5sSYds_<*9Qki)dGlec>YprmT$6x#+ISOGZoU9%p4DuF z{0&;nKL&}bF^Cs+^Dqo+1yUPp&?Dr-4^4ymaf`xS-_R&$Tyw8H@Qc_T&YWAm`)gC> zxI_2BLa-t!#jtiErlHt456a|9Z4g6PqhTMgOrNtOM&r}0ka)sFGPEPq^f{co7~ckA zLC=i9K}8)N`44E2UQhg+EC|&vXtyqXUmBNuB(n;Rkirq$Yvas0=gqC>Y(hxPRcfza z5I9YY+pH*H6Hp?gKiXNto(L&yrA49R;h#ap9d%2Kmm*#xiN>554~;?Mkh>Z}KAIK} z&7VPfbQBy6ak@DZ+5*Qyv9*u^X@qpl_m%{$GUHj@h2^7VarVwQz~l`qqK?g*B}&Ng z?07dBgm8WI@9vMrAtq>~`K5mRVf)Cz`|KQv@F%~2N*;UqW$8o(s9NxOi+RD~q-U2V z7i>fIA4N)f;Y;*JThsZv`A5wxd0Xu%?KA!)Evpu7vLV&DWTsvs30S_?gLyDSdE$vD z%3+5c7PtN89cFvXlP*w;4Ieo-KM~*u8R(axC%#V9Bs-jH3 zi^X93PM(NfbD1l{AC;D3~PTW(<$H zZ#e&;?CS~#c{mc!K5tlAC9?{Tmg36o^!QHBIdgPza~jq*c@TDKps4_6ddbY!lf(k~ zk!NQJ@#xo-hwdwjm04^^oPwtEB7r0K5Vss6q>>fzXSwR(`WEu9gi zDUp9Vn4NmE7Znaw1Q`N%Iyb!=^t$o-=T|uo)#s|v;sT@rJ!uZHjUG`R#eXICWODJ! zY7EKg9%;G+`Jm7AAH_)eu@964_SBcz#l0l(&;8L^L${UYw|=HWs?0AN(-?%wOX2I5 z0P{7caBM!B74CJ{T_d#r2_#>$SXZ0J8`N8HhHFd04OUDDM&to|LocI|56Rj&u1K26^H zXik)@T`_WX|4rx1gmELW4x~u8Q03?GvJ1DhHj$puw)7{uD`BSD8>W$lFj;zCJwaJ) zTK>~8oyo_g1@WJ^|dm)0rX>Yy0_;KmTA89U$j8k%#pp%zF4>zdJw+RB3bhd@8J%_QeK^sm2_Hr5Nvm>I z=cadqs!#kFbVx7tA8HV3x8!dO|JfB2B^~E=H_iHIpAD&o$xPwhOTh9qKR#LM4jVR1 z?zrO)Jx!0H8X6kp)KgDYxV|4x^v}k*e%ABS{K^Bk3UX)3!J?wH=$>ZexgQXRyD7cc_l|nS;bae8s{i|n;~MFiAx$4wiZ2TLGO{3mOb=D+zgsKMF~SZ za`*bQ3XnOHxaLgY?u*eu#o=VoXc*Ku7@lz{9UnJvGvQa0!<}DHg%=mrAnEYWAaNRl z#M3Ybt;d~u8PIM^H@5nfL*030Tg%d{9i<6pb=y#P$iL~jFZItom3lp84vquGyw~_K zBjl>HzZi-2<2hAw^*4SZt5GppDOa6SSAC-YC@MwPrT_KepF@MWSk=9H*8fU#?J^y8 zvGDORrrtJ%1XyP=Z`<~(2@@u0Puk*|F=K{YcG+d?)r)51%*9}JAN*C?Klr`mk39g3 z0!JZVqOM5xSY!t2hxMKLHC*dJT08gd%9n;CEeoFrS(z#g=3=*6`~F}2MW$YRKNelO zJDfRkbpejsy#i&-ZK1*x7tYh>{mncX3+k{<8o`BQh| z=Ff2N-k1Dq!@@MGOc*EE_clx8C(p_9_5xX0a+s72tBm$Ib4?C=&YQdG(m-2RKQlW| z7fEprwmTDkB`FR?7^G6D7lVp(@el0)^M%?R9-|UCsbn%cu4d?74KY<fARa?Ag<-Ub_J4hOak+ecd3*GSv4nU1-T{~p*? z*})H?;!u)7)4M^_JK-Qyra2gHFTi#Vo-}sk0lUlbSg>N|(I;P$pZ^hyuKZB9>fZEj zFsXiHIyb!=ByCHt#iMo%LR!YR!O&ATxX~lzGbpD7cSw)MdEJjlYwMPAmv}dTQ$bu@}!H)a6&u`YmhiIk--oT;|OqVmc9Qr?1H%H)*r|n5B?TK&c(wX@~xMj zDkmMYKekX6%1}H-yzBxuQDx%73rjajt^VY!OoUyjCwpHRck9{)v&{4ctxf5oVg{>hP4^{=5CTU z&^pA;#%U0?TA>%Mt*X|NU?+mWRe&C<@bD8sB6RaG#A#>t2H|cQEBvBlu$VDggsves zs4k#oFd2R4s6iRb8H4^c1RTa%y9J*n7V;9}>}_y9nibiQ(xFmSc!0Dnn<1;)V*4Z2 zzc1*0t^YK{&r3BpAFY>appm}x^v}zLu_L2o-t?Vc3Rib?i-_fmc%^NG{-aoWKsJE? zoQfSKt!9=qE_^4os=EjA_O&xVKAE?z7tOU6S6+FgOq@6|it3tcu94SYe|?=j>8Npj z|8eP@`>YgT3suHoO!<)texl|F`NP@KQn4r=B3iY3~_T$fPCey zhh^2Za_oh7%I_Y39yZSKtbzM(JWmeV3&(b1DrqRLi7dkE?JS(B&9xv_?HSZiBuJAG zkG-oie$1IXJ3CsMgTemcp9#RfPltmYwTq?k(-&n<#w2MOK1Cbm^73-6?|5+2me6xn zJ>aKH15F2JbT-m!GO&`n0}V8_sG5W*3_>LO)#9PzP?ABXFiLMd9@3`&C_bjf4DCsa zUzrZqiW_*WL3g-Q6aC_$>bgrqVtf|2_s=12_Mm(E+O`yKNs#(wW@RHic*tm3I(ULK zFTk~wp}Z31LpRzH;@KGeb4WM?Pc}XjxJR19-JFl+77&`Y_6|A!hVRL$x(2A6i^g!p zV_)fCW3V63Q^>33m(b4mHrNgS=~!6EIm$IN{vfR_O+ntY_)>6FO2CG?xoC`Y((EyD zKANk%Y1-P_>?UQ z3sA{}nVm}$-admpXtGNEGQ%?Cavxw***?;WMzMY6r%sl=kmttgKRpBcvoX|`k7ET( zaLU-&QRTQv=SxvA}0D zN8@Lu?ER$SfDN7`7P%YU&j@6J4*?WI#SC3p1u`!Vr(# zJ>un;Sin`@rgN1uha_=cI2LQ<(pX%wD(@=3Y(-h1P8!$z;UW~ns2@;%9^n1t&g%itZ#jK1JJ9#s&`9JKR~ z5s!oP<4o5@tO&}mW(@cAG%o^=@+aaV&K(Xv`NQAkOP75^>Kd9Yw48R_fjCO{M9eoA z>CIF_iu18jd$0_~;Un51vUGc>^gYcGcbT3%k+g>)Zge!gboY;!ERmZT?9xD6`k2Yv({j*0 zxx-3SDAsxs;!&k~m|VeLS&;%k-7ALixJ=`q;>dgkLEMs(00VoY)eU{8NCx+cRGLAC z$GhVJ?iQaxgrW?vPX9AG#7(JKMaJPkZoL|W1x`x_Z6nQ9Z#lWB-00KEFZz1wWBs#{ zW>1=xNdXqIoPEL}G8wn(nEA$!9+l~DeCYMC@%lHq3YE3w`cKS1>%%{{DGna8wX}Ws zXKe&Ni_;`=a@-7Gn7=tn#8aY}zt2DaeA#{X-J_J=e*5k6!V52GzG6Ml(cUVJFMm(6 zibhBlbf@~)?53H-R}ri;th(TEG08;>O}grddA^aoD%7~;zr^ifk*2f4~dfw zSCwCX&oAWaZ~hEJ@$PGsW#_u{Pn1)RJy7SFhZGm+RFR$xhLzg7T7xuc3<6d99)@vQ z{E6vvy!OPuw&aXS(`!&=fw&F2bkIV5(02@{9iA{vVOkl&45wflX1^rWO>1TDkYl8< zv{Hw)+3BF(Ip53?6$@|SwiKAOfhIF#YRw=V_o+j?J_D-k@nf$6WLrjKP+wH|z`VfY zl%NXn4%9p!(lF@hDINc#%2h8u!+SjP<8qLr5#rv1DTz>V#9>h5)?7i)K3&}FxP_m^ z9m%E}gnDLAgBhBuy6cAQjTOjk(q26qAygqAQdlJ7Apze#G`$<7PZgsn#lTuUcNzj`iId9JJ9aCtdMv`Oza!xd`yfT@XCBlc)0W7OFDzo_WRqKXzT?1Ok>| z&B}Q9)te$#F2lSZ#qX2mW9!0CGX4WObn)+#V(cp4jLR$xkK#w-)Xpc^(&}G3n`?28 z^K-Jeqg0laA1&p>M?^<=a_Ww{-;x_=N+xLots;YT*Jc(($9+1j(nY02MDwLV(~Cj% z&_zSAH9LAx;d>b3bW37fP&Tec#^H_-Eg8r|08NXBE?w^H&+%Y<80^XqR~x6Tbwk`q zlJl9cVnxQ`A3KSOtGegsbEx|Rtm4|q6&7qZtwcz5Q5YkWhT`P)4t=H|ucE?g>yopqBuJ^h_1hOwiD%YD<%kxAo6 z<2u)TJy)BHEOW7qY7q80qx#bDZekc8h)UN%hx6CsBBU`$)Z^>Jzl~}tsda}Bc@A-x z21|!Q5H#9=?RB^+5(kjXN#9-?haW7%Fr>}-X0F_)uBl(vYmqLQo-NRHX`n4rlQ|w% z!pkC5QK~Em%ODi#8cg2{V&e!mwd+Qqt_OiE0e9<77<95TWe^(frRr&_BD4JS$eyMY2Xsg5qmyVUCncHEg`)vv*l23%IZu(aL zOal)vVI!TNhq*%xb?-iD8#(98v2&GQe)}W2>!IK2QJ`_U_UVbUgU(dv9wyblDX=^J z2Q=bnuLu8GL&xESu^QYL{CeDcr$wEDn^poYUu9@-io@KTliq8uz2w3RFN`9dK7G1; z{nl%6?ci>>9|za$+9ECZu!URdIb3%DoHIRKi=o*kT>C)Jv}9+&pyG6(9o-d+AgfP3 zUCcvA7c0s;*T^gHek=!{ag%&ByNa%A_QJjD-@0b1jH)Qt`Q}m#YZqWW30G^UQOan=E{cKDt;BEp5P)Gyc6d=?sJuV%@>*W~T?Vvs`Mc7*!68p*AGTYCXi zr4zM-^vg_T;2Vm7`(K}b^?iB%wfEJYKgEUkSU|e6FsH;} z9cLJV#$Y0@j?s^$)y2k8u{6YU<9u|s)JR*$YRRmetc@2H|5-HlQAWriAr^)e9a#NY z|A7ojTO)&VOTl)3=pPzEPny+>RWf5n4U^dmmdb)9D^&Eay*EpC8o#w{yLBZRVEex8 zS^tZXuU^-D_3!w6h2A=kNF0otd|bXFtym6;68{+c+pt9QfB5KCV9ig=}K)*&9QN|ZB!;t z7t5CYn+5@Sam5(nLIj>)TeV=W4EgsxQnb?v5T>iRO{ny3cy|JcSCCVn*)cqEexW`8 z`upurM9tEa`(JQ=Vv*m@mV(f0BUbE(G^;^xq5@uqtzcAZrI=BR?Wie} zv8pDMi~ID(QpW0TS^no3foit#Y&`e_a)Lb4#22 z;?Mt-2Y>Z=RPn~s?%i7nGUx>-A0d~VcC?fhb5W6{iJ1GEOQvu~-Tb=r$10lYXQfV8;PvJ)Xz=rH;KyL5foVYY{u#>E=J46URjSICc0DCf(N@ z7T{I^${u#Wswgdy@i;u&KQtza`f94{~?l;^XRZ!4;-`DXt{$5@myJlODv` zBm@+PR1hti&kJ6}M>=`xVlkD}ZxH^P`}J);JsTWQo&(!c&wYbx8r!J4zJ*ufIw+q9 zt_DWN=*?FHSEfuXnY?0y$er8IXI$~80Uee@J%hvaj*oqje#=9A>e`xW>2v?#eY9ii z1~#B9l`-cEHU`5qD${bt!+3=eWT57CX^NLSF^ROvGZuCFO}b5##LpM?Hl?`V-2j5N zHK9V;FQ?iKj!RxA*KnhGR?3lv@x6F|Y`BxAPrN`!3wBbh@iMW}hA|in{?KxU?aaBV zoWZWF3K|yxU?^xBc!i3SBtx%M`;Zmv`dYTF=xl1`%blUmv#fvk4Ob(?n1KvV0o4A-e+nVaw zntP0nboL7CUq0ADYg_BtP^OMP@X4je)%BNOWZ_EVsn}Wo{=rvX1~lIN0tMDyea?2t zt%NdxR`mG(U-6phuTe?!Mby?@E9+UOadn)xWvr+6X2#V$ONpzOSzhg->v_HGO5Ql; zr2`%GwvYV1?1`>?guTUQ?|v_BY-<)vZM2?cWiVFG*cr|#k5EcfX#%09)EH{~CRE?8 z_kx~|!#njI@)cm54Y@>y;2OUcer*65-GGvAzUO^j-vx?gOd~z7(DYCzy;gcH6*XTd z${A~FW4swtL(e5+u-W*ZwTl}{3r1oXr-|pw?sTv~ zeCJ(XpswB_XRym(z^(N5^Iu1;8!x8XrgcOH=Xx(>ZS zkVD{c+nb?2>g-*to;ftl>uHb2f;{E( z8mew+B`9XJnlaM^#$c;O4$rEfVJtK_I7kyOJxC3~5nlT4Le4#Rmb%Yo&sqGj#pj|D})9 z+i$pBfQn#Kn?sA)f`nq#LX78>Vq;x2Aqx8LRi~h#OjJn&2R&?$(9-i`s;sGJtNsgw z%L~naD@!tnw?KPu&;kybf*ukKhhXQPeTPrb$3OSYd3#}&4p(>6?KkZ9lm>6?v!6%V z@Guk(QT4U=h_E@AyXUfVVEIDPa*oI2H1gEfC>R-(&S*8KF*A1w^j`IrH!WWaRp@CIY6(ndq}e~;`xbINJ~hb7`A^i7 zw~j_@uAl~1%ve}PyR=tBtJ%rcS-ZFaW*7@i51*j27`Gy;cv_2<35I5?p*dd-e;&`> z>!RB}{5cw%%v8=5HBEHO`~C}E`_8+mq4hilU(ox4tmjxjq^5B#ZM*CiTDxTz4Rjr( z(Bz2gURXi?{5Q|jE}lz^3H=5f3L?GLt&*JpLK;(%Ns980MEPyIW!bpZ1P-mlfI=Vu z4@1csDjWC%+axqlpn%tD=Rf6{a@0*TMFeud6m-oC^A5bn`)6{8@8>v}cyL^}%N>FW zZ~4<+-q<6HTY6kQ$GF-=8+daM#2I|Z?li5Sa5I!&Tt%^-7b#V{m5M6rq?^uM^M99` z1E_FL_8p=4{)eb+-DN!Fz}>{a7ULUAv6Q<_;g$nen)ZT%Abt8H@1$F=-6gAnNy}N3 za}kfX8e_0@IWc7j=*o6@%co*hpV#uG=rz~%oj6m!zTX*^?J81X?tu@^d8Yns6k!om zUQfPEhl7_;ylJ<+_{eHi&fwpirf*VHE2`$@s6!%zkderALk;salfhu`-{|-hcWZw! zt8(6Q$(!i?fA<4Af5$b5$7#E^^Dd_M{-_%C4qP4YE#++BPauz(`$Lr8q(UudH)bJd%X~}G4gvn@Rj5Bjg zhJ)!&fc(49d@bvk^X0dEkZ%3pXQ{9l6Zp^EVGrJ`-+mX}%&l?Rfy(hp7^A)K6JO=6 z6k`IeG%yf<#MV&bNBq+%kdsyZ0~}y_8P>MNoWq&c%V~-#cr~AW!l^V43}@5aGw1GK zcY#mmh4Ez;XoEN85e^KyVKIi!-1mTVNqV#9Y~$)q##JLR|-F_DK&o9>E;&PCfCw5oqpwm_jy^_jGqIQ1B}wu#MldiFgP&` zlY0;c+-p7x0>5=?DhwQkr{qb#H^s{dPaXXO4NfHJwaVM5s$~GPu|z!aYbg8Wnnn3qGBUwk+_2(a zeO=tjKxbtQEmyeyIPVlQ_%sE$yEag$tGa=g>HLQuPO11=**p2KegTc+Ue`13H^_oE z_p&jK*DVLG1}t1%b?G*V4&tr&*EA@_4d5ki2=E%+@yS8rZ5d?B)oL0#5k{TGAILFu zZ9|q#|LI#)RM|kmvU*nxkS`Da>S@XqTn&#+%4QeW?!M4q#)_>49}WYSOrTO%f==`P zZ0|UgFhprC{CnROAMfV*&CgNi%mx~*zlNGy+ihbo>_m00E7`s(V&KfMC1Qj=^_A~sd>7r1OQoFkV_SFKOqaj;Lwah{d+&e5 zTBR(r=BPDD#9#cN;%C6gBt`o*tI`83psO2G8_?xc)3la)s;;3>_a5HuEGl`9e8-9~fHC)~I-CuC;qgmb-rNERM#296V ze-VT;i&|%f%yK`{iu=9kw;swMD06UH{?splS$;Gspg8cHQs=ucg$kSTOm^?1LnT*J zqV+0jSSd5SZy!o@X& z31vS}1iwKOdn`XF1a0o(>UT86b7>Lxg{D6*pQp@}H&V1`k95JRFq2pKKa0izX+4h* z5Do36!n*Yibec*fj2&|qS5Gpo#--9HN{$g`Q}#OzyTmV*L+c)gAJ?>0Dn&^%n4A56 zzG8pF_~j`Mv&yP`gTt6Q-(^ZMo`o|M{`f>MO&<9@olLgSiCW&jxorc%7z{0EEZeDz zkH};Gk8r@P!5mCM4NR?gqk&3F@?zyq5NJj#)l7Nv*}c>=;GXi-wB~{{w$^q;Z9(38 zp0>4R=bK#jVlFMlnq~u(ra|cpwwlZ{T`f{`b!v0hn^(^XDr~6bqCnNuH&JihjTGwJ zFXQ#fH>HMMTtK;WI{kXxOCbahrhj+_eo$C$d#X^~Bc zR@>qu%HD<0SrxQC3?-c^s^c#1unPoWK59iu;YTJu#0t>sc3%JI-k`H`j>S*B?%`iK zrspV_K&|vBm9%lIElKC*7L#7SoXWW>X>>`IHf^T9k{vvM5#!OAdS!TN3N$kn{|pkT zVOxHLDW8?H1r<#c7(c=aI%@diO`v~%+8hAS z(d&Y>90GmMQen*+&Ky(OGuEJ-F`PL$FRvc|BVz-$8a&&C!KKs&niFsUo8xd+3dNr# z{lW|D87ZUexAAKE^_>~-uz&&S0O0lH#Md0NfLTD-JFIb*}0 z2>0&faWy+w#(SDemf(lKgM*88?&IpHW5ql~8cq|SehJL@Q9OZ)pC41I>9;BKLgbu9 zRpu#`F9FBw?DtuI3B-?zNl-pwwgesLI<`!2B%wI_e@j7QtGpRlKPJK^!!<+Xrt}A1>o6;Jr9+6? zJl53CYiGmX#v5;V-D}6Y(?D|q&U+)emIA!O5ZCbL1iJURRdViB(iOGMbi9J6UiG{x zy)9C(nGZ_A66N8n0>rAJ^Fw%$@0So* z6b^iWGyFjr6u(s-ZNGJ0KF@yZ^K!2&Jks$r4NrvVVD;@(*tA`G*)Tm5J#M=2h~H`k zpXYb;E`F^-I9XW3hCq{)&ucK-4iXfZCz6z?<7B^mNj){ZE7fv3?26@eM^E%qDv1@u z06@SCG}EDgr%KG`9q2N{;X5}6fDde%j#b7Yk!{o+K1ppS_E5zJ?zv!GKp8ugl>2G# zndSv3oXMY~Co&auF5>D~7vqX6b$0BzY|NW}baPK1lilNBA&>c=zB@Axpq$FWB>jhZ zUQdGevgc`{+;ir2M^E&5gLsaAro70wX$b?e$t&Q>6a_|>L*5z%YK0+ABF6YfD9$a9 zQn^Pj@(F;7Qw1!pyPWq1z1h5D1~wx2JSe-^XcvY0_fuD3D@7W2^IGDyA{TVIPGH+_ zHMdz>cFHwYPAvG~yM=K8%Gg-;|06%JIDE@iG0H_;eT{MTj|;+;lb0wv z5OiO62KxTR;D9J5@b`OJ-?mb4H36F$(UmY16OAb8LCR3pK;g79Xv-_4U+%Z=P`}DE zF+7{&eb&28c{blL8eRQN`PDOA2@3PlzhR!!f28yVDrwwA&0Jx}R7_ZC!w}40c*Mc8 zQ}E8Kpz$s!=*a>Yik$=^>=uz15LoHS>tH0doG#n=HI6Nl7_P=>=5A>Di`ca)|rP$g@wbEdK5PAJJt_t3}y0%diGn6yBw6#~rt~A(B`ksYwU|t1{E(sjTl15679+1pcoZtgae{peP z#y1mjY$mXLE)kC{-^cwgFBu1XTrC-ESM`f?5m(q=)mEx@d)dk+t8px6s#ukJ6tKKQ z!xoau`W~f@XVA!r4j-ckFQIxgxPzj#7f~Y{fWZh1(=RdX1?BA9?>Lxt8s0e-G~S1r z54tC2N}DLvYgVk)+PGdWxe!Dg6>EbrB-0DW3gWp&aBh>Q>O-EAP zrF}D%)~}^DZZV_Pj9DF+KIM05&%wY`xHPBKL`LF3Ehtb@&6eDSykN-`1&XX4j0ExN z3|-se-X#X{7#-+3h3G8)t)l}cGG4lQT`O%|)69w)z*W@aHudwUOOkiLp?XSe*eHe$ z0G)4nn88X{0BY4x($Q#?#>U2IXsC%whF_q%z&MpOU%>30bJ766OSQ9IWs+9=jaG-{_ru;Qk(-0LkY_@rhC$;dW`R@*~O!R2y zR+UKU@8v44TI#tXYl$>>YG$i`5byq0ObK50u$I>)hCz*wvvfY6st*c!dU~1)CJs?~ zb3Gd@)!464JZb*y+_v7nrk-5H)uweuu6pBdcY9kELf}SQ&5|Bb*zB{eXFRWowy5`c zEti60Zm@)Sxz6z9FufMMg7Rz6rxrE@Lm5{qZSB(byw*QG0}jlqpz&^1(BZ;nDvu0M zVRf7JNz~5ZHFd0w?Tu7lT|q;{#dRC9Ka?dD?}5;$ zui+KTs>^A346E50w^DJX@psaRu>y)Um>qp!_3A5VCwETGkM8a9aUO~*4Ny%!kA>D% z&H94I+Nu?St4l9DPtu^cP?{}au4_&W%Qs8w*+Da&x5`^up_@u{JT$@!kGx95sX97Q z{uZjNX_Q8bvr8LxO0I8byxb2s9S+Q^puu!79|~%^d<`XrtW~(Od<}TGYS&i!$rI1n zS20%DdEhzPu>A@<>}+|)22WC7=YH4cF51lFB@NXk47Cf?E=?^X?ME1^=RoB~u|(9( z!+ij{SUKkjj6~2H)bXfpx{0<-{*nTvWmHttF3+k028DwF(OSM0VCsIHO;=yaxcc?; z5{#dm#g$@;Q2GiKUpIPen+8kNK1{3MkY zGX{%zsb0wiexYUtC_CK05^z;pRc>&DIP+meAAmw447Jju7~@k(_iW_}*A}Q2pb_mozkhI96!f0oJ(R(<+c|!=ksGc8KyM?qft6&po(9eAJ5dPU13Xo zGC4L#uRi^V>oZ8pdv3kb6=zrcxqgcCR3gGhi~DxMmIM$WiM;q z%!U0bnm+joC8mcME)X1C$7gDU31ki|Pd&nharNe_cH}m$ZhM0WOvIfNA%|2n*bizd z9cjupNLo{o35pliiLkRwuX!ichXeHvve%6hK^L;OjSI) zpp9*DustNYwb61$gUy$V!tEvfS&-&|;6N8Qx+|v&+gZ5se$Skt0uGBRP>Oi&+lSwM zLq?>3^W!h`@|tIFF@`4}zK^_f8sEmPoHb2#rrdJ8B@GsJrYAn$Pt!$hb}i8toEh1< zpaAd z(S;DT98v{mrB2c!VRriRD9&XGSo2#WhF6RAiy0@f*^Lb*WqO)oKm}dkAAa}s8K2Ds zd3F2sJ85kTRzfs!mdYRdL2+nhRx7w%?T|2`)-&+y^InSdBMcNrDjuRx*9$a0+)I0l zuBS-DZfa(w+}_?UW!K*xG6N<{9dh3N>^B)}Nt-la40t~F7dKH^iF+5Q zfs=>msYk!LVV{19>9(!Yym1+_KJDT>aHs zErDt9uXuv+)0VUGyL${+gHU{;j}o<8Wl;z4%E#J*p*81?slsYX#-jGqo3(^m4kjU}wD44_FWfz&}**Fm8ptU`0u{yp9eRUn$+)*f$n=0`Z{~ei0-} z`5kpRh^r3(S6kM(@UM76S%4qZ_)-0^(#03zlLM3pl(Ny>THDrvFZ=EcIRGV+Ea8Qf zc&a2guN5XQHF%qmnJ2j>uRd_cHMHY`wG#I{Uds~)oKiVKVS!VD26Ran!qb&QZ@Tpy z=}+=XkD-$<)6_r*?Jv5Drkbvy*0y%L)vQyfApd*3&Tncd%z#A-!0JU(=mnhu3pZW_nQCZtXGQSU%62I1X z<{1PGfsZKY7%qXrj^ve5Z={SW$6plO1WS3&D)I&y!4851&w;`ruP)ijxOzH=aP_5+ zF|K%yyosTt0~ix$#oq``6MqyBqysL*SQ{wiG0?WHgziJkBh|8I?(lz4g#(He>|`|+ ziBs|5b5y{X$}cDqo`dhsLnIZ4fnJhp^6E>FS7zyC#S4l7{8n+`6OOnff+^AlW+{bN z0t|JgTm3QaDGc#Y(?GJ0Ix27AvE~gj&WxTeOp!#d7s{!xoWa=TaSKw=8Z`(?fPXAJ zL#2byQeo{n7P>|y%RLGfrU<|8vTgLh&!3`bJU#Y|cN~7{Nd`|IwVk&^sJ{P9=)PzF zm45N<|C*tk5swGI`nR-seTxll1+EQd0GW>jcQ=N{`f0S~EmT+E;0w(&$&f71BF~|u zapbmsvlOl*8*3+U|hwP8CUoHEp6G*Dl$@% z>%c$G5oZ;u8h@qnsX>a2_0rUu+qes{LAnqj2z_^E9Dsbma-UczCpz9oMaeN%(xxFH zdBLv-m5P$(2igLR$!m6PqaOlSIU%oFIaYH!Q%Xd1iE{S9&lQF@Y5Fk>C)|IKLc?8j zldK`~>K3+XLoJo=53x$|JDubz^EhJZyYI+WmPSEC21FrTNz@bnM> zCEHQWi-Ej;>pFV)@#nbE06uBgb@)ZX(XwtEl~!=e=``Kc_#i$0!!OWtkAGK;LcFiJ z?^Ew57>B_*0j>9GSSlXj?$(#6v-~QmZrvo~E;^M!@Ac`Ka{w5EB0|j^N*a_UD4k5U zQo&>g1tyNOlIB)VfQ@|kbI2bXIhLooqKsa50pp72+b<`sZolqQJC51#pAmnHeK1}M zo!mn`Rcs;K!wuM`-KYwx$FY%fN5^kLw z?W0$odW6P@PEuuE3zcQryV>(f`*dumo1S^>+w{xtf1U=rkJ#Z(&;PjVUGz8axsju1 zL7TsibRZ4l5JC20P3~~_E7YIYPI+xtip8RCm8R)-zSsAk83({q?e2z>mSKcoAr0iU zQ8+wJ#Un3K9-jw_D_kEkd13MptJZDirDc~3S080u0Yw&sT=CQsy#|T{4m8~V8jUB4 zDAs<1U=14#_ySSbvnW3(-&n9DGQ)d86dd*&9jJQZ=~wB_d;SlF zBC~J*Sl6 zxsyKlj%(*kltwSycYr3Qqja+Ic4}*1BQ;7Lf0^?I{>3xm0Put!8DMH+VnTMn0!8=R z#?T+AxU`ar8@KQnV&*oc@?jQ6PyBHiaK(mUa87}U^SSVk>u|BrE}9-XL7h$Srk1tm z3D#gC>kCE0pT+nErlzK*L{9Y|d6C)&eoAF)cX5S}m-mHsL%|n<;W=%ErZ3@^<&sw_ zlT7^5al*Mq9n?5?2$~_T^e`Y%sf|w$^9C0OXpCy>jisf}SkDF`|6U z9eTX4ECi3cR;OejG z4J@y`^2r;Yc?GPOl{}OJ`4;6_62}TIp;W^TQOvN+Mt3%PuGAL9`>21O8yr}07dN7d z8Ws+%?T2{#gTU}!3Zx>u#xk#OJ&%UQMjCg%?T!2nqUZM>_(9_hjdQ6I0sV4gMDt z7Snh^I}OFksC0ZEr9wk&?@%sX-zs5&BeYr@c>2|yZ(UkkJ;1oSfDOXnAV`h!HLWid z=yTy8^PRC9(9sIEmTliI)6IZ22ubk3cZTW@hZYT2Z)IHFW8vy_Q!Q8y-Fludp>UVY`e|H|^}X*yGR1zdgduWq9qEU(&s5V;2HY?W_f^TAf@zRP_MoU(#Ok|1<2-3sG{eSN$V z(viogDRGR)c`o724tT=={#&RU9GRd8*~0 z`6sGRc2Obkh*ii7eB`q+nL5pDP~hsvk3Y+}dY*CBahkYl!_7J(z`bl7{83u?m)znpf z1(nyDv1TY{*f~JHp;tkB7M#!e{@moiX(?zV1>lcFB2qK$XW&;Idw?p-ig>*6e7>qR zKzE`u;qMk`lWeH<{NAH<=y)G>oE%`otuSw9AEOdpFSnwsnAfGP7t7sU7jEQzT-_GF z=1ZB7cE(_a7ciOT9a_dy<l5bs3|BCsBb^jwgRtJ+jCv<%uI^N6jYS5BbQI=PQf-9_} z4P5QIaQ!Uu%Kg6i@RoVEk)O$USjL)TY)E^oa3>{ex5=g!x?eMl%;cHPVUV*Grtxg3;BPP!aUU{=A%9 zx^wjQK}n;sr$Z)?S5rK-bIsHrs3Jea!wy?0P*Np1oK}+QsuxfKH&++9xx7fkG?NIkan5TL zj>f5@^hzqM+h_w=jdc7e23|(z);aQ2$EJZd9iN7xhCe(#K^38+)JC1W1nsEoH61Lg z<+-o*+yx#KLo$=148VC3nkiD#V%l@7b&4UA1g{Dfn;hibxrS*Z*iL;#+lk7WWF#Q1H1uiIupg8Uq4S4 zSOhvpDs9PA#1IYObL-dg=NBfH5Yc#uDy0W_-jC$N>&ON#jsK3(R3!mL5>n=*CR(UgeeH4r#hnAxV4NF} z-qVcRv+x>wqhd-+6q}CoG_Z~W?q=kQ0$MTX_5p~so6)abW?^8C2dR?Qu5Z{uVBKXL zXc+;Adawqi(iq!Eqz0)jFiu5@QOe_auvnfpz|*+$dC%ay08hE$X*U>$l(9(6shLU2 zvax6#&m>~Jk9mw~oRZN=g5g)9ppwSe5NDVgd4Kd)3i66)80Q17&~iZq5a(J61kPZX z;|t5>2ban#C~4qD4G=*S6Ft-(-%quvUNc*O$EfpR+klFkIFEeF5J+cw-NPSCfdh9j zUbj6q(aSC7X6nq}Of%(e_CD*7Y1-&OMNs9Mk2eoLSK#RxD`=!yL66pq8e$9&_EYo3 zGc3ZosknY474Z0=sVFLl`9TYS)`d)colyY19M*Sn*EY9o;^U}Y_E4BNU^t$?jbat< z?reZqTg?Em?^Z4cR8i~rA_Oj4)lk^z_6F5+_{HFiSEMUUPEv7d!cay^X&_Do6r%to z_%JgFQt+{&PR;}=JX1*1sS=9il~5#5MbTgl1$cKP2tp`i;0ZPi+EPZCw*u%}&E^)) z6g$9#;sq^f^umau<{^ug_(5t)osgavtdv*)+XhtTv`v8|okkCg-RPKyC8=0g_DYXU z3{sLSi9NKI28%XRKKBR#Z&1pp41%IE2pswq!Fiav0xvI$f=0SkNu%`wkYkJo$~rZ2 zn9hs8NW}&D(tRWg1sF}832?8%o`<3G&G4A)s~ex_r^!G)b>(lOc=>v%Q3AXelL3$c zVvIFw?Q))1`-R(5lw4YA9z|fI|S|fFEHH%Bs}i*Jluh_ozeb1Bw-R0dDj_ z*C7s7#1M>nAlxY=@eaQgqa4jWiu^{tqjCp~phc|)i10_l+y@srMr-oBs3b8=!7^TY zm=|OUma%O^VafdR%cHJO}=P*EF|l$J>) z5Xu?7fskohK?FAa9-7?4*J8wI(G)Zy1F%8h0qg)ey1a2rjE_<2)L~jrho~qoPK6bX zJO{DCtPF)NMX8oD@hWH!z|jSsN<`B@W-Hz#1*tpLMuS0KyrQhZ?qY=HEI^C{f>S|^ zc>3<#CFhxpi8X~NmgOxOxs$<36Od%dRDJ(X_Bv8aE%iwpmm=&ihi|c-D z7s`P}RnYjT3P{vS0c;)Pfx-sh3&K5Am*}J>>ZbtjG7~7R70XGi3?*aL7z-p`*PF+Z z*#wPd;!zm`OfpSICV4E7TSUP+>gP?y$AWDXFRGU=AJi<-g^dGX28ba)fvwvs_rF+0O?AMRJIh!Ex$w6=qXKz>1C zqcsgb!f?d6Vpk9!qq5X6RnaI_=1)^mYKrocVPnU@Tcyf!qB_4C6-OR|V{OdJIm$K< zg%l5z(ez9?jit&c6sV(65Up~vmaaPa_Xjalm9t(`5Fa}se;k*h8=T}jz zfTzzC7TMi9I{pVB!w;pbKz0gHjh9bXCkIqfD+L#TQ8;Iq1Xw>R070F8T-I~v{l0(Y zb3hffQ+{a|IFz+2Y8@^??+oOe^wKa5&O}&I^Dz@=31;DoB^rAAQuzgx%;)8h*-#M; z2R)#4A+vBmhCwl-rR3KVZ*O75BI|<)pz=Ib#P!w6|}NY6*WMvwN135)emqh zz`Y6@N>>##K&xQZFt1Y9!l2*wsm}qQ13m}Ngack>4dsh4$S?Ro)ejk`G7R5@UuBXi zXee8iYw$y+X~P2Ns*LryGl{$Z?8@fASy9l+A_Xp}v!EGig&~B zJD&qS2Ye2k4Gus~!Ku7*`kmM0CdZG;DNyf$+w{8Q5#PJYIdE1Lw3F9fAbZbouM@Be zXm7aloxb1afX@M+1Ix()m1A?o!710~y3fDnb70vNw9k5<13m|Q4)`4KIpA|(b`Jc1 XrJL@*@lU9D00000NkvXXu0mjf8v-dusax!9Q$OOnRFfeH1zkVpdzop`h2*-QCkYPBx}Drk9bdrL~Fo zX|oUJquZlf2^j&lXWKwGB4$fD3Nwmw6j?NFOLpX zV*Y+TKGXofC~07_I3u9%12H`OTL~f)n+QBStQUN+mjg^+m7LA#xAJ=EKZs=eOWgqm z1|{^r3zlPev;qc35JvomkdiCxX$FF?_Rsl;NnqMy*ahOx`5K>p3My6W zzbaMdJmhSeh{(x(rXw5g`f}v$m{0MO?5{A{C$bK!Mw9mgm;KBp7v08=`}0+=wwAcH z3^xapiLrDZll`$27L$?GdGTWf)Oqs@Lx9fC{s`RF)&9JG zk%-~*&}V;KQZ&--2Urb^5iP{~#WdxGr`BkhqUAXK6LR16uDw5*1uosp5eXUD3L#}E zO1BRqa+;H`*eDSAZ-6CtkX#~jm*(W4jAkk!d}YU-#v>%Di%zOAqeZ>Wc(}^v)gn!e zM>4Hbmg%nnnH#F*M!UNALm%l#w9kuSNlD33z3=NiSl9dg89dtVcFirF2K)tmGuuX= z%*gwB;a*BJ?i2h3|Fh-ebhLLoR()+P^CS5ERrhg3wKG<+lhrQF?+YMEUvFn1(!Ljo zYKXpREfy>LGll5T?V>K@h1;Zu{YVl}=D$z67(zekaepsqq5Bb)jEnoEybcol5+d-@ zGe-u2pBl#}Bp~>GG*h6Utxew-f{{e6ScphNTq4OdMF{exnYkJyW#rg!HPo4}C@+r> zA#wlZylU0d4!QpUoU6%JH+30HW@5J6?$0b1)i4h8(+T%uPEcT*Oy{awWG5RAV|ip$ zul?bDod%{vVh@@R2VHgpFos|eh^8c(I2MmF@IZo@-cM%B-p^+a<11E2DXi}=2Lfqx zC+BS!iLI@zQp+O2s4;!$JY$q*qz~N)`MO=+DK2JY{{&0)H>m_3Lp@(lbQy~^W{YH# zaCJT9^#Ud5D|9+mCP=eKUmkX%n+WnsOG|eT^P&^uWck(Q4t}jKMEYUL^2*<>ImqfW zNMt0Rzmta$8HE2H6kA>ruS+W{v_9!_xGNJ?mDtO+SBLBer(e4VGf}_~7jI$L+8%L7 zk1Uh&|IU^o;rOAWRSP_5&&)BNb$2t|npG(CK5mpjJ*}JMhd@WfS#t&ErdhX^=yCpo zu|2718c8w9qz=cSKmPK<&~|DN-s5$@-ZWcp=Klq-9_goCrh@czce04fZ=AP(Gj8Ku z*OoR_0P(&G*B#zH3#0KT99ZefZXMm5de7~AjNwV| zyE>_b+79$**XuX2teDUtw#!Ow;85Unz+S5d&Os_$oOe@;p;406zz}v*XME@=8_4At zk;}5G#C{t57H6>_^|V*T_}AxUOS9JaaYjmu02&_M3nsA&aY?$~>d??sR6zXgFP6oH z8*;NbBguHCTuRhZ&nUg>!DQP5hdad4c@_DC!1pG;ucDxSR$f!nVnz!cyuUpVO=@FK zMz^onbgQ53*-iF%gJBatLQVm(erb~3(DwG&L8;ZkpY&W$;;)ZeyV=B4Z>RIwIxLFX zP$6;CZ{ck+b?Th|yZZhDcq9V+X&jRk4ydZmY_M=QiTbVH>Sue0fTZRCXvQSgC_)zCo0iL^VCcYC8r zqo2iy$Qe80y9B9KB3Y9SvM2u+ziU*|gz*^goU9J-ulEOW4ePvz7)fCw34l7Qpl0X^ zuv6=5_r}8;6rZs#&Ga`C_GT_lPCl&9>#_%7N5F%1%-}ZFqH?$mz4#ZA%bH!I>p0=B zRHlD|r2hm*Uhbwb(%8!^@O+Q{@gEvS^8qIX-aQwYotWlNs=7A*#s8DNYpC)XUs3-h zo(_6nXzEl({KIM!=I0OEyjfXV`mUzz=N#u)AY5m$SebC)b>RWW5h=;f(6Srrzm*T8 zfO^A*nm=Zv`_Okm$|ekHh^8dIl*cB!L(xCTAo_BRw(nJ90C|aGcfb_t=JJj6xev)$ zj&3h}-Zg1xXfUgMbo145%4GR&mA0EDV4-U@j`u4Cyzz27)9Qi*kjmuuWByFWv|=|% zyjNOQS?hZGq~o@iz+U5wq=$=jRx?PVEv3Hp`+nU;Hy4f)jd6OUp!*3GK3jc`Fw=D# z$3Yd9+N#KS zoiL+5^#g}w0mDDRn*ZiWhpo4ocoJf_#*G-6>3&>d+~<5am2<~aA{pAenfs-ZYhX=d z<`|cH9H*?Q(AI+%USYo}$(sv^xTfqv|2OYG4QME)va-Opdav2$u-1KJ@bY#b@ZRzQ zO;BTayP%SYSFN>$7Fsx0(m92 zegTLZ!n6h%>w4WP=ZRR8IRgK#rtm)w%dQ0B6!V}GvP(6Ge!}ux%9h#QTon^|nvQMW zs=&KZMokv?dHAS|K-_^YD+rUPZs~gbJjUZ`SwdToFH=RFn z-t6{k^!Y`))0%bAu?4?f5%J4{|^PGYV zowe-)O95LDi2bUKp59e}SBa z^?cz@6IwkSmUw`pQFWq8b7uy__0?YA`J*b->jo-`U+1g56Ka};Nex|sPa1L*lWs7L zbCAcMq%GB0x>Lo;ak;B`SoM1Y?`T&eZA2ppzCqI>2a>D8ze3iwZ@`?6vUi&tqUMv2 z9s16etq!fZG!G)q~N9za_mH`dJoNHi&3R(P;)a6ed-8 zp;w=^Ur#_DCw(^XbSocrj}^DyPrBYlfm1OQV#pYA)L7Ir8xKE_>z)P$-sCr4&s`28 z`7MKL$A>s*f-Y>wmgp#o{?qug+YSeN!4VrGMWL_hpMHNxH=|#_B>B?`q2|YeF<-uG zm&9US?3*$#i++P+?3kfU+IXq;+)r*{cKlV~M-4Y@sJbFT(8zbuel6qnxR+!Eg<1pI z6B#_J=mIaceld|eS%UhY_p4~1A(yUaZMXB513IKQa^taTaJUYc6vJw2r#5ok36`nm z13C1Few5|99%E^y|hh@x*}nc}sErryF0-#zR({Gk(!dXx_PP5Ox&je$c98tG_(w`NZ$AWihR z=-_xOT?e7)-FV7J`!Dlv1!6^P6-Hzhs#mg!VrB;BX-D{#px`a}-xW$FmR-30eBL)+ z_R>lCc6@Y^rlsZpZ60!70|Jo7Ld$_deB>n1}WZPU?pXp7| zof2Ah1BT}&01`m@(v7DHsph#(>U#Nim@Hz|9EwGiS~?9)_4+qQ;d9OLN&n1EZjkA4 zhVIuKqF-Jwe3Gsw1-jPDqSX<5Z@jlqbji}XyIB%1(_ZHMAnr4Ec(|gSe@`My1W-a1 zrj5?UJ&lvj!a+}ahy@)n<~y&c9=bCvKYus1>{xPucx0d2{#OKeT-^uyCLFL8tzfY0 zO=hrZ?lWqUsjhVj%6MGxmVAe$7WMcp`r)I)aNtr*UWSy!^F>z|+2^f5jyRebbs{W! z_eM~sWeg;3m}TOj_htYe95oW&Wr*2&RG2oV(68;XZloGzkH%~~#_2Ud?FhshE3%`Q z72}6|mh7_K5^h6&fQ&LW*?Xkw|3LPFVk6|+k;M#8$guHe3h;|r=K@}~Zr`V4i1N42 zac)?QIO2mbW%vi9Bh>|2=Mi)=En6g z)MVycx5H=y5kh_-u5-?z{oN<45n36QwhBy<6W@`lG8(zYur0Ze^?6bZ=*}Ov!P6z$ zYo0qk(S2_vUJ!!YHY9ycCUQT_NeNxUblD-LBuw_$A=4#~V6^&|uXG&y4+bnI&cS&b zQuf6>#A?43xqlQ#RRZ-Ag|VFsfz-3Q{q=5DTl7QQH{IuzMdqU%Y13`!7VDZWnk^?a z4anT45pMx?qzLDB=ZnW#GNlqv+9Z>X_$_BL83s=ylwyFNH|{O5V7Nx+jxiHopYLIYa63g)Az`6IKbX@VO0DMn2#cf zw_-+ypo}{qU4-Cj@-V;S*g5M)}UX&7Wy!LNhys@E$fkw&{iSj9jCzjSUCYw*FTW5te&9QDi%svzeyq2uu zmQT@Q@JLZ*IL4N1G>ps@%V}<+`em6NqGD{uYG+bPu{*1%=(=INnk9j-|C}Y<>G_7P zdg)yJ_IFf+0kNyII;(J=a>p^yov`$>VVRN-#J zyWNt-&2i*?}q1v@QzSguPm1}_#W!JtV z%vn*yLStPc6a0hCeaBZ)Zr3!DEWr+0{jdlT^B_*3zT<6k^@k%gsUn*M<`V3$Oy*@e zpl(a~ycOR(-gMbHsTHZiIXG;@8;fOo?P@(gy)dNESlQFWnjxR!3u2cw{np6j*ljXQ z?$kr6yj9l$d=oqh2()&uSI0`Gm+H&W=`DGl!^Q=0PZ~ShCI^X`3anV)5hcJdPCXcB z@cJYFn0ak2s}Ob%XRZBoKc^^*#KUjd*LNvQD^u6WGn;W1K%)JG0+1pUxK5c&&ZMW| zzj;5`a7_NFFldEy0|Ni@;nEse>d_bMnn=l@LjTyb*G+A%dXVlAC6&TlP(f*?MTl1| zvLj&_=I!vyH>bnmLNnXYIpr88`ofManZ0yf|4Vr{KaUhhISU=j8Dbypw!hTifAWUiVZoY9g4;nQ)W$tK6QEMePm%MF zeQI5>bt0b;Qe3241t@?Tf6ZVdg&arneW76Tyx%RN9mjJ0#M_$PzUb^ ze-MSNzCA5=X?)(?FVlT3oloMX*fOZS=hj;y%{ZF7rG(GcQTCWBlIkMY_gn)d2#<&3 z>2)%qg1`)I7wX$zUxnbh2NQ+E^*H>Azc~7|wGms*XK?oybY(rS9gH2LhWDT_Gxy8%Pi@TK)74)Qv6vQ2nY67OH=I}~!6XXUaCk0}?Gzzzg$ASHQ>MiaTAGLTA8S{5 z(ydSR017H|#BtrFYnImhoo+Qtt?{ng?4{p|#Kq9drp`9r6c8iyMI75)FSGsiP%EcK zk)Q5q5eHIl8wz>S2sI=roILFz)z3t(A|E_2SQtmiw0{#dKYyXv&Mf8{(~Ct9hICwb z*~F5q2Ews~qmx9556zQohvI|qiBjnFLE_g?`e{}PqgrJPVe?lK>veCFPVMi|qkHC1 z1u>v}xn8%a9vomGNxUALgE2HOS0Z8!&!ltgHTcv(Hs>YBt`G;ACkX1N4WJfK2d#h*``3iQ`b|+11wBf=Mhe<7aXl@Ve#8R{jGgnZf|tq zJ79h4ufUE*iRUsmHq8P~Hb;c5W8VPxvYM}LJ^en1TiAJIM_1A!{#PR>(U5~HP^SO@ z0X4p*HLTH|bdOjHh=q$u zAk(xXC!!Y{5F;}K8yyaRx8Y$EMzrnN74j#S#RpR|WqqP6P)9F^9KJ`$ zE*+}A=7LwK72J-KQuLhb=1M%kKmyBg+oDjzM6?9GUT7CJ|$PAi1phyHn zI94q1U=`WYefpgqj5ct;*!9*>Lc#5rK0T zb;UA8${&KB^KTQ2gxU4anEXncsM*Evqs{8WI&renh@pV^MPJ@oj5Kc;E*XxBn8|ULmbito~>`(wBjxr?1QQ!F-|i zig?2YzQe3U?MRx{H|QR$8(x&%bw!i0Wf$lC@X7pHH$N_L6l4GC=}%>A-k9fLJ6m$T zz|Rgxp)Kb1l>@iBzwM7stvd}6Xlt7$uF?%g5nEu~X7htOtRh*&=e`Af26fA!CM+Ao zRZkmSW-!S6CG$BXYI!iCUTRVU))2U5q4b$wbg^afXKRqNN@1DGAFc-+6@O*BCM*(dHc*cbIpDz^HG3~$_bZA%Vn6y|9K|UW8-G%{8s6O^>Af6vI>JsF zhr62i%kJ-ace*_-F=iGN!%m^sH!C5C-syB0V6%v(zla`Sh)?+;keKQZ;@)oRWBz;j z9Ki~zcK54L09qf5rIe%w7*^bPnOIj8K6$Z3T~vURhL?yHzH4@GT!;NUBS!#h);wjyH5}VhpU^TGdqgj%%Anp7h8oZg%Y8JHeSchaRu6;N%(9Ur+F) zO<3RM5)Ir#MI3hG7Y?xJ>$_DdWK ze_hM{;S*xR!1EPF2j1C1fcS^l3Qk*vR*g+zlwz5)0b41|E$gRGY%*Ssh(%JVUwxXb za?-sW1yb1Dl#JytLc^|cjnQA{@`>WaYKR|pLQh_tgC4)3#T5|Fn9$xr8N(V$pqz3y za-XDm5P;%tiO=}MxU*F78Ze~wBNpxJSxyKE^Q>7iwP85UtQ}FT-_=j|{@r!ed7@)N zg1r?F9E2k&_FDH@n{PimztMo`^sMZEKEI4#^h+}{A=UYIJY{}M%u_=JMQpTz zU#;u#Gqf8#eO#c&k|u}gLO~bkW$G}g^XA)XjgU(g?E+VI$hhUz9 z*N2JI2iw2s1X=uEt>PXvJCYhWWf7l2V<3f&b-%cf_v=JDjYgrOQu?deowHdQCV_?d zc}XX$xQK=fg~;&3X#Pph<8Y#Fq~~YyRu68)|5o50n=s@~j`kBYes^@)4v?YT^u%|I zVVD>Pybr{cJ=g%Fk^bR3OnVn^w)}~(;U}}ef@tnq zdATdSP5v3GA9dcw zSIPhZUs36#jwA3{q=dt_+#9)DV0MrjvIyR-mu>Z;tedys>-fTm<~=JGG0o>1&D{KC_v(>0egPO%XezJ}tv8nU(-jY_g;QhneHJV> z47uf|7eKkhIimgjqV{SGKxim8W0Nd&mT5i*Fd)~V@faRED*E|uxTI+kDN0S@0Y_qO z53}p!Xq6H8`U0P#&=97o-6B8Yolq}b0M}v+(*-3^twc`N#_rO%=A0mlA-HDnrdR#X)FeQ(s*3(L7N+9SA0tHWv_0 zFytFHKFAy+IcNHVdT?ZvP_^>yK}^b?z-CCn>^8o*zwX&p2OpeqqtFE8Q}2`tudiw z!e!3%f6BQ*+?Ah1IjWYTF2;6nwYp!nTf?aE^njd{b-x1tLhSuXI=U*kN5rxgiD#N^ zyiPRzbr^49*#WgIotc$Y5B1<=aH@vIOf!>VoLOq`nW;gcRwoPpM4$HSo#HJNr|a}N%9dGSylMtxWi@xP`6gY==nz;v>Vk4b_nYg3fr`se7WUflaoD|cqW!#eeS=Iklh-N;jHgmXQVX2}T>j+YYJP9{z*1|<%}5Y%g5ecN_~N{lUj&>(jm;nZM(Qg77PJIanxr&iD5;t zk5tLg>wX<%`E(M;3Q@&{LX$L7+Z=wl@X9!O&&o=+$BHn@*|gf1D)9Ks_F$g`2&W2r z=c zQU2{-=mxnLI>+elL7n)9*-kjgpwb{tDBtA7d~=MW_LM5qd9V9XII=V|pO|+_MaAdV z5nn(i>8knwfT1*i-O z8CUrnyI}k$h6wWZeStGj6rL8ERf+sdT{d`_rSSK)uPMYaa8=yqqbNF+3dYWfK>G8Q ztt+8S8Rne`E6LqGxr-8Rz+5@)P0fg4WPG@dtNens+|JTUr2$udSMnWoKhz8`q2

kBWF%hcj<{)emTzB>2V!i=ZbiIe!@JUgjgMJicZ z0>Ic-m8>@;&dtXQ&$JRvbY___ol#9+4k}oHap`XV6FSbGY)$feL@Et`*>tDfTGjpD z5zd$b?|98WSIr`Ot#8johrD_DvBgusAG@$n74(7wHvZ1!$vF0yc03yo>CB_9iS2d&;eybYKVhBc zxnAgdlAzi`$x7)Reu>BUgsUC>jibDG7H?i&>oHe+kZCe=_qCA-n<^h}G^{am$7L^y zLnh2rWZQ161f| z$5oXetsr^xu6H~Bn@wTS@UYQ@m1C{4hBk;CmppBzOuYqAtjC>F;m8qp2cL%^FS zcGg;I=qdfBN|Y!;;`3rx7dw@3QxFeNaNv)E&PKJ>yh6+p{1`zj9uHmzwKLATbbvJk z7I5)(`|cPZd>PLh7dzhBIf7ZxCGp_DAI2>jb!~<-W@gn;&k3ItpWR&Ty#DM@^c#wL_D16V0S`!eatdO!y%}4l!(_Ji;zz|Z#L_7EJf-BerlX9 zygT6+%%kHk`hj{GSMqVG)1l-I8tD0*Qa=tqJsPJC`O9)z#{J-rD=GZOkoqo+*6XUb zb&9lzr!JkPuTy~v9KM7ZgvZtE*E6AP<6cVsog^Ob7xEqhR&jTXDI1WEzh!&fg{C7t9;~b)_D(9v_7oTy!QZ-Cn7d{9 zAAjd6%Mj?~CeSj2@@P$Ib2*A4={+k#{%xJ~aX5l|KciX2Om4>Fa7THfSxjwT(vJm7 z6W!pWG~I9LHRSHebxFCG85YytvNoS!SLW%BW&P2<~g1v$j+DZ ze%M#iDM!5wEETUc5UG2)ng}wU{wj4|SjQE+tVd1rA(A~Q~Ah8#>%;i*Z;wIXx>s&njq*Ze;ftXN|1b|oI>Ea?Q%gg`ESc+`LvkgGtc7VW`J-0j5 zwZDnxMOfW?;p_$j-R#`z#!|ExY&vzQ2ouzAg+2}>*m}f@7b>*DO^8yO;}kS;n{%ZY z7k_kp`R(>vQD8r>jk4E&KgJ%Xy}X*P&Nli?bsO}!Cr4=9#;F|@N-mX}l9%rL%AI^S%OtMPh8f|#Vze;3ncTUxfowf_Et3x(!t)sd@Qu84kzhO)=#J~gI zhA)m*NT%{C54g0}B2E%Mwra%%Y_JIrl!SI$yene-$YY@T`n}>C1E5x!%)pL*qVF6F zzp5)_-3G2Wl9Z@FMUHJ*iOjWQG~G^A*NRd)uNEYS`a}aiR&Jg&CJSaek_c)%oRl$1 zb=yA8vG4C6w_gX$V}6-z3|#sgiJl=xEq10xod*xE;rX0!3u(d)pk^n-4i}}3zXPgB z%M))dS}M5?Hey+Nk2-BE)i#((Vd8x!WhxgglqHDaDEEHtrB_V|fE?tZ1@}_tlNhCT z`S^hrDu`(IUa!9d)7~}?KEsPDri_jF*$**@{Je-W>th=gIVm%mbOxIkEo(7KUQ{HB zu)XG$9q-w(_#oSr&9>qRJ^#Tp<-Y{CF9fiS4ex6PfA<0s#cJHudee-jd_~#NGH+V; zeAu)Kljuf8J$(GC|4)bV7#%O@K!vjsdk+S!wJsU8*J)6&x76$QcwU9XX}D<(Y9w4* zsD}NuiP0o&br!{?&=zsG_W{I|-4#)JNcB39D0xk*$=sD8R7o@=JHTG+i~2h41#xR$ ze=?djjwfHpUO;NEc%Z*D92Rjb>musc|2#_0dAPyYYnt%=wFVBp`YPP&Jd%*Z`icRJ zIN)0`4o7g<9PJz!H%+0e$C@uRlF(`Y?Hbk)el)aARl;Yy<+OtPNC5R2LSHY@@-JV0 zqFQR|5exf|w?w_icR&Q~UHiWWkkg07^@GXP9u#Q;0p53vqBAw<=53sNsrW(fUz{gOXH`ZBUOK<^3YM`Y*nxd({$s z^>=?gR`&gOzYpr7k5|d*FD75C7vi>l@Y_$CyBWkeN2quYalv%-Pnp~&ZDF(;07H(aCUb7TV@<3 z*bqY+f#PAhcQ>aJCEjYaD6hrxTJ$HG^Zt%kZ$L_QkX~KURytQ~5 zmqX!vb20#lGw?A|(wmwwi*e7-}DGeVOS-mM<@(oX*?3V&B9(CCI6 z3%x}=hava5ky5$aDVg#X#4lqdG~^|#G@2SFn#WO8R_snLUtUdroOeOT0ElMxr=$6j z&FHF%gXyYyWv6W=fVo*}7q?sG(q;bmzl!pRw30$*n_Ys@^dv`py=}@HHN&|fS{Q-z zJ}_gi#1hB}eA%k+2d?j5K9)XXY%xMrk@ zgGMQeuur~OG=W9_O%_Tl1cgrtCa+Jl5x}6W9VdRh?t~uD8#O87C6E3~!~p|>DW8^V zW>i#2%cig%4D7HKIcOx!sZA^8P*{(Zl^wrFA zNYpdq51}RdT=i4r=b(jU%OaGI&q|rF+1zS}JO=G?D|(;MxbOHS&TkD_bEE8{a+bwD zZK@RxMfbiw-!2Y2)bhSTPMA$M%k29%X2S-pOc#hqi1NjES~GqFB6h@=?ssHH z{Us?wu*`@JWqy>-pl#+2c^;Oe3PD{Fo-@0V)+eVXKUo*&63Xt`s)n5rnjPk?_@ER{ z(^tdl9a4M}dS+dbEsRz{QuL~L@gR1*@NLZ!rA43yl!s4RoO{jF)U2*uvHQ43hHdpt ze(A+fR*5g0@Q91@n%0XG`-V^=3WLfi<=APRE%8Et@_TpHJm4+@O6bA@In#FIht@a< z3Vczwgle}Cyj_LUBe7HCA18VrKJqg7+g#8SZ7}g@as$WIIP+1=PQBvSs0^wY$9#jX z@?P)-RFdBI0FH1xKS^Az6V~o;2!HZLuOuF)F&~R{kOCMd0C`8=yr&mO zgg65S^1?$8HMV3d8$U0`5psv7gjxx6p8OMD`IIk((lJWOb$M;RS9$6Qj59JzxvUTU zI38RkZP}X>waSdNwTvhvYki8DB*mD56Z&`|hXO z%UO?ivbcbx;deG|ntm?KxqEkW)$F>WW#;&I!e}P~cEMsO;ch6aHM39fEv%JK^&pmR z`A;vG)VB#ThOa6a%<+vcKh4B5Dhkzt?PpfgL*^WeWP)LfCiC}4(&KqO=Q08ySJ zH&3p3UGxZ_HHh<`Tsb9djLi5w@n-C3U^{p05@0Y6sh**T)ihQWPY=Db$MUPy9>-QV zW9O0*8K}bXVJl>nyg4t58|o*RZx`MHx$T-`c(-YGg8uWiwiN@CAGt4m*&-@d+`y_? z*2nePIf(Qzlad!;?H_ntM<>^fgbWB{C4ZX>ms=C&?N9RNA&q5WC8hHo7=*NOs+Dn) zS>?0aqr1fv10z$53!@f&AwxRK^dhHEdxU#WV(i246?KcuB6%`pb4)x$1Vd@4>xu$I zXA=#tME&?Y&aqmLKH+Esl{7*Al?qMY;MH6mXbp34Pr3#38XJ_ko@)3XZ>AeBKaSgW z-xT)^e)qEaTR;&!l)0m6DOIKT^prNU=$@fkW*i+KeTt{Phc6&J!K;yP{OI0t6SE?y zOFSK)s3_XefmshY@YdXl+BD-Ynic~52L}$>b|vEzM}ObD?S!-2sAtRyoEeZMyflWV zuIbj7f9d;%EmTUFUyH*|Kof`00zb;YLTePA5^%QcWgK)`ARB}!u6En_AY;?huCSR? zK@=2Z6Xi)@{I^CdlxAnJZ`woF!gi?0j4rC)DnA-*Bh%c)+`U(tc)1;`hg_UYs|GYu zBAbE@zS=E?Iu8CTi;ONh?h~cc*q3vfX>X8cgz-kAI@z10gdvjnZD);XFUP7~$HmPu z;1g}d^FDH-YZh5Whq7lSjNxAIv#}-)NRiX+HqC)&eb_UE$d(%G;JXE^YX}CzR zj}WqrGkd6&Kg@!Bq{@=jVIU>|2JUgQRk12c4U%xrRtshTDgw zLH>oiQHB8nWNWBwuZh+%llG~A)2j8N{)pd^;K}EbK3zI5%FyfRdL@vnFY+p=^E~>{ zB5-;!8x%4>KWccLFs-~;pQA$Jp`Z8iJRgi2Ibb$osxye{S>$ssvL0J74T1QT=5h&U z`WPixAB5C>*yx|T;Q!B3{n_>mEA(@z0>&K=5(|tTs@Z;H-)?qB@Nir1!8#9`D$+41 zWmKFaD_|{pesb&co>6~a3KA2|=mhzw;?rNr(i}#-wsyA!&y-Jl*(lXX{+ok5ywW;jve_J zc)|K1eyes-jqkU-Hib{SS;5++Rs(OXUY?f2qJ7H=cE@oeSdhze5;>qEUY|0Y2;8fY zLKbeG_$XWXS3sauo`W!ZT z0L9{+2xFN_2l=e1Rbqe(ftgXioiSXYkA}XlBoF^h4n{-R$q0sXv`NaOcrBlwpOn5wSg3VC#2*J9QM2OE%3p zH!ZK*c}QH=OaBC!<2NksN20{%Rq&fJt}iG!-F}4H9_j!btu-1weRhPCv5~@hGCb^R zxs%TeMO_c{Mn(TnLBpC?|vZOZ&+GlaNJ4^I66%FG?LQ0a{M65`I(#rf9scI^ZHHd07PBG z{^P+BEY@c5158y=at8yBjgec*CoQs8907uC?qUoh{Tzd{73`TRvw%~+22s%{JmXTt zV#+TxNVj)R`#gX#a?WJA{H|aAT@U{ljU6H0PqSIZ)&UNlF7Qnq_Y=GzL9XkQSnwTW z7c)8&BFWhCn|q8%4-jo9=D`%%kLCj1zdpM|6wzPM*Kz`&w%oU=-u??HjbT5*b?{4C zS~@FJZl^hv+upS5ZgQ+CtXGD1ng8*R#c-9}H3T;_u#-}n3lgzRrm>ds(`5W}6;7B*MAFOC)J>I|I(7I)CQz#IR9W(X5ELD2638nlvl@ zP_P&e+cnr(Cu{kgb3qk983vcCb2^@VO_djkUDdnz@j~<0X;SSG4Nrx+`` zOaDO|`fNf@FDH4gr8G^m@5i!{W+9pP?Sm_Bn|=tX`Gm%J4z|!>#Mi7a{dU4yHy{%qw7eY|qRx{}o2XN1ZmGWdaR+&V zaTz1tj_1UQ4KwZnD;76IEi@19Um|#55~!qMc<~iH+XCg}stgrb9|?AN@}!&&2A4iy zQoKlYi{F~0E-n%upm*MPTnzuj)sAmDCWDYr!tJjv8K0!;SYN`%K6U$GE6lJ z6-0UdGvcSn@E+yI?E*+2eA}UJ`#Y;=f{nwZ4UYAT;%Il>hI(}`qP5>oU9YB}?*w&K z5-H?-nsT}5=k#};d0t$XW(A`Y8ORBKZlmn**!xaovGki_ki*khPU*NzWJTr9HK7H5 zK@-;}>$9Zk&X&8tQVW+=M>F44J*o`i8kevUrr3@!9frLU1+ku1yG74tjD5l_pyJNt z$C_x5Chnc3&$;A>uu1DB2gthiJf{lEEWnZ-pNen?=EhsqAQB7nICxQ3l^#Rax06ZVPRR>_w9! z+7K!ISQqXPoD~`mN%7S0Et1JFKQsDjZ0`kSiz8v*p#2csdybL)m!O}zaOa?55xQsd5PT@q20v`19<~q>P zT1-;)vwZvDXW5Lkd*^1-#fG9v-Ya%j?t7NAg+GYp1tt*fx+i-4@P1W%@YT+sI;p|u z`StKcM7WNHfI?Rs2CeU;FPjJjm6bB+oSQI;A%eXqec=wNK|T7G&<;rE^tP6P|2o?S z9=g9%z;*7YT0EtHN#;NFnti={DlTeptnrq~vay@eh}gKv=!kyIx5CCX0-`@T5;&0x zPru(G2#_?A5(s4=?s)L5T||fy-pWBV@W6hAMi+QzOge4+od1d{pPBwjcw_apOou2W zqeV&JLrMO|*htR>!dYXbkqP57*l>&(XnOLD0gT{V3tTCq82UHR7DIo-;QjFy9?z=H!ABPX2jcTd){U1pF?r(mdOwkSh zeu+`d<#{tK>pf##nKv<);q1G}X&r;^>VeN`b`gk#6@j!rav<<}BwP4(dTNRc4E6d+ zV{ac7Wub#tp2YnU=%-Joa3&}P2#jlt^*y~#|0kwIG7Mw&VYmm$cJvCHWVc`Z-Rj-T zPL%P=9$&MoKV?@cXe}$<12@EP+3$R^_QT#(V72Q}g*pDt(Esx`Jb~0eSUs@g^lk<# z`ucwP3|t;@vGV8Y=OD+#L#=DRY(XWo=TGNxL(4T_;~Yv1|DH(a0>TBW)O~a9_tVv_ z?}ZVmqtoHa4yWm7U$r#c8U4@8|9BZ9U>TxLUBuWj@$}uwNu$A5CCk~;H;L1_eRA+9 z@FfvF8J}0Qz8GxXOv3qk2utyM!Z&d}(UJhoR@0P5hbmx_)G3~jarLB+2Zi>mLi0K? zfk+CgzZMRw9q-}=F z!f)0%m$QG}p5)Ol5JWc4)2Ne~H?Gg4T#7a#Rqdnk*}XCHWf%s)9F-{A*64l2bx!u5 zS6?=&HQV7XS*=&OOM_;mQqi=Jfc>iYMu`+#ykC00UbV|mxo|vv$>;svXB1jYno4a! z+n~SloK}lW(4e5^d$$qNAlL=7lF@c`b^lYsk2r2v)Ug1Nx3;fZw^fLH>PxMpN>PJv z%f&^K@38h^h!89p+AoFuuGXQ>@?9t}X|(RE?swpTfVtyWm%hd-n}g|WMW+d*z)oFd|HW3^&p^+X}@LFn-JAaQo_hT^D|UTkH0^nslH zK;moY3Gc-ft$eoF9KvO+#Wt#jD=t)(L$+w5H5rE^Jc6*D4>K~ zrYh2D_7lFwx^Y?#vr)RvDvw*_kT269)->Fye4 z=xzoih7OVL?(Xhx8B)5tyL<2%@89$P%&VCfd-mDqoE>Xj*LN}L7M1IbB0lVVCgu#l zjBeZ+&+`lZKNdjL>W>9~Srd6)MRZjV!k z8&W^VzOU{7y}>@|n*XvM{bf2CxOnE)8q+DP$G2Qs(E^>ule)`igA6q&Mn68jTF@u# z%Vm<#bu$-S3lw+z=Pml@jimeHwVxe4LNpN0yK0_5m1JE@czAoLb$G$ktSoK*o1Pj* zXij~AG4kH+#$>fsqWL6tp95sF4QN7UV#v~&9Mh>Wd65KgpI{FP##`&m`)6O5qT z4<}UfixMd!60fYhkyL0uUZMPO=>gGkUK(eo%$9GMBaYz4S&gA7;q4xg z8@0(i33oxNaO#x8k2jiC&+%B{H*vfD6Rva$I)6*2pa;}I-eZ{pP{xBa72Jv@FAZZ((&s3 zN|~mBO-tpyQ_8DEc*h^P-{R$aA9kBu;uui36!09N)-0S4Sn}Ul#(YQBHwn;%o44H# zyLopQP(~e-Yd)_s48iow<2qE*`|kx#^26-#COz)WO0H5TkbIP)vVwx+={Kt0fJ@PL znH1a^jFc zh{B0}R9{RcHk1gkaMUQ9&?Ba1_6Ku_hs9>1ug>-~iE}TN&7T-zdpV_C_?Z1^h*WK+ z=hS5!S^Pudka3I2bgi)I6}ZXU`5;Vd^S|5M>EPhXHdj$Cdb4!Q@mbHu2g^^5Q1D7yaf-AjzFH8&{J&Qp|Ta+UxCAHe1CH>BqZ1;>T( zJ;67q5D|=FdjXPRwCU)OI064H&-GZOfz0nH{wz!RXl*1#|3OB$5Rjan;pX&(udj?? z?0wV4t}S9L*)Jc{zwi=~?)U#@#>23fAKMmwSy#q&S6$M27!fpp@D6X9Y?S=LBS`PH zk|u^CB=$8sJqHQcAS`34 z5H$R`i4@}|zamUKy|GMCbpojSXr6`eSU2?*uA3XM;0}lv5R}5ak61V9EWGJ!M!j(& z&}X}o#_hSMY@-bqI5Cj1KT9@gSi+jSI6-$R+%mGdP~GLNW?T8n>^o0U&_GTj)W#(p z8Q6Qs`Jt-UA+wHbATk(%B;}QmE7`!=;gk~4EbOcT+sgQK8t;l6BD=_7Bl~v!j4{Zh73Xj;dBz7b@Wgv7t8I_g3l1}l+K&T`E%6a-Fp3r1(e^uE zJM+Krm5@EPeg0w9Z7+Q%&1sCog3ahlN0?n3Pndd#R`6@nKhR!3(x=rT9l2DV(K0bm z>oeT>b)tqMw*vHrfpfJ-=v^e%E*@zJ3o-|v%o8o7t_bCeFvE@q*EW`>dbwppiTXC8(A> zjVK(U6lE4GbN1{|QnM==C=yAmrsU&)WglV295&ubv!M2{J${ zeaIVeBKY}vEp`#EG!a%N)tN+Unhx_GcrWSoUQ&fMI6gkp9RW2nSiq)Xh?l)%|GuCE z!w^+yI_(lXA)L6KmC@(BARSA$fPqrCZbU6-g@q-&|GA>2k1r4>sh=ryM)c(((uLK> zc^RegsG>DR#9I$V4$`cSQO}yhI^CY7O<;y6RwU-In%(Ze3KRk-vCT?yK|V_o)q5#W zcKu4b@eW0)`|wYz6}bBwqrP};cI)h%h2%;wTX}bB@er3`ix`r+y3GK)js5lIA=Bfi za*pywHHZ1Vgea{S1jYA{;#DHQPQYHZStC^ZEag3xY z$^LgJK#`=Skt87-BW`Gx{zhhzgX77CHtvDCWgqP8J#e;7xa$9=dk~5{sNUZFg=Bp!q^~z({P<3p=Y}Yr7aA`*NtQ?JfF;y* z$L_UMQTrHhX98DoSnWR&pKP|O3XJkSB6sG#OqQ+XR+z?z)t=1SH?eeN$@uXP#{qg? zv;Nx1dH=|P$_uQjsS22Fa!dl?C(VtKa1R?sd5xCabcxBMNW@fg6a!L@-S2+{9%5ZX z5nwe>K{G^n>3iv}8PZ@2%o2hkZil2h_z1KWn;G|R^M<}U@9Cu#3;RAV&ekB_g|*L} ziYS;H(qoyx6i#%~Z}H1b?*O$_c30?_8~@L~sd`AU<{~kdd2F#j7$J5U)kD(iR2{E5 z$nA>|E+0Xjkh16f(9LicF$Tc|&@JS7r|jIN%V%z6u0Fo;7}dSccFeC#kU^=#nrU&}B9XICAmx~Ku(5;pHu-Ji|_20{%Zd8&YNNa}d#@T)=f9L4A>4x9}lEs;BIA$kUTMU%r zPx$<__Se?(m6Um=$7wSL6S8fRCx}8YMJf7z6%M{C*hAgA{}^-?1P`M7l<*u20)^){1mS2&&SqcdwnxfX#3-H{>xqEEzSO=H5aoh@XgVyRALKt0EdfB-nhwVa|x z+QfMew`NonBRXHBgQym6_~0U1S;@ad9OO9|fnEZ>#vGV1 zyo(&0@I5~a@g7f97RcN?DjC3tR);G3=6bF)DT^Dg4CoZwnR}SkGuaP@6~)QdqpKJd zP_@Pw4RZq@dk!3bcO%U&AK48PUk7X!q=1eWsGTm55{xv}H51=;i5&{si@AMa&#IwZ zq(m*2EML`>AwMhe7bB2jL@I_2ToNrT7xDV;yvZtk2adAM)$bQDKPO-y&%!oSb4emV zcpVEyHJs%!3~2S5VoVnCj6JTFpTG>fkpCK4?tgeY&(sks`(% z%R=liBhDS*X{5U%Y@TT`kIyAQplO{X@bcx)%GOVH&Z9^d6C(m7+^)pe2tl=9PGa_s zcp|NNo_4zj+`N<|FiN9Z9((SmC2G%>Tb@_jD_2@rR$fs3eCEy6(Bmm#(<*+pY#hnp zV1GWhmz&4;?K>=T2_Lw}UM%^ZQ5HPa*s`DPEf#I@q$fx9(rSOe?+-Wb!9xiGrSLf?^R_&_}v3_0Ocf!M28kuTS9CZMdy)518B+_9b0345rz8hIA(K-8(@>Rx(s&)$;? zrn9@jE~^8~7kI z1#Rh8n}e*#Q<%PosIZ=xmVuaB%dFEmfO61u{@k_2pI-47-#0_2k#|S%)zZ87QP1FT zH1fUvd-tX07vgmNytZ}Hlz()B8{mLCeAxK#LMqubvJ*2L$9{+-1hD<1ma$Z_pJpTa zJ!+~%I0SM`sJC>Laz#tWN53U8k92_lTls9I_M-*4{iBh7)|d#syHIPsyo?8Xhwa#i z>ji?uM9z~bQA zYPsToQDeCkopzUMiQ8m=ZA%HeQNKV}_V8qYs@`C0I>Ozf) z7)MA_Zq08I!Sr$=%`ls3GLZGApy5gPLB)>_wsEG)Hp9L5F`p7kQG zivG^-CP@)ISGT4HY@ILdiQMt0CN*c^+^1DYkowt(0gdBODvFV2$(p1Y83tds7;9<~@_o>?@8?Ftr^}0Fi}|ZUud^;V$_(#5-IMB6;cM`Z z5>5?;uvevM!&6B4Op3Wg9i0^By>|9 ze_>~-$e%$}@4VpCv4w6SduM(V3uFy)M?2%`?=M0Nt4+FAk_nUQ_>CZ1*SlnRZN*KT zr_ho&>a)6BMxReSD-%yv<$Dn>oBlkJ*E?i<*z4*;YW2+mq zedV+_E9FrhH8t!F*QQtli9dR!P7#@&QdK1F$JUCllH}0~r6}4QR4AnB#Je!J5IYz#4$SW7`d~ zz%0$g#{??d*opDTT-pN;3XE3XfLd4UWW{9ZYSraRhxUv}i|WHyCk_RaCY~O>i6>1F z3|C7WUhe(6OEnx{qG23}!wm}2%nVzCRb>*lX>H09k!Z1WSm<;Lh_@;v%dMyfa-Mn! zDS6_!?=0AOxCv60}CVnA2z_n0sirz{c!G+_JCSOe$iY%n2FG--JPcnV#kVgxB@u(6RrO zDmz-GOnMx$1URovI$weDVuf0ErhCRrre1Mp@ltQHwl=D63C>yDshd*ZQb%f3QG#w! zPY%Mp??z%V;3EvJv}A^(lV2RI=nl8ql@IeuIoq4Tnxe8KN3G=>8L>AfT!JA z{>Qhb-ZA#`f~>Si^2=A##FvYgmcGh1zu8N4IqT%1bv)hsn@j!&JrhJHjFXxEoJ8vt z4>h0U$f+<2;;R(Hm*=R%Ca?m!JQ9J^6@Ca z@94u=55Cp~-MESywKD@d)aDEBdN~rZ!bP|1hBQZ$(al%QgO@;_{Mr$uz`SS-Bdbi$ zY!(;IQ+T$9hdtQ?hi()@=v+(?q5FQGnC5WW6;AA|F&>IpCQget>IpzDI7-89pLrgc z$WZxQ$IIH%l~ZEb#v%YPS3CR~R=(>`JFi53fc-Yv?+_Vne!8_;pSEMBW-#D})9v{@ z?&72;m=CU};+kw>r8#3Cf~T2i|D_r$1p^}^IK$ktF?5RsPwG^&L0F@9Ewv`{mwymS zjDO_+P61n)Fl)w?d^6jw+The{0r}X`+tC>n@SqcjRry3#|1d}3oi{G+58%|`s*p9%oUEmpt;3nT*_0Bhu$B$ z9KeIgzo$zK3aH<|zC$E!2$+b%Pl89TSUD_cdA-9Lm5xv`iqT}37yK)X( zc4V3;B69Sxx}p{_;tecQrR{UjP*i=5AT_q|2^6ZT9o;YDw-o&|*Le8S&c40^Z$rcv zyWYlTxUzR7w?X@AP`B8lpWjqcOTJu=rmAg*euQ=1*XYLDHa3LU!vW0;)8va@OE+5A zeB&Zd$7rK2!e--z0TyP7@J%{AmA@z-7udCr>Zbg$FKt1j_RR_Z)=>!tBs<)KxZxjX z)HWn`6*rg5Y++Xd(5x|Ri_HTa31Ptk&KVrBV1;LV6$)FME>=ZlBvX}!Q;(Fsu)A|! z{iDmZ9)8XgsQNr^LYe==GOZ&r4-Imc?ayQ^S`P*k;7fG9;^lrTY?fESUi~PV+W|te z>RV>SOf#@IuujkBIn_QGCp4q!C=x2_`4&`-sNjQM+Me$(JF4Uv}?DX)BDe`N~KGLfu@0ufy#zXc=L=sHf9=| zTscP%-7`8`MURUG+XBduR08CJAW-!PvhN+&@NV}Klp~`!OXHd#0ODaegxHm$9Ymrb zf*$`t&zE?|zU_Xv-{tLOfV0(QF!*KI7+T%;MKa;MTYmLZYVwDl%ie!sE26Lv^A`iqzy@;7S>R*LNTw)oqV(J4d`52`;Uup; zkIiD;ZOqLNA#T67!439WC_@-?9%o!ewP+`eV+89hEsqq%Go52_eOTzoOp-2aYlJnN zQ-b$0Rn1Y3{}hxmB#ve}vFNmIkh5a71o|~0&)D48*zRfJ+4U-z$qsH8O-tg-8_<%u zhjZPh=_jOs8$kjUxT5gjzV1!$G2$uwAlnn3pa}r0OQ%wgRJd|@J>|?>jN_*>Z0Vv| zC(@1781x45MiOvvo-HOXkdyB4FGp80O~uY7p>npj2>G%{D zdQuiX4v4jivL2XlwQH^Ds^(I3ayENJeK+q@D}2M88qJr-x_sYH;|&)8 zBybBP1@M14#ikEH+o(aGO?=-(FZ$-*#Oj0e(7*GP95xU6!s?(9QOx-CjR;!TD{)?S zL^%36&Lb1RB0Hz~yX>Gg%!N~~0;4WK*@NA}p8FgN)oEA}ZF8&V>{EJw@OcTY?9R6T zz)+b5_M)*D0vCE)ml)Rt=f^@;QM9U}W;hOV2M%|MfkXIv&wt5GWq6!gaECgxff*1< z5GF{ue;!%Y4HBZg#g*2&aPYh_&HTPmrd?|c=a3|u#vKe|0!6Xo5#13GDfGOH*!qqX z?TWm025}IC178z>^2!@UYrc``Rwk+0{=-TkX!A6L zhf8)riZD!KId^B$uR5Pg8xFzz7l2%ov9OQ=Cp1XIj!!ISyNv-|xa_}j2mN9W>{0rT z$|P-gq$QPVTTHFaZ&(Btr`|liM}&jf1BEUx5f>x~*Bnhe!-r-X2)Br?(l6?5csGp> z0RwkEPdzx)C_Hq87BK}}K|K4^=g+Mr>`gjiN6J#RB&1(@Cl}xht!tVT^M2#4T;?Rg4R??b^OdYsscc#)2Fv z0e2alQMDf+?@ugL|5bE=x2i&5?m=FWSCXVb*Dv;8eFdXe;NU`h=EP3X>z0uu9_kv? z?By~UYXi^d(VjKKJN%g?igzwIi24FXM-Bwi)a@e5s}vjWRGP>Zz(Jp>;5X?1w@gJK zMcYYxNjYix-yaE~QG`IeE>N0xGv3YO_xhsQKv*(rQV>Q z^6&ikzytRwFAz}1Mn=dMOCo?f3n>jw<^EsWpBVm)gft|gj`QCSG$F5zY}GegK?g(+ zR{+`R<9TjDPYJ?-KE^_nTsXP9))Hy)I=DSF2{83E1J{*Zj&0^3IDE9To@uD!=z7Jh zs`^Dt5%BNQ|Fsr}^>2EO>yXe%{PzP^uaC5?I%Dg7_-Dt5lThQ!JOpMH1#+1Upf5QsE(1{O5RH`QHD#h48u*IP6h} z&j~%6Qb+$iGC2hH19vx#b3-PLEHTJi^w6IMz2#_0v*)YoI8+Cf(!TyJ;viG;c|ss( zYM~4*Kw47AFd9&ie-v_ zQ0p*aU0NfF@G&Xdl@f6!u$T@pO%p46!864l^{mt?r`_lbG#bnwA|Jr6m&1+$l3rz z?WQD))gQjFDM)S5*$sNFMmVZ3^VKjlRj@C0W|u9lba-z3k(o=iy>|vUd_qE35)&LGs`_mK#5V$5&_)m(L4G?jEL$3zcano9Be+6ZM__!`#jY0hZ!r{Otn!a{G_r zpD$C1fib?gy(XLY$iC9y1nLADY0~z9no?%QYD!VBsQZ{P8<@`H&)0Av9!QUqWL^*S zhCYTWRvRFNpZA-xfcM2F($un^gW7n&_Gzf~c12ny_lXzk#0#L~WlsKNmgW}btk5ld zB}}9#M4JeBQ7j%N3h3I8BWxWKT&u5?V?z(PMf00fX(dEx|2cr2wgz>caLtx1eD8hO z<8Knl_DiVd#dg6%t8j?ri-=nbBENjRA%!Md5ypOZ?@pM9z?kbX?Kl*@@NC+F3PSvN zEA03}UAlAR5qZcBaiy6*pAO{I3%Sm^B9CtE>~M9((Z({d_x%o*Pk(+4dYDwxV2GuY zn{{W{D$XSqCsEN>RaaUgP!HdFgVLJ!RTc3F9ggf!wwT~%NrWQ+3wZtdvyc&%m4MW+ zVJ2=7_R;LLsET4Q|6_rC>4;^sf-n+(l}csS=yxg#z%nN8+S=#KhQIV?WE_I@zlZ2= zTgc+3Na8((1%mdu7nMsQ)4g-`?r)6pxFzBvEK@p;U4L7<9E%kyTMpKG8!Z^aDfp1^X;*$aAZH84! z5OY~+5DT%YSomCoCjo|u1Bc8m-$1m1jnX_edMr1o8idqn%pjg(@=1e_ewQANR*|L} zofl<#UbEGUO=dfxlD5?NBJ~A>#6*WFOC;(_`NWAsCv&0{{PjXPfBeTfflpRWrf52y zp4rPT584{(ot|4-Zhzll;;7YgJN@#lFwB~p_~Cb@t1x1x`7yabmP2WgfPht;Q4OUq zhA+=yK~eLl<0K@Jh}wCx;bdRQ8CNNV)ef!e=j;sc@0~`LDD--m$zfhG3e)mLkTHG- z{_cE|o{-5C4rmEa#Fz&>c;y!Hqm=A<+i^Pz(nlsO6^nud+zkqhLip<>sC)eK--ofl8bg?AYdbpL6am0QOm~_DB!*CHHBuF1pcvB2%fQkh5k+3W z-+M0+#%z(6Tv1u7Y;i`kVDqe#%x)QVEMO2DMySRz3bx{w2l+(btB4@27MTHET08$@ zwfHZ0&#q z6U;PXydm7Z4A#~tI(H0_1M6mpmlb;HciwcSRuWg=HfW*x{9Z<#Y}^Zo|E@%mme~45 zm}0bf29)K}k>*i#o4ItL9)7v6zBtGUIoET$oARm#G zm-VUf1oCvJLye0Nkb6a^Yc#V~gESmT#$W<_B%Fe84^HJCOVOChY=SPr^ID%>5b;-e zrG8N}VLmQh#!6f*ic^7a&~H?9+htse7eCd~qsNw%FebAb_{pX5DG!7b(|e3ECNhYZ z<^y}(S!$IxN}vI${_OADpA^LfXDT62<>$LuD<+6UVt_1Cq?x8o{^slBSgOTe`#bBg z+dW5_hoI#O^->mmr~^`BU`Hja(;7d&619Vg9EKXD%LmG3ODD#w9}=I1p5Uj^45mXM z#8^k49q(}Wpxv!?Bf@bGlMDDmJ2Hhx)r|xS_UJ`nJ zc^j=>eiiOCcu2XI9g7x*Q`-U3@O4qFDp?X61Uu@s<;8We5n~D+>bo4D@{^nG6MtqX zJYEJ~PUE2gQ43m)ypC?S2E&I>6aJ=n{**?+c)ZqjoJb;S-jnQ%I?Hg#9S~-6l`K5A z0=5Gbk{t3dn40{zr_4^MFMC)nmpY%NPXFwc5#vMM1_P%0Q&&AzGTI%InB5Ou9yZCg z7qlaUZz-BD{?^{nt*eu2cANpPgcK=W$9&-XA=U>%5^@Q+a+IH}zHf1$AICcjbnOI* z0KI*lI+B80m-WMc4Uu*FWA|mECg)wp?GVbtlE-|80^QCZ{t#7e%EogRlu*!+P zKD;`?1!rs=Z_U56QtaU!v7_0@xA(qm2xZ97M_F4WlHBe@gE{o(x}2AbNz%ee>|@7u zPqj8jppr5pIlix*t}ObglvVd}nO0t5AvHfg;l{=WO5OvK)VFVMPf<#~e1fsJO#GhR ztU`X<>G1P38)$Zu+J`jSm=1-4Z2xBFEU$ynwl%H-_cpdTlr-qSqv$6l6=gvwmPA|wl@D7 zo_={@A^vDi&q}r`J|6l6>qN3HJU@m#%&S$X;+6v*3+dEWIsvrg#uXwJgutKaGx4&D!L)g(TNk5<%0{eoEZF+Y)T))fq9;eBk`eG{mRciS5ehY; z#RDTMJ>6=~&tzPle78)!%IC-^n>xLmNoTD`iMET+m^$#RiJp-z7K>F^furZal}%^V zoi~0b2`!Lg`ZB6aSbu)u%C8yi{Qh?uFN-Qi;&+h7n}9ova-q6a2wO#6MB3GW?yXFz zv5oJl3JD)bGu}xyL1Pe)oH&GK0<1Z)SjyVs9Fr{fzKO~du~$8%TsxEQyXpbO{(i_*w>uc za*Fy&IB}09mx@Ij*PM3AnIC(1kWCJc^)__htu(&tmJ(wesE z71Q?dqKa{H)o<<2bE9=`P8e6D%5^gWdv|5Ey10dky5NEfmB%UC$7HQj>^<*@Q(gI; zC1EhzC+0eoncxmp&kj43+0z@3(*r%kTfI6d{_#72=;3609u^lr&iN{R@7%zh?vqB# zlI4s0kygH)c#SUoZOR@^S&ZXZ#~;s*@TQ%{`snn;A3wQC!y>0wTuoCy=?oZb7>##@ z$+%yv<5}luy+J$HZ=a_pjcPt$;%oX#<4CsR7cS~XKAds2(uudXnghH))uA1Ady7;i z`5cjmuIz*Ri4h+5o@$=!s3gPqw!J)zXwX^ZC<+q*cbdEkZ+rOdB-!mcQ%w2NkCy-@S+Oz7!7-cwl5xE*K4MVf!4Y>!cY>iHE* zsFM4=Vv+am>tDqKW1knxv+t#7Gjz*>uQ|<-!-xCk$lTs*?Y$J+xX4A(K5f11WwPcT z8{=L&UVgwMDk!Uv!y1VH!lDity74V*UW_S)`3RqWum!$#ulnO9Q$J|e&0r8<_=d}XrM?`$wJkn$BrS5U z7@Hbzq^CeQGIzW9FG4APLH5fJWR3IkbYK`1VLWSiC32)0*e@Q~MFyM9mz92Ov=FSv zWV-)RBcOSzzp_IT{X2tekd1m}1Koa+p(QkamdzlN_~B>a)2E2z!FXe37vdC#9aHzx zYpup(#78|6bAUT@&Nl_dcR{7$_-TuQ_nry1EDD)fyy;$fgdq=+!jZqUC<^VdHOd&e z=OQZ#q%9=w9(FbJQh_G6D;{c?(+?eI1|GsyWaOPd36BHfA<`UG&gsP%HqmhLO5xDp zz55S7g7P-s!*!U}9MLM*R>F2~UGB(qCaeY;FdO$_Cf*Y5n9m7u_#Rzz9zJCuZveQT zdbo}>jZWoHy>@I{9ob%*hkKi?Y%^?@z@9lgP=_*W@{~Ucf7>X9+K?q@ObK7^-GsaC zuxu{0QL2wig`0NzR)X@Uon=y~xV8B9XG$#0oM1A+RGDjvlTZ#^begd&e8zIe4_ID(Tt z&iOqbf7&_hB3(8#H2=v+L=~Bxh=5SOz~d0TErLzD5^$>^9G4-(SMhgZejaSM@w&xC z7&iPPsZ0NG+H9#B5`^G!-(~38*w}!Z5?lye43o#K==I%7#C_I(&}z&W70f(*lV9+m z-HkZ~#tZb)aFL>Ae92^T=N}xs+PC}FbK>z#H&ra^jZ(hkFZ*qw5tYHn)IAfVhqi?2Czp@hF z+`0zKGr6}q9PF_t|7ej9>qhUJYy4xpUF*EkYpeYqjvejUDw~H=C zdwiXk%=3>kBCoklPAfmvN*@F=ZO5iYLPI_8@Hd$nFCqs%qMRblLR_7Bp=eKp7KV6h zQE`m~txTBNoS~29W!eD{V12J(&^<*)Xs$N?+OOY_eCnY4#ir(3a=rDxUIV}+ZsrG5 z%I>pw%SBE~#N;b`sXT!D#G<+KeLkCd;VTREN{eR~KF0IconN7y_+03TL6b*or1TD_ z=`byzn$KlTx|Vv=tBX+H%`ihN{1oE*cS@AM8|Md0wmQ9HGW6(yo_(44VVhooQRvE` za};i{o7hcqY^`>;#l|gm`)cFtP;A{<-ne-AI0?+9MepQr#@x$sE`rl*h;ACu{|qQn z*{v9-Rf0K zZ7sry@r8VD_V6hbT|HOe+#KW2boY=Cv)Pm?`>T{Q5jlzT*Il1yI&sO_{mk%UD>yuwvbW^3&TF*+EVhDE zBNOdA$ThE?Q`rpY3w60`Q*W5hd`*6M-6?aclZZP|YWG5U)>Cy?^ZTNIkB;TRyxpMt z=6@rFSo1Uq4+*SL5X3$79>Mr4Duw7LJ0`ZBR}}O;D#xGZFBXs;UsG`XUUAxg_!z=A zVs_W8@_ZV*(el*V>v~G!qQ*SKM$L-n+v-y2;!m>%hLL-J3*TPLc9W5%SL9`@=g6vm9(KL(6!4X9SARr(E6Z}Bb+ zT$Tm7es=N7Hw*SnHb0?uoKAh3H+fI?_Y|W*qPHpU>0bX1()RSj4pWlY>vkChJh|kk zGRo-@6nD$xY{ar=F>*`U|gOzVoZCf>Qy+$8b+F!cVNjZ}t2gS)f zY}XPG4?Gn2J9Y^<-hYIbW#@zg-YMP*z5n}=D_G_8oB%Bnc{sw{IP*lbZ2R&2Nu*_d z2etAsrZx8i>T6@sXB==~mY6UE)#BCVA?unhE7|r_h}-=kW~37%a*mkKr>OA4h$2CJ zCiCYN2)Vi|Ln5lsLw{RU<>ohX>*XDa%uzX^dZ*gO)pI4h2GWz4mz`Q8Nzcj1adpHs z**W5>EqpT$S_mW7=H^0FX;~9md0QrB0!xR0zCZIIQxYvck_(u9R!^gwq?ch|iT8do z2Ig;(3ZVZ1-P~W-SWaD%&K39p@mz3cO3>ACX!=UmQR`F+wI0o3Gn%`o5Zx}hc&rU{ zpMFQQuu&{oqrN!OUs@F(QsqfnV{Vvai{15Mgi$<3kt0|29)L!uGTSZ#8G^hC$A5m) zyu@%7#-mc4?k1Z#AfvzZ@JZwNN8{5sV%A&vW1XAe1#*C z0+5)w6nhAnKd_F^_XDgLmU6_raiu)g1zXBvvUi6S&w6KA!wVj#k3Nk6^J|zyE^%t& zJ^DZyc09+r74$g^Gl|{B!G4$@yozhZS2?HS&onz_h8xSe%I+zIE{;*>8|1wc*f4s^ zf;Ilkp|@8svi5t7iylu=H<4AcP??WO7Z5IFbfF5m7|da$e5Z$f`5Z8|SPyh{-5%TDA`SMqW_lD?}jUu+1nLg}ILG7;8 z3R7}uXDD^*!;KUn88E8C6tGlpNA6;?^mFH`hyADJ> zhD{~aiY(>($-c$${GGt<$&c5HN=w7QY%iJA?Je7(DEIOfXsH%&2)P|7{?vR?rtAcl3Hi~d1VmIbw{iEQ|RK5#PNJJf?!cPGrG)RZYtCqNEtz* zb2i1mie3s7LZZ_lpGsozjzus;R!+P5(;5)pG2$*A4 zNf#*16?y6b!Oz%P_@0P?DYh$ot4KEg_X$u~B!}S1ijl>tz$GJRlB$WC@AsUHhJxlz ztaw>BbIB4(9ZF!k#bdcPJ-{o{j&tx3$+%^{u}iIW#^w0u--FUfSv+)L(GP1izTOdS z0##Ps+UXzgvh$`#lW|pDqR#2$(=®#*}4h6=P4JJSg-1eb5Z=!FpedSF-!nAreJ z>HE(M@||#Sw3jg5knY*oHhf{4Iy)Uk86KR`xXN9hCX7$i{kQM7^s5s~UtfjFKu5b* z5(@7s)b1`;r&jFRoFp4`1G)gGPFmlO8g+N#SFa&!tuBDouXlaKBD6(94Qq_hvlQ>_ z?+~dOe$7+ilUR4pvdg!4>R_HDav;C0jn_$<`qx;^B57vxH=*g=ZtuLOY4C!7V4
E5(Ga>O8$~)4Sb8TOO26U)^L38;9VR zqkAxx?YRJ}X4@~TZ!iT{YrU?Pa;M2J^1!f1aA|%0%;u#OfmkYs;Z5#IQeHg$tuE8b zUD=td=by*+1P=`A*y&bh4V|Y@2px}OW33+*cH`_{?XL0Yct9rqyV+Y;1+Aw!6_0~# z$O2QOc516|VUrE6))RzY-jeZ7;kN-)S?`e7BDitgebmdn(lQqUX-qOo<0gFX_EYIG zK4H(QAaneCtfcp$p(r2+`nFU0yBtPM+B`}ah4yXWqUD1J32zk2}5n3h%q zg%hnw?T99EqVEzk?DMD)uifrieEUWVvV6obDW1^)xF}NmeG6av5)oHTTUm8&L(ujy zTRFA3^|b9OFplX*@PK5Ka;*Xr9Ca(Ji$B*fh)>6>shwm>4%ZWsl8Tr+ge(FB{wQ~8 zDBf}K;ygw~$|!tQ@h1A!8+0f@)DRU($po2{&-j15z4cp^T^BY?I3SJEjTm%DHzEQ` zhje$h^w1KL(%s$NHGp(?w{#8Z(BH*HmnkTjZTz zYOk2(6eAY>%u#x>>*}-MY5-=t0Iq7*CwTb}xVz;D=yi4Y$Pbr7GOxMW11Q-CF=`eV zeSG2&{g}Ec>^#Qo7E*=bEBI2GS#$3sW1j1hF2k-@G~+E=G| zAt$D*DE5ND(_@3gRN#G9$EP($-T|xUz#-Cjky;(OY{i?H5wF=w4ZeZqdLWsa)uZxX zCzUrNh!ZDlxyYFG+$de&c$YgxWL%K$jEv3CAgE~I9H)-^4qbfWAo3>~k%FjLFnNhS z2`Hft72TyK@CQS{l=Rbj+VF^NR!1^1q&Mhc;gnTvGWj?bXr=mJ5!6#hyvr7jD!1*4 zt-7bTrpwT+`*LZvjq!RV5p=Xo(Cjrd=+!((Ds9!#@?^hUH1I-)?D&cNpZ)V1pL~9l zdxIBc?<^QNa6t%S@JQ-c^NtgrD1@$u)xL5$Qkl4w5h%|e9ylrLY6HSMPRgS$lk0Wh zmliI8OL&x%%Qy(o+}d87d>;--uAVcH?C8pEDRw?IW#|p@AQ#3`W0*-e4pfPCfa0I^ z-uWZwU#H^6tI^S-J_|rVFX#7%T8Inv#u5`$7B)-@&w|)9yYc4%=V*mJw!bbE_YKF{zLZd1y5z)d6yn>%w1W z;(*8iJ=6V2PZ_t0;^CTWpY>sl1_F0L6>>si;}u>VTbht4fFsNfs;E0o%IqU_BeEs( zCDP=8(hV|MIaMIC9DTi=j$-U52J?8WI)a8`5V6)yOv}2L*h$VWqdG%I8{EJM|TpkPUFYO!Oj(q7BCx>p=Z>ttwNTkrP8Du#KpFDNb z&~R8jpbH~#;vu7$m>0g~VdjFJ#Z7K~&(1}vj_%t?evx{hw_VG*ZUHmkb zYCyGj{c=0SyQLR*h&lv!NCmZe4IP#??KzsPjbLu(t0=c|acQu3#DV&oFh?b??<$g>fpvZ4@BF`UkA0p_6tu!TBUtgfYpL;+)*`@0{ru`(B?4!PYdtzL zR_yu|HG$k0iC?cVi5e#(xppv@9R7R+zZWVWEvy&U;-3{gT)4pb<>`SJH!MqXIf-@g z9lfph(VndRp^M|}bXalCEMxkcf z8%E<_wYu6)IrfgaA?4mDAIi0L^Xjz({Hp3Pvkv=cH4Ac-C1*LAm=Gsz`n$-^k2^Y> z+X}$Vh(#BhCCf-LP6`R{D>Gd60*Fl#*SumVe+$oL|lLg9n%2y{z76lgq zU?7Vd=NG8T`X_lNr*oZ$54rJQvmi9|wx`Qls^@E{F*hk(LP8W?3}9GbOEOIAhU)Z zk>)i8V_%pLzxQ%m3vb#h)rbw1@}CEb}S!Kb#`A8OQfNoyCq+p%)I zvopLg`0#rTl+-Bl2be$ZSu8^PQgQ@ef{x1`oEe%kS-dQ!M6;tj0M zN7VYqlvq-e>%{nV%bti_${idx1$ULVhumatzwQk|knnLMU24GPv4%3LHsbZlsC3Hw zx#*9kM#)H7bHBc*6q##k`%G48G&~8L!>2s+uXH5-{39n84imj7U9&)bEtySV5wY!8 zfhQ0jCv+k4nW`SRFmpm<$QHM@eTGtMfN&z)>yOG6B@^E=QZ>mYs5GFtiF)Q=k&$%j zV5qPvn0oMXj#hXu0Q4h8G1-a@RQ12cH$D8EI4)u2TCb zuG>vmBZ{-o_N&rkcVH3YuBFmZGjzFMy-VSKpJz=@)%OR7C_bPOi=it7IfdK9v{*u`Zp$u*dDTDDwkuIUR=+=6cKuL}l_-YY ze(aYb#J_9QlgOWB&ORn0${?1z$$$fYr2qSDA~i0Uc~lL?qAb)-GSW0Tr&C4bP&017 z@r4zELm%yMUa}U}Z^NuBHiMq@iApTB+naIGX1=QUuq8%W?VDMls`9;FACPSXEB_qK z7XF641)Pt9I+R$Yq`EbTuS49+McaahW)k4S&~|tX5p2^xRf^Zue6Z8v54yEs{6rKh z=w*aLmnvTAHviGE=elNL>~oIMXh&B>Y0q)=1v+mPMXsbt_~4C1rl=1@-kbNic$9O= zdTSr|?_n>(Q3}(}mDqiicfhT_%PY;#WKM5s`n}idBU+Z4Kfd2tpRPz>lywo-8%4F` z1z&ClUPk!_?B@Iofz&x%D!BZ>qygV;Zpg&XsM+9dYh^jl+d`90@-tfXBs9Cf z3U4=y!{LSZv+15jy1UIh?@NlxUBYPSL@z+EvTQffHL_sNzieQK%KQ=R(M7CH34p3t zoqNV2KfbqqSu<53R>9uU7nj59koT(a)>i`8|4wag;rS1gzVh;*4L^Z# zvf3=*?-A9w5RoU6zywb>PlG=vio*ZISzG15%z2r0*u_REyJklq2IpCe6zMt$-F~Pw zH<*lyx3&8@(a_3BdBOJ?ev73Zf#B3nbiu@9>p=zj-0^!7jSpY!sYwZ<$Zjl2*xc!0 zbwX6ykMFQ&N^ktjdp=6y>)3y@)v(~eW!tUtD!bw;euR%nZF7xssK@m9hNSRP9+z-M;0gR|>&i_$Et2Y&Rp~sj3e_2omtU z*Q*B6>KY*?aa$PLc)diHnk|GRSm&cW#QnOfVxt}{F4tnD?VtZN7L)g49Yb&=cW9v- zw|u7LWYs4w^*Vh!SZAjprx$1>mEcO*n+#VM{Y}A?Kb$1(tR~%onQ3>F@uO%81~Ww{ zB-@O=W!XFT6Jav|u6apJ;`t5Apb$M78I{8)Y|AgpRl?AYYHxO+ zXT)s~g859%G>Kgb5pNNi@4abPo!k+re1ZS|jEu}%A_I5Kdi-SQ$wk0yd89a2)@Z_a z<*J2UX@8j>iEGOUV^N7P*tDOV0WTPNEi!r`%PpqW->F%Gof`A+9r)7gFHjEM>;G^P znjY0TcN<2~WZ>;$SPBV#`}~$h6YLz3+A*q_Z~thrpwHjDX(n@RJ7mbD z*+j~R`DZ09z&=b2*VD(6VR0aB-1|&BBX@I(>CFy9XAuTpWOEhaTX#u0DFNI8r?M_T zp4HCg%gmrJ3E?H7=mMW2d6U4cRW5vf8#u8qVB6OXSTX)I5Y!s;V1}g|Ms#=&v$CZp z9zoZ9$c~-MA}er`9O3{PYJa5f&D`OZp6_bt^gv zxz6`;E52iA0rm3K43#FMxhiYKtEoKQt%Xa~W`)alkP(as1LZsY7SEn^@5aZQG`#E2 z?}YjUes{~))Da$B6YhIgY-FmbwjIQ*@`$6_E8i;Sdj2WzQ0;kMy|dPy&6rnjCi3ELQWzq;1~97O&7 z{p|Zn{8Us_h6DP!8!nHVpUyGLMBv-(WUqeGWiP1h(WX4^vcwrPb_NK1(91NhEz&A; zZ6$4vyR2S5bN)aZ#!_j5!)dwT7n2pM+V07S+CN&}62t2*#Elk`E-+VGE}2og_zHy% zS(X@Ns_N9fR?r?a+yCKujd$63`*EFQmH8)pG;i!^mduv>z<@mD$lyrA-jA^>Pf7A8 ze*}8MLj-$@>%H2WQMaH4BO;DJRMTm}G^Dd#Y5UBd)F3VqcG~Pcq|!2c#Ns#~ zFO<+dCZ*(!Yx~A&ut{J1o@@Ai|GLs84?)~E@fkkDZQ+{$=6EEXgP)u1yphrKz z)}Srj%)6f2^zmP~nQD)cDEd+Ip}R2R>m?KVG|!K==Tjn<(;^u=)WMRnn~mbTZF$?S zwwtRPjZFx?2NJj^t3fAB$p@~|w@2-bSdxR%#iQ~ag? zKcZ`E-%FQdGW&_!c)X-eax!R>*>9Bsb<=nm2yDOG3picQzgi!>j=&)-?-(ej=tj(5 zJL|(^92_{^y~x2_d4c7XYt z`F?h&h_JmihGHC>_Ov!)+E;#7D#PS2LezjXIwXHb$?(p8NdY~_tB#zxT%oYN{94>C zf>tn%Z`rdqVZsua1v}~w)=2M~JxPJTYpd`&nz<+Q?mQ~Eo5$&6C;vM72xo%X9;y*v zB0Qh>kztgn*N1xBiP;RUX7SiRicmg-8njz_+u2Ps6sE$LoWzt7>a84za42w!)QElK zO?1@Y)<7uvHUBJ@+y3}gH6=gYY1HP1+-dx|1bv-lXDJDO$Mwc*bLcZp!9EGsn*C~*pDYRMaU_sz?TwxEau@?JB9xj-@ z@fVlhjoIAS8?1LnA;{1aELf3Z?s6EY@Xaj+~l;?v1_o_hF=E;2U?rFI;W7 zeT{!S=~%)f6UdjHJFKW99S*Xr%{VEFyV#DoQ1)NdJvFm++PawL_7C(dvTl=sv#lHl z)kVixf~O{$k0n2j3mx?a3q&|o6A}nsd+EzymHw&oeH(+4IM{R-beME5S5KsH6z+5v z#IX82{LH%?LEWI+D5ShMLZyodk;$&aq=2pNISFKoAA_bE>${yJ6x~M5?#ODmVgUn| zM=2n%ytz+_znZB%4F@!eIOBeD$(rqjB|8;d|BfYPP@T=ayVj6*wy#Z}F34$VNN?-n zvH>oQ5^xvk8GbCM(Yj?ttsmRPH94pp+rjv)TEEDI+Mh>bivvQn+n7aWSq1KWkoG(k z*U=6OoH^_W5S861c5GgEaGx|}nmzq7P)XW5aFt1PbTLZO*_CYj)hcd1!Z1UP)D<3i z*UtZLShaR;74Elyx#5hP3m9|f2PwCsp)3|rlkLMYqKo6s;M^}kazD(j*wG9$O5|`h zki;qH$-APSj3!z5xw;&$~j07`7%d zO6wN#;~mBov8p*#G`{i3_>?bVB?!lN#4|Rbf`K|*9GNhPQ?Lzv_{gf%HnO>^l}(++ zqv#*S9)5)-1{m(W^?$FuDmWZHe!y)-C>UH2G{LCJ8UpC1v^&zkDU^by%!R5a|v< zsW!sxaWB6UNqlLyVu>_8`Ja;8(y@|Iey8=C?=-4H{Ry8JIxOFtWqCWbUR?C8s^kTNs{4%aS=*kHYvalKr&dM=sfFOCAX2$?Z>%rqz{ zpy}O4SeOsls=oz}G|D{f+uB5ULkRV_R7uYoC;zm@Cjw93RaCV==RFNqvPtN?EGH?M zwv-jkTTi5dJQCsI*NUUMYe1C1gAoCPtSySI9eiWgU9Z9aZJ1kN2gbp{d4Y%bR#b zSpcND&)ctp@emhyQ-rybi&viTV~!YWX6$wzQOCUSYhP{lkU5hoqCTBV`P{H?U2t*w zLLk-1(1=7B_Lkd2A=;~B`%-oHq2qqMj=C~nXvr-vE!ySbhY@?1GN-Z0Tl9iFKvBJA z=zahl!$^lyEO(WnvvL=EjrROyPQjh~vW6q>G_Wnk+tFU9B;n)V$2*mW>;8QsBf@Ur zc8Ak zccafhz1xG48?kr^PVU0$`Bpa0V{h;;8|bTv*z(^N%j>!r0;O*n+iSjoSdv8Qz$5-T#?j6u)-vfAbqXg=WR@^F9+#3|X{B1m z`=>+3>oN;F`VD7$=WnVUd}&#BDo^%wij59_KK#PeJVchp+^u`lYMJgaL|#ur=ZYZ< z8>Za3H(HFCbc=a3U6tA5l!ro7J=;Rko@Li`dM}QT`=pccL`Y5_CA?=O5q_3NtXpfO zqGk>hyJE(=w$tr5JZYFJ_&QdgG4H_xV+yv)E3BE<{De|YFJZju=5{+%#=nh#OBKK% zu=w6a#FiP=&YHV;uoB9Q01WTY<JyqR)lo{mGnpWjJKDE`nuuu4}V5zsWW-n@Gw1qQJa2nFK2wR~jkEkdrYxKK;F z%gtX`QALf%Fj=8Bi2t~yKeZU`>wFvNQ@@C&v|OrDwjTktT&Lnd*YB8-&SqW8@Dcz@ zQ$hg{x$0xpt?LbqXTi#l;U?{$h-tiQ|G7W%v-KQHvK9o}{vLSLbLynL$fFIbeS*9P z?TysD=gjS0dg_%Jw{ePGrZ+3A(*&xw6e|bHrseK#^BrCFmXbGez`NMR(*+@+%pTxOcz)6;E38Ro|Eko!M`curS!NLh51y+M4fuVhFo17mOKgmbL7_yY&qGnE&;9V0#?_tLkzw@NR>tlrAGp zVr!kPe7pqxyJuEfyVDwCHL%ni-v#E`1H9cwSF~VB#%2x(Sjw{s7qo;r1(4Pjn(F!- zRvla0_twp=R=s`|6*#zYRL5(GPGJU147A#5%@oNT*6vcL>9ie4Ua5EcA2;~M&SXyB zU@|k?rKc~*%%1NM;A!;I9X&KI$~KI8EeNf&8ZGQyo#m1L!Fm(rJ!8+CIVCfyZkxXT{}E{vL@oa;6G%U|S zhxkN>g<+S=#@~9JioaR4nMa7aw~ppMwZGXL??{?E9$fVnwU4j+Hmb=Aa6RrtVCa!2-NO60V8X4WXAjfivD zTp5Ntg8j1hreS%3$8y*+{2JA6 zTW!ulX-`eFG`;&?a@9hNYx%t~)O|AQ3xd4vOPY*z0(Ou07;jeoVCQ7#A^9;j^O8Oe za>RJr#zMHTUi@DCWji1o(Q2Vdm|(~8+#+L|Y$lYsoEe+`*k~=6-=&WY5ke=6q;rPm z#I9oCX34faNmyD~=yt`KrWK?`_6)2>i(*jbP2!ncmj)T1KJRVeR)7N#QlSFohil{BW$K1dW{wnSGZ~`AAAwWHZlJ%-U;y*?G(^Uop!GQYw4l^V) z>IV(W1D@_@qvrxz@*(MDtwD!-^Feu)-*{Lqh(nk~N-&=_TF-*SbhN3@l<+lUdobEa zO-`{*Qz8~O274XsL6e53Ru-kA>R;)PQ^<1|+AJ@D_Mt|K(Sm+y-5uUjvl4hMa?mr+ zKymSYQCz|pgeGdM*~~kn*y*2G`XML(;q-1Z=)XSo7Vu-hSkGB2s=Ad+NuWj(Zz0pC z+7{)Bsil5C@e!GZVA671!JdJo8TG(lQ01jm3;?KC>L^dFQ)OslNcirpkTH6E0IQRGg!j<8^CSD6bX$bYT_&Re=(s=kzHU#4 zcX{Okz&Z%0cH?;b%4;+}J)`4@4qqm5PZZSH3!6oL_O6yT9o2h2$x zjvz!i`7)yR`dM{{FH`7Lkcv1oY+`SHgC}RSyb`9#mZeha_N)9=G{E!73HN?s$pugX(-$w5tMEVwTJQnm3jz3dyK8E1-sMno z@F>C4v6T?F$J2ixNeQo1ZsZ*c{-}uuOh_r~Vl*st`Yh+ zWK@jACYYMm8TokW9lj2}*1rwfEXZd%pywR5FR=p9{pcV+)mP_y<0XYerKG4>Cs>!5 zQ2AHN7J~<~*&E#G=UQ|@pZUh=Ik>W#hbMp3C4OVYnG7sKeae8BuDnmG$O`? zC>e-!Hm~gtr=8>TF%r?>xBsv!o_+nW=8T~IwJ?fiEFJ;94Pb86J#H{`b{e33L;=t9 zN+Q!9FwI*K*%SGCFg7p!k;*mT-u4=JCN(B0xp|WjS2Mh3@j+F?4+XqvS&_oY|MTF3 zn0Xa~4*B4018-j+riT{CJoVE`fW>J(SzLTnR0ItRCax}aE#E@r{Tk6XB+-dJ%tZ17 zrjtpZj#+fI9(*mba^1Lz1lkY`xO2NhilIWnB#c91i}yS8D>I=ozN@zmq~-hj-Ya-#)-CUK7PsHvhFwt_$|ehckbET6Bm(#A4^|8;z6(z!Aap0PxE!){ z6H@6w>pvu37hJKrSArW#lry!k@Shy_7M*HjQ&J#isM%I^_R_v`T%*00L$9#L|wx$%(KR8#cN%plR zK&|Y1fZ@V{O!>WEzqD`NUi-r6+@Ym&Mr$p(vk`}Zu`+5@bhH^2nm{MjqKXYUByK6KB}tCf^s?Vx?5<67NY3O8=t8pT;ih*>N!NsTz(aro4_gX(9vRTY;9n= z&96nqWbatearlYEdNlkzbdw%Om8Nm)ui62$UrHzmCBetwSE%SumU?oxhZ^pw>7V$l zEmIFT0Ek%xa6fw`ViwOjtkKGRbyoUmFFtB`o2B~98xYNuy|4~YURxAiTb$7hTl$Xw zd#8u$^=lDn4N?*k`mRJ;j{mKb@w5(&%Etj<9Y2Jn#v`nB&EE0{AnrTAqw4I!gW1oZ z7xzdDZ_|Z`C>125n0q@eFN~o9ZlH#x=wvp&meO z9k}i*f&W*nyUGByu8^c73Pvyhg8WJUXwm6*b9^MmUr^(Vv=J*-AO9$s z|19WP1_DAmyo-F!=PbwvJY9>!yIdH_wq)YYLRXX2uEoOL0j_MaI@>uT{`S6#u$vZyIbk-hi z@n(sk`wK7u=V5?=?S z&+;@3&8j;~lk)9bH4Kv;K!3aISL0<~0;Ia9;29bT$jRVMONufsVG*ERu}ab|TPiy& z08cy$=Vh#a{Hs}2iA#$C5f7a?TCHYD@;FC7jHx$A}lyoX3S7^C>oYMOz&Kb4ob zCO@j12Up$nba%g^qC)8CV6_Q*&6UT*O8WkA%8cj{J9M-|kI8W>~yaiaK%mhR-PlZWgnl)t^2{Vft2QPe-mxQVj z_sj}W!2r(*1Y9?Frt+7yMM^_yoMdMknG=-dE$`hCFH{>=>e6_j_^UUcQsAvLeA$Y4 zhQtroRK~`kiWcj&*z@tY6`8F^3y4bodTUIYsfte}??`gsg^%(dXfxJlKKc!ZQ;^-^%8#S;W$gK3%(2$E>CNz2 zz3)ARpt1WVrh^i)9NRA9eN?x)jI{ywvz=QXTeyug`>TR%qCbZ9Q6Zwi#1o>+T9>5#dtq<@!J$*1grFEGDDTz!750MhKp9%7xOo~jx&CX-k9nJS( zopv^)NK@hNEz}ymRN}A8fYMeHTPKES((9`%FQ$AUEp&y!|TCTt|QyS5IGBM>z5 zEv1@uO-;E>H!||@s9j$N1Ox=9pZsnk%f0$nt2Kal)_as2?JFw=nwuadei`a+uN8z+ zw>}UXV*3Ku;!zW$A+#(;O3m3LkWJC;{oKoCIyfh^zKUYTwiVb_oiaBW27sEPOipaZ z7K=zo1e;uO?yYeWod_0I|C0{jhyWSk4?D74{OAwxPHbzDc>4PK*c1dI%)_>?1Zz;@ znRL7-_Z09o;(WA-YHH)(Cs;M`(m;%$O87c;P;>h7S?Q1Y7B=<8uZHoZti+5ZYLXyS zw6tF!v(ITI|JuJ@sdwlVOBr)Iu4feu1o&Rv*;L1&!ypqjdZ#n8(q^`#bU}-{53Qo{ zPYUaTPA#kcE#c9*iDJg)&<@m4THkZF>V`cRD@JP@=Zw-^OlKK0vEwOjjuavduz`WW z^!2Ze;y1BEe-&+0)rYd`!9`kRvw=$Co4hV74PIaTEVi`(M%qX)XCtEh7?ev_Lo~JF@N9K9k(*>iP z9OELQS@b#9-$iPxeG8J>9qgr*P62E8p`EB;Gh7sLNk(gFvkpeJ_qrD44ZKnrB;rL> z|0bPl@*mb_U;CE?A#94kBk9(4ihcUzf;yDVJoK6%NY37eTg8eN4igP+*0UA85xwpB z3w`KTd=^bOLmPVA@N~o5Mo_&;FWv6Z{ytMZIMQ8CzCNF=8`MMFjIL z!2v%AOudZp4iEnyS}OuQU=}{i$DaxpB43MF{62`Fd}XwsQH(1Emq1G4lO)!galVE) z`7&E>Qdh;Z+HaWV?W>oUb(~u^d~^Is`(>M9_d;DdkC!_1Xcgr~%0BPmp*rRazo>Iw z^HWJ&|EU6qf?huYl&&#gG9eb-1l6zcU{%E;-@etHo#H{s4yPkG|EX7ju{Sk9uRpZ$ zDsbk)IX9uc_P!mcpTSU%+zrCMTa|3{rXYhEj=4g8EXY$syxh;3+3U*4zv!_fmfrQ0 z!uBE^Dr$L=9U6GIXWx?i1W4&KuFsI1d{`>XjkVEG%s;wvV3Q?(gd;r?B#_kss&rt& zWQOW>6UccqR8Fw%a3Ave3-olN(P}JS3={X?68gCSJQ&LJa$%f?2HjC^PRR47}0ImA@Q;8y#oDfWk2+CKfE}11AT8t2jN_(i@qh zoK>3@}pT02m59A|y|Q#76ROxq%IOR}4I3^yPqLD|IRYLTmJK zl%)SZEO~5@o1Cuc>xVx-Ge7Hp#w(O1z@Mr%sd(F@8P4C1h}xL|U84?y>Ds{TL|D5d&-icdJ4 z;9pBfC_zO}efO@|Ucu(SFFy?~eDbl&2V80I^4sJTkW&+{oL#QxSgc8WUoOHMy2A^QSW5p=ov5n`jvz+RF=atmLP&Mj`j0m=^D$9+9qZ8#l;t7L^A20zx(@- ze;3YtR{QEXCMvusLgvVb%r|u=)l{ozzhvgftW0yUMuQU5aKjES2!Bh3i2`xeaE#Nh z0s{Ts7W)po1w@Ivw{fU{g(CF{6lzpon2SM?UMzfl?eG+j*#L4dgc+yCzXOt?irSityhaeg;DO!RvSd)`bKo3Slm-K1R_nyH0^=22 zY|h|*`k zvYKpjyMut&1Ky$>LzsX@x>F7S+kJWY=X=sU3TI>$qd-s&~CbvLiKS19W%S zVdkW_zeoF|e)M=X^5N(tMCOzXZMST5s|6$Yzas$0Yax z+V4mTenizb&_DHW(ZD7UVL~z&V(d#52IZ-4Hm`+`)oF39IQgq^(P*^%hFo7ms=LSO zWR8>$k`k6(4gpB@I3q(>>Wiefm^pEgxy9*ZHeNy$7{lxQ11GDZ0mygx_A&N69b^ELdMQhr0UfSwhf-P}C(t#?+ zL0as1b!!C4g^XEleElB$ZRg8Dk_V3vnTss;*2$NSv0+ytm{Y!_q>Pw^K2@3WGGvBJ zX7gAh>NM=OsxVtKfj6C}ov)Dr0K9;7%2~=VKKg%V{(wp-X(KL(|&ZQ$QuKZr?&Pd>{4|C8oAML=_3lvrv+36rV!}dd-kO)lqT_DRGZbgKZnl z3{T|jSrAF2LXEe6N78@7aJoBVjCZQCF?_;x)I8q)(fbOy zud|lpNirC=mrHEGA}-?(h><-REzbNZ9S)%UKEN!9_enm1Mdgk&0X9W^5c5#dmzfn^ ztSLHUj+TW8R@RN1cPdPn&*sSM5$hJt;(so>Eef=*7`5d$y*W|#Z99{dlo95?rhi;N z1ZkF8cb}DS7gh!lc?)dq#btjALU^4I#L+%8dLFE71U>en?UKWoS3BIKuD^!~-FWQt z-eMFot?NyE3w)D7%7FST2muK!3wHCnb7$65-~RR{dX%Bd8oA5dW{Z7oKTmO_7mkLa zbG+y=nGbPrnMj7wtUW-hoUGp6ZR9&JOh4&5BLLdK^BYQ$xrKA*TO0S>B#@#N>FTr_ z9MZ5o9T2s2y~hFXl-mf4*)triSGln&EdrGtp&95m3hxA}~$9M+ijswu@c?Yre9 zY=R$1NvobTuV`PHf8JViaNw~Vwlb&5>WItD?&6O`O7tSKR5qX# zg_UGm< zY3L$q(DE*PdT*!?r{(urnb_5ZrDpz!?c>e1S&7?kv_6Rt93kD(pLb-UfG&y4Tk^#I zr%Mo2manX^hkZX(F!YPS*>mvg*RRLTJm=1sodVPh@Hh@yav(DbS1*o-&CN4nLtzmEJzaV}2@u6Wo=sA`+ZtUBXlG+}w`o)G0f!JPw z&hK=Tk9H2_3!DcvL!YHpcHFNaMBtr+>YvexpQ`))T`;5Z>Q{%i3Tcg9VVT#h=gMyE4&R09e)4a^ zelZmsq)YIzpk2+hKoncTR?6?j=^IjJtH5(Bt4Pxt(Tu2>`K!UUrbzkspL5K@yTHbg ziaUoUu1#?f^@@F13F7|1?12N!Msi2jO{@jE#$AHhE(>D&l+UJ^OKATWrXkX}x{`;; z`DGKh-Aa9Lp%|~ximOU)8ec8FiDqh^FZO0^#t6GKwKS)cb9%+o0%D_DjEwm_MuK0m z7R6Z7(V=IaTH4d&Ysv5{PLxS4)yascE;3#MY7njSERv(1z-o(|IC|tR04H5&sn42;p@ZE- z7(yA~u>%#Q`z8Mi;k%;nb!FP+Rz1LBkz5x0?7ovUMlZI03pna3dLQ(M=*pjF1&C=o zzhs}8QtmLe=$tYou})tQ6AAK{Z+k!B#Z5;i;ls?_)gPgv;c*Gow`{zuU15dYcme}1 zK-uHeOa=_$ZOej75GVUyYA#x?OGuIk78i?|OCYJmMYB)QIg~ephxXfQBx$9@1!ZHG z=nHB08r+tDP*4a4Qpc|p!7UREHRr`!$?Fz86%IC?Lv5?n$s__z}LLIM3t45?E2pcJm+QOg^-+x z=xz}tN0MJ9#G|)cf{d|^pIe)m$*0e48|N%)$Vta-JA|d0T9$~oEqnznX)l)9{BT|V zSwUVHa9|ZW-xB|->1(=h=(70CeziyAPt^1GuqP&1Po1tshlWDCN_h%B8r5itajpTsNOq4ON<$C~eJ?ELL8p;@!bSeyb7auaU%X`-tx5P_Y`m9oM zZ-Yqr(|>0HaQ_)I%Jck+;p7J`49^b+TbCJ`ni7T~MEazY7ea1&H(!w9y)QVsoe`Z8 z^+^Mgl%p?rKXCrbQ38bF1A4ZVkypTfSfipQQMNz!B}shxj9Xx$HECO1Qc7xKnlO4) zy&~n=TI)D#XJP-kN>QXAH8~*X4Y8|_m=sG}YaULr&#WFz+ko%+IY{rAy*G7tZopFT z#S4s}Fw0+sBr=-K$5dQFQXIv}g?@#>YFEmSgv9El(<|R6{?I{(1C8lno^ltS&)}sa^R?=VNj1mIyP35zT8xCGmwr=IUoZ8zklDT` zQb)+gq%6OJwV=0J+8ICRWZ`FSz|4#d`|5nQ`4(>5MTvs^v_nfsaAry#Z!!Pj6+5E1 zr6%jcX6vBK&A7{!^?Q6rQc<3*uh}JlfzY5a<^SEn1Q;sI+^H!F@=EikhlU@|g4ol_ zi7w)bY$(U7-=MMZ5HtiL8c@tp*3@uj7gi#n<+1ClyR}roSnsO3XG7Om;^g{vwy|yz zl1Ci#H0#x-cN-Q$52j;3e;Gc5VI8rsueY&j* zB>5sr-tV~o59|imZy{Vnl$>2Hxr(`|xgG+~@!IyS9&FP}+Z>x+S^f$>&n#eZaZFCf z|Lz+}RsOr5TikxUQMt5Gj?{&omtc?D4f_hm_+^SYvM$y9lav>}F@4JH>|e((`q{%P zB9OR>h4@52^}bsl3!I%Ea#Vl4T0PK2!{P$CW#sZyQriED*l&v7!eN1LZjXK(1OAADvZ>xU(@LfR9Pr0{4J<@0Fel zE4%4Kreb?fU~@ zzY3pq38;ikzNa~xcoqOr)?oC}Ct+1kjJbvLv&ql^OJ5I9z_Dm}&nx+Pp)Y;mW9H&D z4d5mT!;lgj6PQ{hEswj)9o`sc<)^|wd1yrCNnEmH`Lwqkugv+$2mUG1`Oxv^dX5I7 zlQ%A}=O@VXhiaQ8hJUEvpn$ZMe^e?;F#ax;9IxLMT`+zf4l0O7%LkxWx^k!$Hs;qm z>{A58%J`%&_x&{}64;2?4sVbGvq6!sn1C}Q7ZBTg_BRhSj?-oUl7p?KCA_iL@knF9 zYZKz<#s)EX%@+6fXYUurq9(oEwvZN5nJz`V;Z=QUSa6>(67=-1_3r$ae%Cq@E8X(|Rm7iSm?$o+gGhl&w`L zVeEbyU%&ajKP<9T!_WRQd;Sa*7}d>%3(R15-|;BMZaMQecz-Jmv_uXEU_BoP#+)~Q z!`G%Nyl1S9MdhY_Em`Fcbmx^ne0Rq` zD-l&{O%*I1xEVGe=i&&Y3|q=5E?bPD7m<^z$bV>4hq}#T;}ZL02^NmWt&U+oo2(?A z>m-%5*|g8=aj%;WzIy?>k^8V#k7Z~0KbR1h1^WMpddKiMxbJJYv27cT)7ZAr*fty6 zww=alY}<`(yWzynJMI1ZKkujcHrJUwXKSr}_QHPR@0b&1sR=<#hN#%>>N4Yz-9FkArSitgjhqPB`I}jQIq80@iJC};p=r$;XHSI3t^)|78 z$!Id_Y7_eZA_)^0xV`6=a0Rnoz#3V&Q`}+$$lLtxj%!k?u1uO&g zql7d*H565d(2%pH#u?1BM1d{!HU_PGth-@H*9;jD?O(sV4o8omdU}SgGdr3VDr*Z|goryqpu+lzM?% zUG(+YOmzPNrOayiRWW7*R+czR%C!^;BiaNO7Pht_1G|caNbpT=U6W=ib|Ac`eWGx% zqocnyHRpvts@BxgO0P_#_e~Z0&KtOSMg^MDR#o=;y1UW({)|6H;vA9yS-bCfEdxP# z4sYLLtY$BT9W*-YPjMG6eqoIOfhd9TQJsmUgQ;fE3F+qjRJu*BcwMn!WsdF^f%iKV zoi?YC4c_x#bi6~(YHFBvZHJgmoAZ23vwOIBV~1hYacxHdIs*TrWPpuf`VZI*ph9F> zMFpN39Okjx62E z(M>lg`qlPw$E7RQTRUj<&l#ki{y7`5`5yd+OZ!?pCRQ{yC}(qW>@f^cgS96Uln%O&} z-!t50>pmjTlH0y`Nc>L*g3EF{9QllK=<>k-jxr0>hOfM)hG@x(C0tca4H4RR_0M>% zmbNxr=uO+;P28MyaQ3 zSyPu}?4`oYV?bs!{Bh(xbTw_qzO_=9p-|OuIyQw=ZW+StOxgUdw4UhD0sjkM>97xm za670s6Ji972tPn<&pLSQ5QPAjLW@dC!DMD-EoPyp@P6#}FUNzVpIP+-Wv2ZF-cJ?& zaDF&;c-)%NX*G_X=OW8|fB@m$rDLGx9cT~Q-Y;p2e>4v^1%&Gr@FkZ-?43t*C=hRb zI~I^oQAw&t=YPMO_bE|OP#E!f+*~%tUFC9zh+Y@bWl`W4Y2bKCtoE>1@bYGo*9^m1 z5)RZ!PelLeZS3L2`o->=jbJ6=m+$3iI1Vh1e+`nW0geFk-2pn{0pA+s>87r; zk$VlgFWb+_wv4(afOjp%Xs)P2^qgp=JFVJOA zA?=i?7ANg%;X1XDbuD4rpBR=^3#jX`i?C`{!rB%P5HzI3$dbe`WAP@5@EM;k4;g$J zvSv5(qsCQl@>3bk54Ty)X26a8P0Rhn%tO0$fieCgP94G)^I1tr#*%br{;%uwnR>B-MEB@xW9+26qhFRq#-`m`we@yB+171qmXh9?WN%NtpL31L%u=Oll zeqWwDOl=sA(b8M{a3Zp5*doS0;$>qVmQ8cRNH|kv>PMF+mAPQ-5-2Jv!UmN=8- z=<;-F7)-MqiNOoI^80+{bUJ|kc=Y>JzeHh#L?WD!p}~RyS$xwY-)Yhk6GTx2M-SKv z_%a6Hf*a>fdcrt=H*BijwMyePb{0#Sq`ETuHa9QwLsVdJSQN)LzilS!v+gssu|8|z zV{O`T_PDk~u<`f?R0(YM%VScAgC0`BADt(MBaeNAAWBH`DIgdSD(MEH%KH83w|ckH zFC#C@Y2YvWg=1@9)?8Pk!`p7q2ikzmq>9V8%mz*Ue1V(Yg!#Wy>4krhq7N}cT2STA zB|OM>ZugxQ?QY~fry7~SBQeRq)wd%L^ATE6xAJc*Wnak@)?v&g{_vd(B4s8jf1t#+ zy)Ijz(cvS-Y1A%#WkS_2xuU5|rPBoUbj?@PeogCngAfTrv|U~noyMZU!UCb_!NT&Z z(*0`>5V8C)GC|Zc8$lhvjBa~Fzd=b*rGg#ugfp6N%l#!ToKn zqRbErzF1sgQ-7S)oTcXWLS&nRwD+&5P<(z#c$3-=MUZA0WK_-E@{JA7%hdOj+~FI& z%FHVi1@o3_$Fr209bRUgB#`v>=%Up}Ww+Zn(N7mATGfB*$Kt-A{mCO}m10SRe-f8f z4br9nY@7AGS$Byj@G={>r~h%NKXE1Q zY|12Hu5=P!(D^y5z51Z#>yNw4e_qFj4e~LSpnHWM}L(+ zU6HZ_Qj(%I>5ooG#w4RESTg&2S$AWl7?f4e-f?HD$z6BX78Z|J%~AEmTN=!eStBH6 zD%ee;9x=z5>ww|d7Z9b7ji!4b{+T72Vc-<9*}o?dpp_R z_W5!3$-caj&*$}!etbjp`4k4=!p9~^JjiLiY3sabUdL_vd_K+_X|$Y|^+}>qfCT$K z;~hr$wG!Fcs`ELAX=XAS*v=GFK6-<6?Z@)SY6kd@M^H$yt~WRx{;oBxWoXU;@bQG_}#cAO7UxWrBeaJG5m8!V%!i z4onI$y)mABFp8DriHxLm_=z*AsKN&RLCl&WUvsYuF6mv<{bE%JQNW(O#tLajC2EThr0T$QxWi^N2+d;Q)JuuNKu5<49-u{I<)tN~%9&rI z+p$En8GF%a>L6IMt!lSm)9*!9t5R*bz# za}8NYQB3R1JI3SP+YD-5QJiZfU11Gu50#POg@@FS2oinIa4P+Uc3dKUGT_>UZ~8Fn zf*j^KU&%v&!Ifp2>Z}bYWjT2(NcWT|9erXOsI^%vla~1djZ&6r8Y~Z~IRP&ZuRUP@ zcJw&$)V}B46|FW@=?FB$$As{O?x+$uB!K25B*-Mojeh6>7MnuFinKlN!MnVSBhMuC9 zUlgl;U+h6>>Eg(2;2`S-2@S%8YSFf9gV1Rm_$ooWFcClFv{UpI4CXM#s2y3lOzMzj z&|w5PlJsfX*s(CG#EI?rtt4uwpFXPJ}=VCvVpw7n8#+4Z`^zB?McEIYxe&?}wi( zH6^(tX(WlSMLL-5Ve^rCa5`xjPSPV_)W0neh4iRP&zeDUT5^rEJFce=V{UwNa|0hi zF`L6?ol3G7g~d3GXWwbKHo%u>Z#3}wmbLv>B4sY;<&LB*Gj1N$ubJidzU;^L1o;@` zKJSBH!x9b^#@ONQdNLO?m_3au$I^XQ%=kQxzz}t>yl=BU`;TP$Q^g3F<3(P+&={oR~=vNujnz{IefVHE<*kkq;v}emenBZ^s_QePEr!*pqCt*X|)Y z9Ib>;LXZ|U7EQ8Rbuo95;_Et?1$!7qc2jVUh!b-=8j#vo_P)1SkjLCNLGW)zf3%M= z>KFE6PQT-lIn+6#T$Fr%VBu{HBb@KBblN|kJM6Q3WdzZSxLYYp<#Il94}s5ml{GWz zG;+R5Sxpl^J%`CaU^ayndiX6<&1d^y1bTpxXr0@GfXB|-iun}qc-!;&+T(RdF_}Dc ztP1UDfLb#_`0SVTS^%D$hCbOaFs4?0 znwWZI{MloM&#d2}sTvQPjGD?H8a5l~tTqN<*2xaA-@5X}(~q#CX~u@ie+I*yABf=; z9=MSu&-iXn;VTtRXwuNW<@_9j<6y9ZqsX32#BwXdL2~J$I8J_R2B(lK1Xhlbf=HtN zL82;i%7H@b?TWM$LF1GOra1+neY8g6LZGwRGrR#{1Q+OBRqM&|#8F4%@hI{hPNv); zgj7~mRujx*7MeYkN1`)5sSjM#S-$7S(D4VFbL$IHhMXF$f(*2@hvQ`Wq?Rwp@)jcN%0H#RLZ(f;LYli-eF|pY!t;Vc?q^jC%dt$4GJvwhW70`^ zU{HupQ9v;){~j2QQ09RfD!Y+@#BEpqQcaA%SO3Hk+JOZA!pHArJ`eYT4k{xSl|lBW z_BU85O15b^7$`oOcqt%tQ@(0zMb>t>GuoF8XH88G_QQNQAT6j|ws>zhjL1$oYg4rY znADk^4kSOj#xhVC-mGF23#Aagno|UF@H;O!grZIjzU@E>K)aNS9j~Y{Qv{|z`sXU0 zoE8@jqB?h@JAiz>xC{!J_L1>bc``DXhxAO#=VNBx8K=`XI)w=8Q6~`E0$cwDw*@*o zWBi}Tl7QZ&je(@V$)1y<$Ok~S)1H;tsfbL3nBrH0LNzHKHqfWODOcdx;j%F?&O16# z%r@bYAr@pK9vKl4Chg4o;K49qtt*ZXf7Qki%P)p@_>-FpOWj#I__4GhLq9p4HjZFK zgG@~xeYRA^LMQ<`4Mlj{X89YOgsF<}h~@6@0GhxDAWo>K>uth3c-mbiyo|n-!Yk8= z7chtx`o{j~w65y0P7>WfV2&Bz!RFccASc~rF>DF!+pUEh)i1@RxOvkgm z_dcGP5|yo8x$Ys0Sc@eAg||}kU`h{34n>;2Z<~jjrTWB>?ShRnWrR;jhZyAJj!oSA zyM-+e(2FsdIunCZA=nZk-*tbQMF@y(G{UwharrO_3ti`@|Eg*QXYCO=Yh_% z4kf2NJ&d9sAngIl*8AF1=PZgl>YG7R_dsP*|7d1PN8otTH8X(61H>{?WRF?Ku=Cee zN&ABRKktA6UGR7EFM3EJPoMwDp@&zn>0o7h~M zf%fh9c{*hqOKqv-kr??tbpfvszn89JkM+b@#su;Vpcp#5`{H-?XWPkjWj%?*T^Y(k zTvQ3vA01B(g};&15wT_*0YpvR*_wZ7cI_c-VPx)=BkqT zmrWRi{iAd$p945g>=U1yuFs(j@7S!gEv;n=+AL-ZFfnADqtU0?<%&IIwrN?J*1D!mQRY?n3K8B#-jI9%&#v`nu_g~4L0yY)%(ma>J9}EiLCfTfrTD+t z)lLPvga@iUVma&3PmIz>9$itPO^c6Qep-B#I{$NFZYH#a!s+#*1+QNA+mE{wB{c4) zugU{Ibb<$wM*~2ZIhmjyDQbPWH%pmfTxyBjyU5}+p~hBUs;&wSIxEetsJi3GjFc{7 zMy5Zm`Sn(v`;l8p*47g)PVG$VwukYAit$h}MH}}}vB?X3r}sa)oy{0WP#*B@&F%S~ zFJR#wq69{=)f4agN+cRIxZ2;7C8$M#IeP?CaF(PCj|H-Hf3pE=?=sEhdn0!4JFcuR-w^_tYG)5H?#969! z)%|L>@nc@@2p>wxPP@g2fXC`$mf8!lvo> zk_^Pz{sL>MpC#64%9Yv){{>p!Q?O>$v@?14La1A+1}PwGId*R)!^-_7DnlfgaZkI5 z^Xe)u80*6mw#WOKk7P8L62{BCsg5UECi^$7m15XO=$EAEcxY$)*hAk&udOT{%+vF( z@C53P$%LlZOER)dfX7dqgz>{~?020t7ID8rQgr->%H6GuSJ|gb6nVzYBc`D*jHNO! z_+(I533tbd#yF*zHvJ&1g+3ozdT<_nZJq3nttCVfC zyyYi?S^NkyNMQd-U@@=IKR)V|Q*2vu92tQ|wdb7oir)-eu2xV$x-RRW`(>H&Is>?4 zaIMku!@7!3F>8B?wB+=^)PDr6xGpm}LEyGw2sr{ovhi+4h*9Cgq`ibtWfDyig%IOe z;dTt90}Zi$X{w9k+t_tnup1jo&B+uNrU55v&aq%(ple78byTz2Ca(07$~IB>M?u#LpD2{04z@u<^R)(P-h3 zZ#*J&fcJR;`JI}pRM=U$2kR}5m=(`giT-`L;ccX}pC=o>8-Qop1Y|7ZY8UaWcH@k} zKy6JidR`Y{Co+Vld$vWu2T9tj0&DjM9&>3q2;`m#fcVy}>21XC4T}HnClU@llQ~x< z7K$gbp#fx~Ud3whG0Ilb+XR;Ey53d=c|3XM1xhz@BDqrn-dV8+q)0{*`pvW1E9e=; zX^AFE9z-*V;*$*%A6Wz6%_!N2dDt{w9>xd%wTICUjzojRxhd2SNYmApEMp z-q=U&`(R=Zmrm!J+HO=zZN^%o(jj16KAU} zCMv97?mT>}N0YoWawafu zCd#h+1`0;=-=>&9^@DeWZ}Dd@Umjx1gPi5No!TISvQ{4}M1(5~rCL^8DdZcSeSq-` zLF!nsH$uNRM|_N1!mWk?Be1F3=v3%Dh^b+R zqu@uhGZX))<6N()6^h%Jn1bCdpxaM44|pYk=fFK65^yi%!=jqO=J_p6v)``PO>>TM z`$1tlWeld7hM}S!I83{h-E|_ZTng?^AW3oiIX@MQKU}-{GuDxJdLNZL9&S zT)ixpHaDx?e*F&w#s)9@dbq=FK84Ss7#wsM!?DFgo(9GaMgKBg^ay509`XZh`1?GI z`P5+7k z5giuHlx_#aevo6}AC$!$sh%W%Rq6M%+VLMX04JY?2mEfHuK|S;bEvFF~BmE`ARR*Jr>^7?tNVeB~3WtQCE*=HnfZHu z1b9;g`cdICD09&%2pzi6Zg{=y4N-d%i{^7K)&70|%|iJ}Au zy>;?#-dHAFj$6g=BoKD%+s7=F#p>qt0Izs&Wx?{nt4rNs1H+-0ke()4^myWRMp z^0&+Yg32n$`7#Zi+aG9bE=u8%t|jsBurhgpk;(FA4?YX>{McJ(+2PUXjG*si*ICPP z`>IkDP6EuoWlbN|KqnY?XU4t_ibkb@Zm;{FRu~w);>-F@$^^T6gvVSmY8MF*j4LaF zYeflK48$y@7W@~3|6=?6;a{541^fWW(zKo7;qu>#CKPu5EOrWe6JSnstp>xqK04v! z;KYQl{CGRu;i`PS5K?Wkj<$8R4^DRd878kLoGKNWs_g(h*dK<(OwPZ-lhw(%;1QoH z*l`-JS4D6Yv+dz_mSpAIvCdI&0U+7w%bgVHBigP>g`f+_AH5@1X8N;s#{9g$AHPI! zxijZ8`r3zSvQhtKegp~Lrjigyk_a-2`!`oi_xPeq$g7ilW6+h<%I6;==~yyL$&^X# zyf#DbB>ei}KM&|z{l@!J(LzDG0hAOT#3@+THYkeuB~j4WrA@5IW5fbaN%CsOy6+U? z1zU%<>>#52f|KRQ=H_59`)MnZ&!ZUD+@#}t?*#tqdYKB-01#W!o)NMLTp!z?#XW6q zuF|w_T}{SQE2uMQGU6>hRs$jS_}|$|$-C{lePHbgsl|W1eSujZ9{fI~T~)((lq=k# zYtn5tjrOqOmQzYCK8}+Ft3K($OA7w+(aCYt|*vSfonB&h3;0j(Vj5VI4{Dz&Fe$ zBUcsNlLM5zVM880Wq-NGhd2i##$=Mq{nu3gK7%;@p+}8Fi1>!Z&Gj&d<1q%#)ySZM0cb#YIoM;s zNFYwla>N)|#4SYz%rK!ATTj<3*V*~j*XIkH;HEcM(s(+40@Fa@H|)7Il9tAw`%LCT znCE%|)5%>YTl#&AQ(B&?u8;g1T)So)w{IPMzgE{Y#PVB+1wpZ}G{ef#<^h0NmsajV z_%sU4F}3hnKqJ+W^S^Hf(=E^MRoUW3}yi{9bJzxWv!A%z@M zC$q#kxRZDXov{{&PE>}n$H*?tS|rb@&v5sr+^DIW@q$0*Bwf__0eK_sMI9{c%e=v$ z)k!DTFNf9MkKsqUF-EC6=~rZfQB%mo{9riw`EJ~HdpnAr&Bu<}RJJhxQqW;?_bZri zQVStxz2o~jcll^sv4M>;%qVeLcJijZNbmd9|=zxNo`y;_UaNSFivuK!oNF@uhlAMix` zOyPTh7AnF(P0gh74Ov_t^;LN>?WNM^!VsF%b80xo6)V?O-Yjl}nxs1{Pe0bWgfH5d zSGyK&++5$7@Q?4iQ?2(r!;C^O8R7YQYS=2*)V+1Jqt59>w>iQC`a)@a`NbB!X?2@< zbMn`2Uen$v)Jc)4H>ld1b!S`D&yVXPC|Pa9BytW^j%Geyym`WvjZvM#yK7fVx7eCZ z`3|R&$>BKPZoM@ggn4q4i}TZ{ct+DB#?fzirH2PQ0h3U8iv?7FBfB#)F^;LmadI*` z+ECQbd?`X~(9{uiy+k@|bSnoTNo_yCmp-89U4q43xMWHWMwGPHHw(n3il&$O0Mj$K zWMldLMS1XaH};G;H+!sM-jEH+7c?j^C?aDT;mY70ZZk5YcO8|8T}xCpdgiT8_&XB> z_v4KOTgDU=XV5++o$30ER!Lc`nk~Z41|15cwV<+ z&ZJR&m=SU|{PpC)*6cU8ZU(0lXAYz zt4oj92Jui41M8{A6O%GIvYKtJuc;ME9^R$F!8)fFc{#KnpD9DH#q@`q+AabN6?xBh zx-~;}ws^%pd7{J|{M5nEzi?=CSXUcr586REyZw0BkL(9y-$XZ9tmA{zqM`PxLY&zl znKz2ErDN3SYl*!16v4SsKx71^{H6LO&ntmjWam;7awAbJA*h|yvFpuhpe{4w4j7mO z#$#n9NAk89`h)9&;;DzbQ}5P1!NyNuM$Ua-VLb+$4eCHtP3Nu z>cCB@a^p;fN78f{NT>sksm4>B$C+N+kOBZ*%KI%27yQbQA*4L&UN`}*=#3$M_RTK| zCXTy=TG3QdH!m#;ONJ3wf6}%G`m;SBZ3Xz9;U#F3m|aaWRnpCgVk^-$uhtevvjiA^ zKzOPF%L{G-5Y#zA<=@7cG>WvmcJ8dckEt@}t^^#WG12(Jy*QvY5!uLmU*YSR>#j$Q zYA0(5M>-j!?~}IWuVkrIdp~A3c7N2t_4wS9Co1FbjsKD~5V_9_UWOj4ku0VolnnH^v-@I~Tp{y0FA z1Vr3x0Ikpt3zDdx$ddULeo9hQe2;7YnS}W(#Sb)Z@Hnh4C)xH9A7N!~dGx&gaX+3| z1qJ-wM{B(H_)Cw}!PKmdc1S#Z?!c8NlRgRyTt?Z?RUpbimDO>B&(B_f2mkv>uMrz8=KwZ2#11ONy&J3k4uCq^i= zui!|#@5{yIyPzuS4{u{H-1C+oF85cuMIPEJ9MzG(Hnsg4EZ~FFHSpl9$U8`RtNqRKHf&g$X|J+4fpeOCrwC}cR`z(5c;Q(+ z6JKy86gBojORt0RBceZ;yv3mqlH5kAN=;W=e6(06Y5tBiB%u&LA6}u4U5)lhjY^iG zAt9sQ^-Fk8l2-d03!tRRXBl6bMqy%Ek@qTek?$rcq&`p!^su@R12)@lu|GWOK`kwV z;LFWFwbuN)nN+hG`xb+Eg|n01qCPxtx_;VT@&6qb;olBvE-FN>rQ;~;jQqx2j}c*K zaEY88PGl^Z`&b}40mqw*%Y|BO<9S@J%|{UuYEW_ZEwH8l`pz4?RWc@OTXL;B1)}MA zI!`3$F)&`6q#5-j5U;Rv)u$}n9tHJ~j+q2y>?_#y!P3AUWo4y=g+D#ueP3UoH*fnZ zN)3hxeuwEp3&wj*C7py$FqCl^6)yg%p+pz&-J5w~$DQDqz1uToBSB7VZ?7yGBaVH~ zx~hf&mTOO^(i$o5fnk9ywre>)48|6~jWwmhCB|N|hzv@X)c5P4%j#JNUK_u)Zd<0- z-&+j^y6}-$NkK)mpD6UTmHZI&_xM!g()@AvxP{a9??T2d6%vn|W}+;8~yzA0?1 zESR-+NlxDV{nCo}eU9Y(#T*9`35TJGZRr&3l{|;_eOHft-h6utAL_2B^Nmit$dk!P zvECyfA5*ufd-HVt^4S(hk>B=+TnNFTU~@hrR!*f33Y*w>nxnx6x}RqFWF=%Z^(Hr; zA9gU!rAxpM>7J=&ILnm64`g11;Vrsa9FNCL4Vg!$I(rR4tSJoYi~6@B?eA{DFv(Nl zi#4XR#0|YGOtVCoML%=v@hox9gsu}TXe71&gyhP{nl;gcx_SOi_IGmXn_!Bn_S*#%F{E@UGt#vmbJZwFn=0kTv+&)FLnK z^-x&tM+#}KXVm$D60xSQH<}e~GZy=%Sn!|0UdFVw zA2WGQN4G3Bfn!8i?341t6EU~7Ei_2`HmVVno{jbcHY5dr$odAS|(!V$Z41l-xNg*?3;;HGN$ym>iL~X;DN6K~K=ZSI-}L}}>jwxZc^C!;2B-58ygYjLunbi2ASD909FC;b1+q77o5jXATgEH}U3J8-Fj7*FOt=7D|n+fj9cFY0z->F?_^^|+sRZxArIyx@)n z#V%!ZT^(GPoBj8UPuv1|)SQth5yt>Pg|ghJF5X%ju+mvJ19FPsr1=LN$OwFw+R&}K zoGN>uY(q*)Qj&P-=%fbX#*#YU!rQE@DHQM%=jM>tC!(`E?xF$sAX0h!igO$c-%g_O zip`~aH8ABCcOBDys~Uf=SR+(T@crvI^5XvFb!xthQ#m|g^Q8*=Cu2sPuH={lp)rC+ z-UY!|E_VtG;Wz1Cu+ulAZNI^|Nb3%_sG4%MvgsL6oqPzB8X$L7h;SkYTd75U@nkY- zp^m=;NU@sV4}&tz$>jcH-{t+JRL_|Cb zKwUT>K~MLr-UOx`TaI9@|Asn&Ne-D1#LBOt!tz!-u)klp)42%TD{2tw`g)c!o7q{b z;7oCQ_<-Z7Jr5d=q}oaSXVD#kM%;VWN#>t?hU1Q3v8(mu2_bM+fFD z_YUaNgJK?fwt;xP9zHuJt`^bF(g-un)GzDotM3^N!aKU=%+Jak;(Fh}Gcq!Ux6({2 z#)8D89tk4+6SreURmPS7B29m1P6CmM0c0T6m6fdEPrx~4_p=j^PW{wq83{crP%Us% z%Ln6Hv-E@3_LAXrk>rlw$P`;>{NE@M)ZrC|ibax8u+UDa26n@1DLN{%N6e?c@6JSg z>vX=u|M2LPWnFZXl$920StZpM*e9Rohi{#F$EMn|wYP1dpgr^+yM=3kk0!DASY>my$;wjfA!_89Dr# zeVW@K;pgujKTqSS7tm6h76Rt&>F4Uk;{Ofr0{!30&svm6TTRUjvljykYrhJKm%2=L z*t7toL}0yHx)#AUAA$L-%Bf?wx;*uzLJUs#ELaB9>wtUTGsbS2RCJuCk84m2=lynA zTDuiEk0RYa*{J4a6KDJ3X?1ybZ8N5cj+xWn6!e?f(*hO}0hjz?;MA8#KdYw7-GW~% z|0^(lvxmg$sr=tR%MA^JDq`ioDJ&}GKb^~GEqVPB2*>HmFD6SYPR2hK+*#g2QtJxZ zcg|cU7c*AU;6BAb4W}oxymr@bbB489XLlO)i7x=P$_AcjuZNf%*so&rp%fg4_;rLS zqLa&Erxy}#V_}K|nV#lbH%1?>&;LeM!tsXtv zkJ{;0gg6j}0Nj3&DMB}?vuUg8`=YKEZmNGzv4pNlsb+S)P5H zt8GHH^FZdpK|3q#`P3`TwwB+0l9^=%-=TUh_dY0huoXU_bR+J zc*6csEZ-N&8%Y|=^5WTzo9wvG%rg*^#jhQ?-l6gek5=r(&gV6=yHj(J`hh!;*NruH zNy2MZu=}~F$+>1&SAGuQb?-F6Pim6LQL2Ri3V@unbmE6K6C4!h`!>ekViY#H3GqMr z5k#yvOpqAlh?yho4TFr~1^*!juxAhgmllh?*o%SNuspvSIuhyyyL;!Xz(vV{pg0&h zp{<{5Oo<{`Q~YJz0F36rymq;@bTowNbUHGs#h6MU{tG694%cXuDgaJ4hLOX{JLQQR z`p;*z2=KNp8&8zG+TgNr)O>k{iLhxUH>eCbf<=llA~)Ha8=Tz zhPGTEcj#XlP8&?y+lJ$6irRDEB7B{Xid^P0|K3H(@rh%}&^4*8uP4yZdl{58LIEwo zHEfz}pdns~ysg?$8xysT{h}FBBc}1F8{yC3ekR@6#PEMVo%jn#2Xw5`&Iwd&W2350 za$B^(&OEB@o`EIJpO+!&>oAl({pC6|GuOo9qfsU~Y|ybXirNYLxB)$^l)9=yihJf5 z42C~EU;B7r<@5UEgQcSm=7XVZPkVrs->A9Z7Z5wBWY=ggW66bSyj zq1pKKX!){&MI!TjE7A$HuK5J#=eIM7>PLe*uv^^C@R>HgNL5L9UADzeD@mfP0#!i! zi#;bn$1^zr@c4u@P-RKIKfECju{<0GQ%JjARhVR1`yz~Jc3LcJf9bAqT(7Kgtr%~+ z$RxLwDl*$3U}d9qcQ*5We$Uekk~@F3xB-7RUKQKxz9*VloIao?z*rPcH{v# zoTTZ>qIP62dik9SP_HwMv+|(bVKB{(ADtJRO^iw)u;Wzj0L35`Vf?Qf1QsXMpya^| z;d9+-UTXu&D$;xD=^CL2gHAd$cKxxbtqwa~PH54jEM#awkS3B2u+NiZTunxCKCcFH z_GI;E<5-HqB9k_X9=if%W=eo|mVTdm~>6|?iEo(WKN=>l#9ippZu{E=Ta zYP>(;I}c##JxME7Lgk-7&GbPl0^N;4&`H>xc0eYnGSVHU$<({pFYSFD0e#F}TwZ^ zyQKm?pl3Ia+->#sbv6L$4YxCLQNs1mHs|GjHaZxo>ZDQgNC?gJhZnk?11TM!V>AtB zAG%muKWt|r?r8fe{g~_o#3ugEXAo~@Kc?o7C)%g=$L@fQV!WK2!_P{JGr$Ffvx8>P z@O3?$SMxh=mf!8S2P|P$N?r+DDGFGm#bu~j{j0T3NEtIaOc`Sdr(^no<-zsA1xFDd z^B<+aVBiA+G^Gs;(?vjAdZlz@&yoyy+t5ja57bl9Bfo70r%o9AFJmb z()E0Fq;c=+Wz60b;CQ~-iaNl~;B<31HBb+Ov(oj0g1tJMc0FrFJ2%WR^-J1iAKxk3 zb*ofPlt+KfvMp;2i}_`@S8feIQZwTxehD4L#pH0J>1vs$8x)~_3;Oe!tFc?k3SWy< z(*>)_=ijJ5E$Tm*h!;U=YiX$tq}7p7Hhv8uwv?0%<0m2CrsHjvB_PkIuGrl}b4Jaa z8?mNN*H@Kfn5g+23hV_!?+ZnK0Zp7*Gj=$&D2TFwKlcZDn6t?Gkn{$aY` z+FjXhj^eESJyfGjYZ2sM1uB0F_`jCMy6_9Q6R~gJQcQRQY1?-T%;xRso6VIseMCS` ziz}7dbe_rD?SO>aZnuJd*vO;Ve#7J8E+vhNu2*g7Acq@9tX(&ed>Y|k{r8UA4=Sj5 zloc^QI3V8$WW`}YQ^B=6v=S9jRZ$HSbrN;$$Oi)kmr)I(&?1<|>y6&8)|{mL+W@?0 zJGslPtL!VC?dss3u%O(*56Sk@>zZ+b;`N$3Aeq!pK;YBHoZrs&0Fo3_g=jsn{Jt%x zFaV8NDE21F7#DK5%qg1Ycf|mXIylyWw_f6BF7oy`C_~Bak|`~qVx&goqq(P0b<{r7 zT7bPeiM%2A0Zrd59G??r+v`%3m){JMaH|!L1HwIHwEIJafA0)EhxG?4CV!b8H9qUV z{udb}Q87_Da&ulK_#WGkt?TClWzw554tZm;;>ZL3BM|FXkPsxeyuY)roo>?C$?2+v zP~f{q&t0o0*jhvF?9{S>5>3@}qjF7QhYE+Po@1frbPdNu^{3ObBd*dr7H5gjk^Ph< z2HdBl|6nUQyjhW`O@bsF(J(1`Wt-_v4#)SPkY%h)i)eL+PmJQh)DL_1Sa%#YR{8U< z(FwSQj?Z>#$kfWDo?#8dV;k52wRJ83dNl0(fcwafbgLaxgSvg)dX|zK0<3Ft?|m$- zAwV*QD$+(-59qvM8G*%HEy^=TVPB76eN7WZ(tt-vAkFO zTRsPvA=ehIWYWdsL)lyXo`DbdY+L?&zn8sJd`iiFCO2>)RgZkL`pz*mSZ@n~&`a%Z zi%>8(VlZ5gLY>{EKe%+(oDtdF`hTAV;Aqos#$j*(vGPXd5o#=p05`OuAsKq#_m zKt{z)8(e&^+k7c&sjAb1*;Nyw+#2h$GawhczSi|vs`GvUR<_3yRXi3yWFmI%r5QU( zQucjIsxmJv=B>C*@=;EKGA_fYvHN!EYPcU@i6A4>4nL1o-Ro-4<;veNp`)GoYHEX| zM!+#qocF)}8TdgA53JM%z7&>v5rWnkAMbp({&2W5n~_~|IUyRhzpA{SHuxwK+-|*p z*=JxYjlqvRXWTx3;?UM9tkUYifQtMT&tcyA>AfFBE}Qv59DBr=AmasiKr!u!VV-08 ze+|{`-+q{oOl~#^Ilc1$X2RHOd9t^#CMO}tO8*QG_plwQCw$~YZ`s|8dZ;@HL5{-T z)9IWxU&Qq?2#?9?w-LY~H7@D9p3Qb**z55|a2;4s_CuuZ^}~giBJSXX0B38D$lA}4 z{}~jRHvV!qKKEHy0q=-%#zEYqa6{d%e`uVY00`UtKO3!w^weYnj?H!hI*;qjr>$3N zz4`Zs4Ev$JS)>+i2P4!=Ug9zW(`?)@_Qfmp8Iz82gKp8)Rx{-)s zLWq*C8TY?ooxjGX`)hpRLacTyP$kKND@MmJnA_3H?6x<;2iud2_Gi~CSzv&P%Tfdo zy*6SjG|CPCHkoC^v6s?1y5;48bB`6z9Ykzy93H~)JY7<(2cyHljC9(lgNI1k1`=gU z)x=;gG#q3i^M|867;ltnxjD1pA?R9qdECcOL91@ZQytk>>GRn6B!U)^|8I|6{>{q# zp72@Doq4WFqTaT3Te&vVV!51?*D(+YWhcw++bz-}6MzHPCaai2We&p4De`eOXzi8mud}fL{@qh52i5-n!b|*&u!Xe82NLmx}1Wp8|ExNZwpw zczoM>!RiODC*t-B-xkjj+3v>hk`u0%Ss-9@J0C+}cL2*Q_XS+bxn_DTI@qePZ;#QZ zHvdzhUdCmzYQ6v9gAyW*+d$=o?utw~*9BKOmOT8lIgxUnf1G~fv|TTwD{LVwj=neob$&ajAG>=EObE2+Cf?N{x9;~V z^Ywqj;FbU<`T2pPoZI>9UfxQD1roRh)dHqdY>GepXS{O#;4ZD~P(ubF@O1TaS?83{ F1OTJcbW;ET literal 0 HcmV?d00001 diff --git a/docs/img/vertex-cut.png b/docs/img/vertex-cut.png new file mode 100644 index 0000000000000000000000000000000000000000..0a508dcee99e5ad61f72a34fca24eadf18d41715 GIT binary patch literal 12246 zcmW-nWk3{f7sUx#xy2#6eZ($Z=Q($X|) z?k?7Lj#dZ=OvDzZrrfR`ZthkDrlw=ltjrkh-Wt)-5DnAbzP|pxX__h1Dbu`6L&F_X zyq!UWSH5??l{DnUUphzn&9(JHUOpV^H##EyaP4vM8lY`mMNJz=HsA_#waHN=kOhg5NhDIXx5IX#4E&M3LU zP2;6tk1}Qq`t=w(Q9N}8AwYSHjEsyjn)X+=l)6yWq)o|@f0rp8g)&Ek z49AfSYO0E%4DIm7QH%Kz-CzAiW>n1&7dh`rTdfvZ+ui4ouO@DP#cs~jwxKY3T-@GF6#}CMZ8XSIP*Tw2yvOQ>eP@1c^@q@2O2e)BJ$eJhrevP4r0$0 zNu%~`i@R)fWqQI%&deovIy!H4dRp>CyxmEwb!y28b^3So^z<$UFgg0Pb&ZB?KF^P| zIjz60(yhn23WK#c=+YeIt87^SC;sDA1BEV>s}6p%7!?$?nn;U5UydPUyzPvq5Ev+w z47&}y@BX(nnU(X=hWSM`=$=rmy+}H0q(-jS6&E=hwmI1Hhb<;68=-WN_1k=X@AHHS zn^8xa$;wt=ribw-;myGBWMY7CO(tyl)SVO+MoDFh=5R~D61Ul zty}_cPt4;^P1DRzy}P@+DBA7%>HaxXV~8Gwm7G=ardi$x+!;)g#H~`2Yiq(=351}b>JX=t?KyW-v!kV5yY%I zeQPZa?KgrRn~Ji(c}cJhxtZ*?d&CR_?+(=3-4BK`ZxmwguaE!sr0yz>CQ!ynDBt2U zQz5J7i)N9$ZPRSVwh>6CGQInA(BQf&6~^BC_y<#FzEq=BF_+ykiUUsI`)7t7E95r8 zP@&XaDpl3%<>?>K8=`%Y*+0~EcB{m?hW!l6*+Z@(QktPLDG9aYzIY<0$%C6Aw|$;K<0)kLo)SbGiY<`&3MS zRopA(Hs{S-|9kc58t51*J{kNfv33iL%I0$z88#b^B_c0-?`kTBSKZ5Mv+ad-N#g9og=q{C7vh4q)!|S+zsn z+W`VK;;;l=T~U>s#l#G24_vO`45J#y)kZ~ZCi1btNbJ9OtcN*e{T!qm9@xql0&DFJ zITyj0(iA>WXISL2Zi;DWE|Lgzuae~eM2Kx|PR8 zC>BqzgQKO^>NSmcZEL-7CP5MozbfJ|a5WgT*q3&paoo(KOtrVSLm-e81=2xfyNn?8UxW*(96TN0ewbBs68h5VMPFeZXJ zc|uKM&6I!Xc~Vj08X-L0z1xiuD3u5} zYyY(av=i(wW@DPDhEO*ucxI)m4t{wicm39vzsgpoZWu`tT=&g(3%LVvtk+YST+p$0 zdtLH`d9*X9!R_apHaeG|citQtg5bz)_@6ER^W+i=sKE0S4iQw72Ju6wbQI2|;J)!7 zgt231$-spV)OK+Vb`|P}7tOGAomFSuj8C5a+YWLmt8kfk6$wkE{(xf=cvtFz$AMtf zXc_bUdS5Z$l4&mV;)lv^nv~dR663NZLHiED7=&4Htt3%s*`VV7M(mX?q2UvN| z*P^KLU;4oA-tHnmZo0pXJsl}|>r~!lY2(p5qK13zq2zPnkAVeUnA#te>G|Cyzkfx5 zTjkp8vnYO+j?9e8(Q|7~A6ujN(?NTGol6Q`>VH=C# za}N<2BpUO5zWWt|MDM)bI-LZGzPo&EXEHxNl6`SRB)?2H*-d~)UXqf6M?cSj%O5@V zP&SA-;(4Fx)Hir1pv_j3ual>{*JH;j!DYiaJ|Q&kbq?&^IN)?mVIgK7*`<$}8<7-j zi#x+4CcJ7t4%QsZ_IFTsrb3mXEFCXu z?x{2U1tWx>NBxm!jg`cIkEk=3$PJrj5U*&dtE9eW!}(HA74lA%lFn-%><=1Y&;8>WO*f^D`bxBztib-!ysmq zNLUOL(Z!rsC{280iw6eD0 zQf@yl4Ld@jXf|HO!eDLR>up21Mi7j66(Y!HQwQa!16~1=Lf8~xxY)_cQp)dcB1yeS zF_SIqN+PQJ?#!kN}qA#C;oY}wc zAdO1A{bx+nFZ*s|G|XzLMla#oC_NU54+@Ra{tona;@i8ASW-W#wC^gv4ZYiB-EF2* zkx)@e3w*kdkt?llvHuZTUEiZ`M|z6z72GT6C9ZW%0r?DSZq?chPhOfCI9XIOJoBxg z&fV%kS2SFpXytdrCgsnHyJ(Js^#%n2-(LHH3!kz~U8wud1Z(hFvla+OIeH6hr-={a<2tt0cOdsfKK4qaB{6ZH|8T+Av1bB zNhxsull#+ZprUTWR<iZqS#xv8S;z5)J-5Bc?H(o|%l0~X)iGZ& zFI1b53ar1A&K^UfmzUPHCgR?alnZnATJ|uo`IGqbV^12T-UTAmzipvXvz)Xe%PRsW z`KnH{Ji(nHds zc486gM%qAtBA2kb7f>QKvT?8Lu&RI?XBNv$cxi~G{Nm3m^5@cHc2@>JdzU>%tE2Eq ze6UveU6{cHg9IRKXwzhTn}f~y>=#`{v$qwamlr7E+zEu}2pqkS-1HPX z$!Siw;o=RKsr12Ia|nZoBWi?GEZS8>-e5_Ru^PQ5Yb;7O z?yF9nAx8qf3A1g@DKFxM{87{kvVhUrQ!!**r=3BS))qiie{42xvc{G!_zPM35u?&a z%we2cG^iA5gGNdF{eBC_~%ah8EHf!4%w4FpOmW8wK2W;XpMcWkl)bwH>>F%m^G6(2aHyG1`0R%) z22#F$9sMnZ$P&XYBqWr}?m?b}!&7PNNz%l_-h&3|v?+H3{}h;6Yptdf@p`|LTIFmPd z+9e(96Ua)s8OJA#;zQpZDbLi1EDM1t-~cnkWItF{#8eLp=ud~pzbzEQ!l zyro^Qnyme}!0u<2aav|Lp#!a%3&^{SN_e_*#CI5ClW zbPH_1+xznDFA+vLdeo*5`z_XK>hMtY*u+9eB7OPVsaD6Vx=Wj_vle_MC(BuAb4ACL zoGPh*t0qlepZ5tv_RTdv)RIgnFQMW`LWWo|#*-}}Fs3=t%^cZO#at9Y&LX|D`KMn?~I81m1N{eN!#tcu6{cf=sOL+akDvA(w6bD z0MAZDUTqMx-kaaNWoyhCPU~{dh8^tk+fP?TUxdP}C?;)_#f~^##WZ<3j+s!^B*SgW zRVDRR+5z*inf9}iZ*sY%5$v`? z-uJqFn|5;em>9+@?~x)T)JQt z(*6)GwW6tbYY!{=OA(q>2IZgDjdXHW(&X90EoP9yv<_VWqRfNKXs5^HKlG3DnMnGr z#wPjem6vcz&{l5ypNknsk@cP^@RfaNjYsSTx5W#$Q%@`oP#fHEosi9ceR;QlNwA8N zswt*FnUBSP5LiuS#2)Ch%&v*ak>)mTyU~)d$!{lir^eI4Xe+Q>@t&>A^vV)qMb~n& z#?dL1?We8C)1jLwuv&+Ozts@i)E-5pY*}yA**Z@3+uYa`AQ>c|+UClz@OyUp0&zm} z#5~v<4QTSEWci@1GLTP&f@}kru+15AqqviA6(wFh)*datEhK)GpqZTD>Pnm)7}0j_ z%7UM)otwv;K}z-+ZkE(fK*@wgV3G_}X;+(`uC+7(BC_!I2S6Hv`+xgB+Q%xo1Bz67 zp-g%A&m>0_IG)_jo8vIF>LhZq0bRw9)c`Rn66Td8K6^(z3ENmyH7 z$*-+(j?jH8CIX}`$9w%|vw0o~-yKUmkC(myA}Q@ZTI3--@P`^yXA=g(Hs22TmsfkEdHohLW53hk z@p~SNY)xW#CQoD)m_+sVM&jeSVYaDxuk(zx08;a`3QREZ!o$kIbwY$;oRLl&-_^V3 zmjy`$eJ*Gyc&xEHujZAoA7!BmHC?&6Zz918j(~*nfnR*g&G^|HdRrgY^K_WwLW)nB zcfO7!svlL?en=kK>n=;NASBD#0PJ3oMR`~oP&UzrgRdt4O-<8{O-=ESwmxNDDZl&p z`Cr{g2bg}{2iy3tELUl>@_#=3xb@t5-b10ovDQcuqGS_pVY=ND@P+QZc9jCWntvlR`m#`GB%;&ms0R%sy3~`Dl(Z+LQ8`B!*c(l>IhaW2aba2YafO>GH)4;dKB zd-Ydi$i$kB*&3sg3&#fk!(sqIvW#Od^5OSaRkCSn@|OZLa$ysr)FE;R3puVd&_yqj zMUZBS`SVXsPq)aXPAWdmz^%AzxbA_8WJ)Dpv~nB|1=%F%iBCEQ)qoe@Vrzz&c>Sa= z1X-O1V>WgYo+x&cc^n+&m9dnk{}GFEB^;eF<$kxjPKy@mVvF#8I+7sW1F51N7nNnW zz;!ex{k`5ik8wLd2+5eu_ewJ%&-_9aFlCc{QD;S09@BfMxB&f_KBEtGWR@mSW%tqDPCYpEp%M=m)#`Oj|G70 zW)hd?^cn46*Zqlr^uR?QAS#~bGFe)T=<5w;VOpgI^rr-3dp-VB)H`&*5WelyEL#W_|fr z@-n6zTgx?isg&Yh1nn6^^H%}L`OYp>Cz07fkO$x;@9db)Q8Eyj4Ow;1=Lyu zGN-x~WR0oKZjE(cjs2Lhy!NihhhK`3aakC7adra7wIX2(EAZwO00`#Q5V)S_mWSEo zR^L>X=eyT&4?9Q^x@Pj&B>YrED)hkl&htkkcj%_%6HOfNl=gGtN&BAD+wdpI-D8o} zWJbcmUYuV9&pqWqZYdubq(n+;8zEsq=JmF6zUV3qdI9V=35M*y?RJx05iL@rGe%ad zBA!AR7g*T*$b*3A9ud{XBQ!Z-4kVU5L~(SjbNw7SAI{UUO*-mwu||fcS@!YUU%d?% zcNh^n<_nV$bBAU?r34|b*rE9Airzr^3K1@b3y<9O_AsOm2<)bh+b_?k0Nf-@4C!M3 z)X;Ys7_*wo;^haHBm5}g)V#-8tyP4TJ$6E!o%mUfWDzWkAJr9g%ba`*bk%|fbW5cM zst>1k9Q@id^$}wtknNO0g?oUiJ%U7{mg`+}B>C;u z?4tUArzP5#2$8-H;07zteD;Y~X#iz0R5Tpc737x1oZP=s<{lr-p(gBT{PT9D%rrwb z^+*rl`ZLDzFB64NcVnUJMDz3{bT7`$Q`ZIryyG$=_PzZxyH7{e^`>jhc0J)@_;b_& zSL(%dty^u(h!?Z=2y(9cy=nvJB2TxVbOzuJ>SoLJe@fmt|JY3ZkNma4X)2@g{ld5? zD4cu7F1|B3xa;NSWK~o2s2Yis21=&yRT8B)%Y={#lB5D*fjN?vBkT0Uu<#OX@;#UI zjwG#-=&r&sY*C$yOI~76)><&fI64<9uUro6Ra|P(1;<|mJF;IVhIACKTVFtp(J0dq zhHGbKOtyOH(Wo|Gc~tK!tq>x{CwxGl6rpFazU$6}S#b&bF{U@N*|tLo#0#zHP&6+L zq)2o{)fD{zj<)7N%6qGisPqj;Wl^X8~59s_1Tkb2%4^Wina|B#);8o+ArYF1!by25Z z4&z*^v?`GR$ltoxAl^h#IBP8$917y$)=09Gy8UDE@@ncgDL3dsumr0^sXBECA9BY~ z8%WG@HdhpeVVW5k63PQ`!VwSO53jE%l+i|!ti{v40k)~@Bll0XCv+L7LFFGlGV8z^ z2*HPnl3jL?tejAG?A@tzQtAS4ay)Aa2JDxJOC*n9fp~yr+i%+^Qs^}7UaJNIA|4U_ zjK!-l1jGLkOG{D5-B^0bNBTUy*=8CmhA@fKRmnrHy`17Xn@yqXc~W{!)Mxu;2DA$0=aTFGiV-I z#-g%;Zu=g*%ne2%2wmvL2As*vfr=fJu#(@0vg*uuWXur#VALj{qV-xF?RlTy`MWG< zK;GOZq;w;JQiNq1E||kej3r+4OWe0GNlOJQGT~j2%~dY%FH*|3ypJe%f6+UXL+$`! zA|ge|2q&+_`I7;u8ymPM7=d5SSCY|_Ba9ywd1{beR9;9VbF=E!WA(Rnf+DR^2j!dyAWEz=6~ZWgv?77=e&+?LsEtTG_lkz*u_IOjrVs z(P9}oQNZXDbl*qo1Po6{*gikjiiBBaN1CFFN4XwycTpKa>$GR`(s8%{zgAg7#^Hy{ zz0pw=K?dl8E?~-;+Uo5oFwRY-7Zucf?<@bQWbUO;VL(aml5bQ2@h__i0FO#jVi`G= z)2;2@WMw|(`=5RLNLbAa(aBHyb>8(?#qbtxWe6@u^4ZD1hQkJbsLb(G55HPw76!>f zg5X6_?_riYr&3^Qjd3by>i4?RYWNjiJHSYbt2=c{81*_lDs9Ej_-LVV^(MXt#F0@e zY}j?Sy%LvyM`(=S^SQ+hSO)x5r3tkKh)Q_B?HjCVZZftn8(!=^?dbN>c?H_=#E!S!}PO!!F+wQ_OUY*hI*^@g8alRZv$~BNo z`VY|N7M7}#>)T!_Y+w>>vqzrDz%D(&Q2yuucUY|GzqlgvKdx;f1-hj6?`j1sJZaqu zwSH4kQ7Urq7s$oH9ET`0z?)VoWA-)=X7H6s7^*qun1W~P``XFn4b+|d6owo1dRLi% zwS-j9Gi6vGDDF>SFql*vsvKTI_51#AW|(XNO6e(&p}AHnN|Trf%APi^-Ug1mp(1k>9{ z_>(ahkn4^jKj6I)$K1dv=56IxXr%%@uAy>BzNtK@INxW`isQx z&9{Y9f$cAZQj{MH(Qv;zHfg3mk5~cLxHs9ZS56a4V^r(s+c1R&Z+I<2mY&+f`u@*% z&|)0x>FnX^Wx6j+89#b{QCTm7)TQOrZ+T$CW4p9Gp_8mu0Hu6yB6t>&UQ`8Ul#z;L| zgpzGiiW)c%14nqXwX3NtP3Djq?L{bTRO2p?29HdxZ?Z#q|IBlaun0HP#%n`-HH~NC zTRURb)he+kToi5>vb^~DSau#7m_RZ@H;yU^V)n7QZYPFhWxkBNWN%6TZuA+s{i`u? z?WcL)wIl{RBE?#XS-1wwq4tkF+XFDnv5}3)IMnXUAu=>=-29!D_f58yezTC-bw-6(z-&ezYI^I?5#Yc;<|>+2;Wb}eifK2JSQW$U>yV# zaQdFfkIDMCfY7y*CeK1^Q){#Kz#L1_=FJ{6uGV3b%O=!u)5=N(@q7zjp!@QE$|t9u zY}4Q|>^-nLlbNuEd39P{u%%KbuyW+~sa2nN{50T7&lY-goxvYDg}K_bDSXwVepJ8S ztu_A`=-b`RKt0N35X}qe|74t@S~gts*hYfFm@t?X&>3<-9dYuPoPp(R#otMOkCcV7 zQl6*dt!=Y0dd!KJ*vKcB=^PF^v3UrE0M7lt^K)!Y=B+Agk3^6}iP;oDL{U|kcceN! zZl45ph|Y?B@jBwXt2?5)pGg1jzhPt(@)s||(bf0AAZF>`$lE5)6yTtaiXqF9Ohtb= zm}puW=nOp!kGBP@M$_3~jx9CHNwKE6amQQHtz!i&0o2UV#~ZkFtBpzEke3{1hoeQ- zn3r@x%WH8WimeR?us8m>bL>g6!GP%qwgnl?x*51p1QR5LWUVFZA3)(sM{k3?9WtSZJ zNYdCd5dQgYoxsxW-&(aC$=f=Yu_y;ry_Ct%M6NgAxN=fWy!3!Npb`3P`T_7VVhZ#p zYmAjDsY#sV3FFQG{Aq@>~e`z z5de>%QHW{2Cpzv1i2NKd&}(vS6HA7Pm^Kip3P!I0ZYpUcsX35mvWfOo3IaTRQ=ny9 z+(Pjp$DKDj$gX!JECmGzw>mGA1P<+5ztJ|Z=`OiLg$$hVwoqocgjv@ctuX5uKYS?Y z27IEEY==6tu_-W_IzNm!D0&qLdrb4YY&l}q6$ZSLJWg7Dek1-R(@aYvlBfAVlE@)x z*Dtr$usuaNi~C)TifEwy>-Tb#rV%J2~ zYDJ;RB$(UQe=f1akAGB?7J(0trg7uMU@aSYQ*HnWfCd29 zzd?4N2f{u+!$2gViTG)exdKD&SVaNFBFiV>Xyx169W1JP`T6rChY+IR>*M99yT88} zjS99|*P~PwxI)0a}gv~oAz(6pM-%!6@x|~zth@c z19=+uuY588Ld|lO3jj1sI4joPBfl#cf=`8_V3INq;r-2j+yuDk36M?UDpcA$0fzgz z45US7;S{DI;49K-BzBhJ1pzl?()eqHAY96%fcvXU$J)A|^5A2MjP&#aA)_-uViCjk z1V4sYUr`4=CZ{uNj{NPXCw)$qqjJj)zGT^$Z>M@2{{Snc9%enf?{w+88_`5pnv*uP z^XG*#H$W0bh_LcFSO)ja)uuDK2GhAb8D5x{zb}a2C!~+j)q1QuwIQ}kW#on1klD+3g#q;3PVkO&wW)2uLwZc)IM^rGN-{7lqTcbty?*O@ir1*PJp9kFZ73)6JQs* z>mE(tB)IPjULI{`TbrlwaFs>O03>A{yT<<3P+lFy*#9dnsoGRHDwR*LbGkCXuTU4M znJ`VbgHaM-ziR|er>pYkrZicii+DOPiXW!3co3)6N^e$UCSc-QG4^TIZ(Skc=0j)) zNpG+V+duzO{6UKbR%Ok=v|AN|Nx1UOa(HBk2MEtq4i}b6a0yc-1t)%ZUJ>9;X*Xtb z^Hi-6x8T$AJtdsI{AM-?lJE!3?s6g{uyMg}9p^L(N%P)_;T1imYz00z*%g)*Xk#=U z^F^8~b0jZCuEr`b?w0Qh_)PEct3l4%1~CAAzn4HXg*RcJ1^v4fhHOy~@+jL>8+hd7 zJD+pQTx-<%y488}h%t$9fSeR8mt*y*!^3U>r&`B!cOXJTLYUhdV?Ul8BkDy&Qt`=} zwKPnu(VKK?z`S&$ST?SPNVmt)l=KPcNW1I>X`?=`g7@ERfC(l9>-o+AxQZ+~_ZAwb zahxVgQYkT7Dar{^`I$kT`u;l#)Sv8gJe;;VsL}3J_q)aJJKzwx_Xh?&SIjW7dLaLE zt;%2f1laCffb9-LHT*BUph{(JMQ;J-Fj4_ummR15_^^Pm@*ci?NmShlIPJSV%sO(~ z(~MT>3o%9O_FgAU^-ITm9gt_C_ErYpPkFaFm>tGe#PJ!`fwcxhEMJP@J`VZ*ui!Ym zfWqhx>&7=%LQ+__*Xi1~`wEX9R&{N$pq5l6KWgL(H%0j5`0|bx`fjyEg$wQ_EvK^#>qEh>GiSLb4Jj{}rgdNU=k=ACH`c zOzZ|g!1M{hex5Uf-H6=z`)*{(u^SX=t*n%~?EiI&j~YfV;n#7$Ip8WWkBnKx`|0VY zS^9=LsuONrA~LJW&XdLskQdWsmq4If4`vn+_*%4p-dni9CFp;1Vz)y>L|wJr z0RAaRyGs74GV$e&-orm2ScVi6S5-Dj9rOlBRMxTh%MK&2T1B~JL~h6s zS6@}{(Z2`KZ%0bgs4R(5xQ&X6@T#`Sk88j#XIA*SRBO-#P=iC#6#Zu|7X-o6OvN{XwS-^)e)V(PPQCr8t(Ht8;B$*#z;nhNgZc$ zg^~;s^|>#d-nC!6AT&#?{D?R4aPAR7+tp^A3U0wO%s; z78APO=3-+T$YQ0o`|ab+^TRo+bK!Sb(R7YL)7fUHQjrqC|5r~|n>OK*DNZHyP)tRj z14r&6g22ZumNqFP@f81$@nPxER4wb~klq5%Sq~V2NPi2Ylwz*3easgXn)Bbcp34PC z;AJuAxptzKDd&K`KcW2Be!IH5!fy*dF9}BP#In}yL}_O8J7r%`81<2ZqH%}>{|@4Q z2nX^%2ry>#;?Fqyyv0D=I+q=os}-*l6uXuoRFa3Gp-2+Y*HmVffKvJ5n(XrD Date: Tue, 29 Oct 2013 21:06:29 -0700 Subject: [PATCH 158/531] Updating images so they render correctly. --- README.md | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/README.md b/README.md index a54d4ed58780b..a68c690b1bd10 100644 --- a/README.md +++ b/README.md @@ -26,7 +26,10 @@ execute graph algorithms. Unfortunately, these systems do not address the challenges of graph construction and transformation and provide limited fault-tolerance and support for interactive analysis. -![image](http://docs/data_parllel_vs_graph_parallel.png) +

+ +

+ ## Solution @@ -43,6 +46,9 @@ the PowerGraph and Pregel abstractions in less than 20 lines of code. Finally, by exploiting the Scala foundation of Spark, we enable users to interactively load, transform, and compute on massive graphs. +

+ +

## Online Documentation From 4f63b5e17f60c8b8d87027a91274428007d65263 Mon Sep 17 00:00:00 2001 From: Joey Date: Tue, 29 Oct 2013 21:31:12 -0700 Subject: [PATCH 159/531] Adding code example --- README.md | 43 ++++++++++++++++++++++++++++++++++++++++++- 1 file changed, 42 insertions(+), 1 deletion(-) diff --git a/README.md b/README.md index a68c690b1bd10..ba31ed586db76 100644 --- a/README.md +++ b/README.md @@ -1,4 +1,4 @@ -# GraphX: Unifying Graph and Tables +# GraphX: Unifying Graphs and Tables GraphX extends the distributed fault-tolerant collections API and @@ -50,6 +50,47 @@ to interactively load, transform, and compute on massive graphs.

+## Examples + +Suppose I want to build a graph from some text files, restrict the graph +to important relationships and users, run page-rank on the sub-graph, and +then finally return attributes associated with the top users. I can do +all of this in just a few lines with GraphX: + +```scala +// 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) ) + +// Parse the edge data which is already in userId -> userId format +val followerGraph = Graph.textFile(sc, "hdfs://followers.tsv") + +// Attach the user attributes +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) + +// Compute the PageRank +val pagerankGraph = Analytics.pagerank(subgraph) + +// 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) + } + +println(userInfoWithPageRank.top(5)) + +``` + ## Online Documentation From e1099f4d89362cc907cae0c64cf518dc99567099 Mon Sep 17 00:00:00 2001 From: Dan Crankshaw Date: Wed, 30 Oct 2013 15:03:21 -0700 Subject: [PATCH 160/531] Fixed issue with canonical edge partitioner. --- .../apache/spark/graph/impl/GraphImpl.scala | 24 +++++++------------ 1 file changed, 9 insertions(+), 15 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index bdf79bf9f0fba..d7cabce34fd0e 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -270,8 +270,8 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( .toList // groups all ETs in this partition that have the same src and dst // Because all ETs with the same src and dst will live on the same - // partition due to the EdgePartitioner, this guarantees that these - // ET groups will be complete. + // partition due to the canonicalRandomVertexCut partitioner, this + // guarantees that these ET groups will be complete. .groupBy { t: EdgeTriplet[VD, ED] => (t.srcId, t.dstId) } .mapValues { ts: List[EdgeTriplet[VD, ED]] => f(ts.toIterator) } .toList @@ -447,7 +447,6 @@ object GraphImpl { // val part: Pid = edgePartitionFunction1D(e.srcId, e.dstId, numPartitions) // val part: Pid = edgePartitionFunction2D(e.srcId, e.dstId, numPartitions, ceilSqrt) val part: Pid = randomVertexCut(e.srcId, e.dstId, numPartitions) - //val part: Pid = canonicalEdgePartitionFunction2D(e.srcId, e.dstId, numPartitions, ceilSqrt) // Should we be using 3-tuple or an optimized class MessageToPartition(part, (e.srcId, e.dstId, e.attr)) @@ -622,20 +621,15 @@ object GraphImpl { math.abs((src, dst).hashCode()) % numParts } - /** - * @todo This will only partition edges to the upper diagonal - * of the 2D processor space. + * Assign edges to an arbitrary machine corresponding to a random vertex cut. This + * function ensures that edges of opposite direction between the same two vertices + * will end up on the same partition. */ - protected def canonicalEdgePartitionFunction2D(srcOrig: Vid, dstOrig: Vid, - numParts: Pid, ceilSqrtNumParts: Pid): Pid = { - val mixingPrime: Vid = 1125899906842597L - // Partitions by canonical edge direction - val src = math.min(srcOrig, dstOrig) - val dst = math.max(srcOrig, dstOrig) - val col: Pid = ((math.abs(src) * mixingPrime) % ceilSqrtNumParts).toInt - val row: Pid = ((math.abs(dst) * mixingPrime) % ceilSqrtNumParts).toInt - (col * ceilSqrtNumParts + row) % numParts + protected def canonicalRandomVertexCut(src: Vid, dst: Vid, numParts: Pid): Pid = { + val lower = math.min(src, dst) + val higher = math.max(src, dst) + math.abs((lower, higher).hashCode()) % numParts } } // end of object GraphImpl From a4b8ddf41708d1601ab2820e67efc18c7d983c09 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Wed, 30 Oct 2013 16:07:05 -0700 Subject: [PATCH 161/531] removing unused commented code --- .../apache/spark/graph/impl/GraphImpl.scala | 62 ------------------- 1 file changed, 62 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index bdf79bf9f0fba..13f9c531892e5 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -80,21 +80,6 @@ object EdgeTripletBuilder { } -// { -// val iterFun = (iter: Iterator[(Pid, ((VertexIdToIndexMap, Array[VD]), EdgePartition[ED]))]) => { -// val (pid, ((vidToIndex, vertexArray), edgePartition)) = iter.next() -// assert(iter.hasNext == false) -// // Return an iterator that looks up the hash map to find matching -// // vertices for each edge. -// new EdgeTripletIterator(vidToIndex, vertexArray, edgePartition) -// } -// ClosureCleaner.clean(iterFun) -// localVidMap.zipJoin(vTableReplicatedValues).zipJoinRDD(eTable) -// .mapPartitions( iterFun ) // end of map partition -// } -// } - - /** * A Graph RDD that supports computation on graphs. */ @@ -105,9 +90,6 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( @transient val eTable: RDD[(Pid, EdgePartition[ED])] ) extends Graph[VD, ED] { -// def this() = this(null,null,null) - - /** * (localVidMap: VertexSetRDD[Pid, VertexIdToIndexMap]) is a version of the * vertex data after it is replicated. Within each partition, it holds a map @@ -136,21 +118,6 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( EdgeTripletBuilder.makeTriplets(localVidMap, vTableReplicatedValues, eTable) - // { - // val iterFun = (iter: Iterator[(Pid, (VertexHashMap[VD], EdgePartition[ED]))]) => { - // val (pid, (vmap, edgePartition)) = iter.next() - // //assert(iter.hasNext == false) - // // Return an iterator that looks up the hash map to find matching - // // vertices for each edge. - // new EdgeTripletIterator(vmap, edgePartition) - // } - // ClosureCleaner.clean(iterFun) - // vTableReplicated.join(eTable).mapPartitions( iterFun ) // end of map partition - // } - - - - override def cache(): Graph[VD, ED] = { eTable.cache() vid2pid.cache() @@ -213,13 +180,6 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( new GraphImpl(vTable, vid2pid, localVidMap, newETable) } - // override def correctEdges(): Graph[VD, ED] = { - // val sc = vertices.context - // val vset = sc.broadcast(vertices.map(_.id).collect().toSet) - // val newEdges = edges.filter(e => vset.value.contains(e.src) && vset.value.contains(e.dst)) - // Graph(vertices, newEdges) - // } - override def subgraph(epred: EdgeTriplet[VD,ED] => Boolean = (x => true), vpred: (Vid, VD) => Boolean = ((a,b) => true) ): Graph[VD, ED] = { @@ -529,28 +489,6 @@ object GraphImpl { }.cache() // @todo assert edge table has partitioner - - // val localVidMap: VertexSetRDD[Pid, VertexIdToIndexMap] = - // msgsByPartition.mapPartitionsWithIndex( (pid, iter) => { - // val vidToIndex = new VertexIdToIndexMap - // var i = 0 - // for (msg <- iter) { - // vidToIndex.put(msg.data._1, i) - // i += 1 - // } - // Array((pid, vidToIndex)).iterator - // }, preservesPartitioning = true).indexed(eTable.index) - - // val vTableReplicatedValues: VertexSetRDD[Pid, Array[VD]] = - // msgsByPartition.mapPartitionsWithIndex( (pid, iter) => { - // val vertexArray = ArrayBuilder.make[VD] - // for (msg <- iter) { - // vertexArray += msg.data._2 - // } - // Array((pid, vertexArray.result)).iterator - // }, preservesPartitioning = true).indexed(eTable.index) - - // (localVidMap, vTableReplicatedValues) } From d513addb77916fba78ce321936fe5b54385991d4 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Wed, 30 Oct 2013 20:05:29 -0700 Subject: [PATCH 162/531] added lineage tracking code --- .../apache/spark/graph/impl/GraphImpl.scala | 70 ++++++++++++++++--- 1 file changed, 59 insertions(+), 11 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index 13f9c531892e5..016811db3660c 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -109,7 +109,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( /** Return a RDD of edges. */ @transient override val edges: RDD[Edge[ED]] = { - eTable.mapPartitions { iter => iter.next()._2.iterator } + eTable.mapPartitions( iter => iter.next()._2.iterator , true ) } @@ -142,6 +142,64 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( } + /** + * 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("eTable ------------------------------------------") + traverseLineage(eTable, " ") + var visited = Map(eTable.id -> "eTable") + + println("\n\nvTable.index ------------------------------------") + traverseLineage(vTable.index.rdd, " ", visited) + visited += (vTable.index.rdd.id -> "vTable.index") + + println("\n\nvTable.values ------------------------------------") + traverseLineage(vTable.valuesRDD, " ", visited) + visited += (vTable.valuesRDD.id -> "vTable.values") + + println("\n\nvTable ------------------------------------------") + traverseLineage(vTable, " ", visited) + visited += (vTable.id -> "vTable") + + println("\n\nvid2pid -----------------------------------------") + traverseLineage(vid2pid, " ", visited) + visited += (vid2pid.id -> "vid2pid") + visited += (vid2pid.valuesRDD.id -> "vid2pid.values") + + println("\n\nlocalVidMap -------------------------------------") + traverseLineage(localVidMap, " ", visited) + visited += (localVidMap.id -> "localVidMap") + + println("\n\nvTableReplicatedValues --------------------------") + traverseLineage(vTableReplicatedValues, " ", visited) + visited += (vTableReplicatedValues.id -> "vTableReplicatedValues") + + println("\n\ntriplets ----------------------------------------") + traverseLineage(triplets, " ", visited) + println(visited) + } // end of print lineage + + override def reverse: Graph[VD, ED] = { val newEtable = eTable.mapPartitions( _.map{ case (pid, epart) => (pid, epart.reverse) }, preservesPartitioning = true) @@ -342,16 +400,6 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( - - - - - - - - - - object GraphImpl { def apply[VD: ClassManifest, ED: ClassManifest]( From 003f8a505d4c7e8697065705052b012ea78b4d43 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Wed, 30 Oct 2013 20:06:54 -0700 Subject: [PATCH 163/531] Removing potential additional shuffle dependency where an already partitioned RDD[(Vid, VD)] is repartitioned. --- .../scala/org/apache/spark/graph/VertexSetRDD.scala | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala b/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala index a64eb6a99cf8f..28e84f6539c28 100644 --- a/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala @@ -618,10 +618,14 @@ object VertexSetRDD { def apply[V: ClassManifest]( rdd: RDD[(Vid,V)], reduceFunc: (V, V) => V): VertexSetRDD[V] = { // Preaggregate and shuffle if necessary - // Preaggregation. - val aggregator = new Aggregator[Vid, V, V](v => v, reduceFunc, reduceFunc) - val partitioner = new HashPartitioner(rdd.partitions.size) - val preAgg = rdd.mapPartitions(aggregator.combineValuesByKey).partitionBy(partitioner) + val preAgg = rdd.partitioner match { + case Some(p) => rdd + case None => + val partitioner = new HashPartitioner(rdd.partitions.size) + // Preaggregation. + val aggregator = new Aggregator[Vid, V, V](v => v, reduceFunc, reduceFunc) + rdd.mapPartitions(aggregator.combineValuesByKey, true).partitionBy(partitioner) + } val groups = preAgg.mapPartitions( iter => { val indexMap = new VertexIdToIndexMap() From 09ea661bbbb4c4e46daf99e016ef7efe04af37c9 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Wed, 30 Oct 2013 20:07:26 -0700 Subject: [PATCH 164/531] removing completely unnecessary map operation. --- graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala b/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala index 2295084024f9b..313737fdbe1a9 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala @@ -55,7 +55,6 @@ object GraphLoader { private def fromEdges[ED: ClassManifest](edges: RDD[Edge[ED]]): GraphImpl[Int, ED] = { val vertices = edges.flatMap { edge => List((edge.srcId, 1), (edge.dstId, 1)) } .reduceByKey(_ + _) - .map{ case (vid, degree) => (vid, degree) } GraphImpl(vertices, edges, 0) } } From a3ce484a2c01eee05a272715d53cdba7569bad5f Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Wed, 30 Oct 2013 21:02:21 -0700 Subject: [PATCH 165/531] Adding additional type constraints to VertexSetRDD to help diagnose issues with recent benchmarks. --- .../org/apache/spark/graph/VertexSetRDD.scala | 31 ++++++++++++------- 1 file changed, 19 insertions(+), 12 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala b/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala index a64eb6a99cf8f..b737a9d0e6476 100644 --- a/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala @@ -190,7 +190,8 @@ class VertexSetRDD[@specialized V: ClassManifest]( override def filter(pred: Tuple2[Vid,V] => Boolean): VertexSetRDD[V] = { val cleanF = index.rdd.context.clean(pred) val newValues = index.rdd.zipPartitions(valuesRDD){ - (keysIter, valuesIter) => + (keysIter: Iterator[VertexIdToIndexMap], + valuesIter: Iterator[(IndexedSeq[V], BitSet)]) => val index = keysIter.next() assert(keysIter.hasNext() == false) val (oldValues, bs) = valuesIter.next() @@ -222,7 +223,7 @@ class VertexSetRDD[@specialized V: ClassManifest]( val cleanF = index.rdd.context.clean(f) val newValuesRDD: RDD[ (IndexedSeq[U], BitSet) ] = valuesRDD.mapPartitions(iter => iter.map{ - case (values, bs) => + case (values, bs: BitSet) => /** * @todo Consider using a view rather than creating a new * array. This is already being done for join operations. @@ -255,10 +256,11 @@ class VertexSetRDD[@specialized V: ClassManifest]( val cleanF = index.rdd.context.clean(f) val newValues: RDD[ (IndexedSeq[U], BitSet) ] = index.rdd.zipPartitions(valuesRDD){ - (keysIter, valuesIter) => + (keysIter: Iterator[VertexIdToIndexMap], + valuesIter: Iterator[(IndexedSeq[V], BitSet)]) => val index = keysIter.next() assert(keysIter.hasNext() == false) - val (oldValues, bs) = valuesIter.next() + val (oldValues, bs: BitSet) = valuesIter.next() assert(valuesIter.hasNext() == false) /** * @todo Consider using a view rather than creating a new array. @@ -296,10 +298,11 @@ class VertexSetRDD[@specialized V: ClassManifest]( } val newValuesRDD: RDD[ (IndexedSeq[(V,W)], BitSet) ] = valuesRDD.zipPartitions(other.valuesRDD){ - (thisIter, otherIter) => - val (thisValues, thisBS) = thisIter.next() + (thisIter: Iterator[(IndexedSeq[V], BitSet)], + otherIter: Iterator[(IndexedSeq[W], BitSet)]) => + val (thisValues, thisBS: BitSet) = thisIter.next() assert(!thisIter.hasNext) - val (otherValues, otherBS) = otherIter.next() + val (otherValues, otherBS: BitSet) = otherIter.next() assert(!otherIter.hasNext) val newBS = thisBS & otherBS val newValues = thisValues.view.zip(otherValues) @@ -328,11 +331,13 @@ class VertexSetRDD[@specialized V: ClassManifest]( if(index != other.index) { throw new SparkException("A zipJoin can only be applied to RDDs with the same index!") } - val newValuesRDD: RDD[ (IndexedSeq[(V,Option[W])], BitSet) ] = valuesRDD.zipPartitions(other.valuesRDD){ - (thisIter, otherIter) => - val (thisValues, thisBS) = thisIter.next() + val newValuesRDD: RDD[ (IndexedSeq[(V,Option[W])], BitSet) ] = + valuesRDD.zipPartitions(other.valuesRDD){ + (thisIter: Iterator[(IndexedSeq[V], BitSet)], + otherIter: Iterator[(IndexedSeq[W], BitSet)]) => + val (thisValues, thisBS: BitSet) = thisIter.next() assert(!thisIter.hasNext) - val (otherValues, otherBS) = otherIter.next() + val (otherValues, otherBS: BitSet) = otherIter.next() assert(!otherIter.hasNext) val otherOption = otherValues.view.zipWithIndex .map{ (x: (W, Int)) => if(otherBS(x._2)) Option(x._1) else None } @@ -384,7 +389,9 @@ class VertexSetRDD[@specialized V: ClassManifest]( // Compute the new values RDD val newValues: RDD[ (IndexedSeq[(V,Option[W])], BitSet) ] = index.rdd.zipPartitions(valuesRDD, otherShuffled) { - (thisIndexIter, thisIter, tuplesIter) => + (thisIndexIter: Iterator[VertexIdToIndexMap], + thisIter: Iterator[(IndexedSeq[V], BitSet)], + tuplesIter: Iterator[(Vid,W)]) => // Get the Index and values for this RDD val index = thisIndexIter.next() assert(!thisIndexIter.hasNext) From c430d2e21d4f614805b6a4415055ca9f07af4c2f Mon Sep 17 00:00:00 2001 From: Dan Crankshaw Date: Thu, 31 Oct 2013 07:56:29 +0000 Subject: [PATCH 166/531] Added bitset to kryo register --- graph/src/main/scala/org/apache/spark/graph/Analytics.scala | 6 +++--- .../scala/org/apache/spark/graph/GraphKryoRegistrator.scala | 2 ++ 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala index e45d7748d97c8..a67cc44f6ec8d 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala @@ -273,9 +273,9 @@ object Analytics extends Logging { logInfo("GRAPHX: Number of vertices " + graph.vertices.count) logInfo("GRAPHX: Number of edges " + graph.edges.count) - val pr = Analytics.pagerank(graph, numIter) - // val pr = if(isDynamic) Analytics.dynamicPagerank(graph, tol, numIter) - // else Analytics.pagerank(graph, numIter) + //val pr = Analytics.pagerank(graph, numIter) + val pr = if(isDynamic) Analytics.deltaPagerank(graph, tol, numIter) + else Analytics.pagerank(graph, numIter) logInfo("GRAPHX: Total rank: " + pr.vertices.map{ case (id,r) => r }.reduce(_+_) ) if (!outFname.isEmpty) { println("Saving pageranks of pages to " + outFname) diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphKryoRegistrator.scala b/graph/src/main/scala/org/apache/spark/graph/GraphKryoRegistrator.scala index 29ea38ec67fdf..13edd8fd1af9f 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphKryoRegistrator.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphKryoRegistrator.scala @@ -5,6 +5,7 @@ import com.esotericsoftware.kryo.Kryo import org.apache.spark.graph.impl.MessageToPartition import org.apache.spark.serializer.KryoRegistrator import org.apache.spark.graph.impl._ +import scala.collection.mutable.BitSet class GraphKryoRegistrator extends KryoRegistrator { @@ -14,6 +15,7 @@ class GraphKryoRegistrator extends KryoRegistrator { kryo.register(classOf[MessageToPartition[Object]]) kryo.register(classOf[(Vid, Object)]) kryo.register(classOf[EdgePartition[Object]]) + kryo.register(classOf[BitSet]) // This avoids a large number of hash table lookups. kryo.setReferences(false) From 51aff8ddcf3faa5a596a07f914de5da36d44e731 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Thu, 31 Oct 2013 01:43:50 -0700 Subject: [PATCH 167/531] Adding logical AND/OR, setUntil, and iterators to the BitSet. --- .../org/apache/spark/util/hash/BitSet.scala | 82 +++++++++++++++++++ 1 file changed, 82 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/util/hash/BitSet.scala b/core/src/main/scala/org/apache/spark/util/hash/BitSet.scala index 0ec002b5d0edd..69b10566f3399 100644 --- a/core/src/main/scala/org/apache/spark/util/hash/BitSet.scala +++ b/core/src/main/scala/org/apache/spark/util/hash/BitSet.scala @@ -27,6 +27,70 @@ class BitSet(numBits: Int) { private val words = new Array[Long](bit2words(numBits)) private val numWords = words.length + /** + * Compute the capacity (number of bits) that can be represented + * by this bitset. + */ + def capacity: Int = numWords * 64 + + + /** + * Set all the bits up to a given index + */ + def setUntil(bitIndex: Int) { + val wordIndex = bitIndex >> 6 // divide by 64 + var i = 0 + while(i < wordIndex) { words(i) = -1; i += 1 } + // Set the remaining bits + val mask = ~(-1L << (bitIndex & 0x3f)) + words(wordIndex) |= mask + } + + + /** + * Compute the bit-wise AND of the two sets returning the + * result. + */ + def &(other: BitSet): BitSet = { + val newBS = new BitSet(math.max(capacity, other.capacity)) + val smaller = math.min(numWords, other.numWords) + assert(newBS.numWords >= numWords) + assert(newBS.numWords >= other.numWords) + var ind = 0 + while( ind < smaller ) { + newBS.words(ind) = words(ind) & other.words(ind) + ind += 1 + } + newBS + } + + + /** + * Compute the bit-wise OR of the two sets returning the + * result. + */ + def |(other: BitSet): BitSet = { + val newBS = new BitSet(math.max(capacity, other.capacity)) + assert(newBS.numWords >= numWords) + assert(newBS.numWords >= other.numWords) + val smaller = math.min(numWords, other.numWords) + var ind = 0 + while( ind < smaller ) { + newBS.words(ind) = words(ind) | other.words(ind) + ind += 1 + } + while( ind < numWords ) { + newBS.words(ind) = words(ind) + ind += 1 + } + while( ind < other.numWords ) { + newBS.words(ind) = other.words(ind) + ind += 1 + } + newBS + } + + /** * Sets the bit at the specified index to true. * @param index the bit index @@ -36,6 +100,7 @@ class BitSet(numBits: Int) { words(index >> 6) |= bitmask // div by 64 and mask } + /** * Return the value of the bit with the specified index. The value is true if the bit with * the index is currently set in this BitSet; otherwise, the result is false. @@ -48,6 +113,21 @@ class BitSet(numBits: Int) { (words(index >>> 6) & bitmask) != 0 // div by 64 and mask } + + /** + * Get an iterator over the set bits. + */ + def iterator = new Iterator[Int] { + var ind = nextSetBit(0) + override def hasNext: Boolean = ind >= 0 + override def next() = { + val tmp = ind + ind = nextSetBit(ind+1) + tmp + } + } + + /** Return the number of bits set to true in this BitSet. */ def cardinality(): Int = { var sum = 0 @@ -59,6 +139,7 @@ class BitSet(numBits: Int) { sum } + /** * Returns the index of the first bit that is set to true that occurs on or after the * specified starting index. If no such bit exists then -1 is returned. @@ -98,6 +179,7 @@ class BitSet(numBits: Int) { -1 } + /** Return the number of longs it would take to hold numBits. */ private def bit2words(numBits: Int) = ((numBits - 1) >>> 6) + 1 } From d6b512253231f43c7a7e6eec45516bf1c233bee0 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Thu, 31 Oct 2013 01:44:24 -0700 Subject: [PATCH 168/531] Switching to the @rxin BitSet implementation for VertexSet Value tables. --- .../spark/graph/GraphKryoRegistrator.scala | 6 +- .../org/apache/spark/graph/VertexSetRDD.scala | 77 +++++++++++-------- 2 files changed, 51 insertions(+), 32 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphKryoRegistrator.scala b/graph/src/main/scala/org/apache/spark/graph/GraphKryoRegistrator.scala index 29ea38ec67fdf..f858797df851a 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphKryoRegistrator.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphKryoRegistrator.scala @@ -1,5 +1,8 @@ package org.apache.spark.graph +import org.apache.spark.util.hash.BitSet + + import com.esotericsoftware.kryo.Kryo import org.apache.spark.graph.impl.MessageToPartition @@ -14,7 +17,8 @@ class GraphKryoRegistrator extends KryoRegistrator { kryo.register(classOf[MessageToPartition[Object]]) kryo.register(classOf[(Vid, Object)]) kryo.register(classOf[EdgePartition[Object]]) - + kryo.register(classOf[BitSet]) + kryo.register(classOf[VertexIdToIndexMap]) // This avoids a large number of hash table lookups. kryo.setReferences(false) } diff --git a/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala b/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala index b737a9d0e6476..52d0baea4bdd2 100644 --- a/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala @@ -20,12 +20,9 @@ package org.apache.spark.graph import java.nio.ByteBuffer -import java.util.{HashMap => JHashMap, BitSet => JBitSet, HashSet => JHashSet} import scala.collection.JavaConversions._ import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.BitSet - import org.apache.spark._ import org.apache.spark.rdd._ @@ -33,7 +30,7 @@ import org.apache.spark.SparkContext._ import org.apache.spark.Partitioner._ import org.apache.spark.storage.StorageLevel - +import org.apache.spark.util.hash.BitSet @@ -167,7 +164,7 @@ class VertexSetRDD[@specialized V: ClassManifest]( // Walk the index to construct the key, value pairs indexMap.iterator // Extract rows with key value pairs and indicators - .map{ case (k, ind) => (bs(ind), k, ind) } + .map{ case (k, ind) => (bs.get(ind), k, ind) } // Remove tuples that aren't actually present in the array .filter( _._1 ) // Extract the pair (removing the indicator from the tuple) @@ -188,7 +185,7 @@ class VertexSetRDD[@specialized V: ClassManifest]( * modifies the bitmap index and so no new values are allocated. */ override def filter(pred: Tuple2[Vid,V] => Boolean): VertexSetRDD[V] = { - val cleanF = index.rdd.context.clean(pred) + val cleanPred = index.rdd.context.clean(pred) val newValues = index.rdd.zipPartitions(valuesRDD){ (keysIter: Iterator[VertexIdToIndexMap], valuesIter: Iterator[(IndexedSeq[V], BitSet)]) => @@ -200,7 +197,9 @@ class VertexSetRDD[@specialized V: ClassManifest]( val newBS = new BitSet(oldValues.size) // Populate the new Values for( (k,i) <- index ) { - newBS(i) = bs(i) && cleanF( (k, oldValues(i)) ) + if( bs.get(i) && cleanPred( (k, oldValues(i)) ) ) { + newBS.set(i) + } } Array((oldValues, newBS)).iterator } @@ -224,6 +223,7 @@ class VertexSetRDD[@specialized V: ClassManifest]( val newValuesRDD: RDD[ (IndexedSeq[U], BitSet) ] = valuesRDD.mapPartitions(iter => iter.map{ case (values, bs: BitSet) => + /** * @todo Consider using a view rather than creating a new * array. This is already being done for join operations. @@ -231,8 +231,17 @@ class VertexSetRDD[@specialized V: ClassManifest]( * recomputation. */ val newValues = new Array[U](values.size) - for ( ind <- bs ) { - newValues(ind) = f(values(ind)) + var ind = bs.nextSetBit(0) + while(ind >= 0) { + // if(ind >= newValues.size) { + // println(ind) + // println(newValues.size) + // bs.iterator.foreach(print(_)) + // } + // assert(ind < newValues.size) + // assert(ind < values.size) + newValues(ind) = cleanF(values(ind)) + ind = bs.nextSetBit(ind+1) } (newValues.toIndexedSeq, bs) }, preservesPartitioning = true) @@ -271,7 +280,7 @@ class VertexSetRDD[@specialized V: ClassManifest]( val newValues: Array[U] = new Array[U](oldValues.size) // Populate the new Values for( (k,i) <- index ) { - if (bs(i)) { newValues(i) = f(k, oldValues(i)) } + if (bs.get(i)) { newValues(i) = cleanF(k, oldValues(i)) } } Array((newValues.toIndexedSeq, bs)).iterator } @@ -304,7 +313,7 @@ class VertexSetRDD[@specialized V: ClassManifest]( assert(!thisIter.hasNext) val (otherValues, otherBS: BitSet) = otherIter.next() assert(!otherIter.hasNext) - val newBS = thisBS & otherBS + val newBS: BitSet = thisBS & otherBS val newValues = thisValues.view.zip(otherValues) Iterator((newValues.toIndexedSeq, newBS)) } @@ -340,7 +349,7 @@ class VertexSetRDD[@specialized V: ClassManifest]( val (otherValues, otherBS: BitSet) = otherIter.next() assert(!otherIter.hasNext) val otherOption = otherValues.view.zipWithIndex - .map{ (x: (W, Int)) => if(otherBS(x._2)) Option(x._1) else None } + .map{ (x: (W, Int)) => if(otherBS.get(x._2)) Option(x._1) else None } val newValues = thisValues.view.zip(otherOption) Iterator((newValues.toIndexedSeq, thisBS)) } @@ -406,19 +415,19 @@ class VertexSetRDD[@specialized V: ClassManifest]( val ind = index.get(k) // Not all the vertex ids in the index are in this VertexSet. // If there is a vertex in this set then record the other value - if(thisBS(ind)) { - if(wBS(ind)) { + if(thisBS.get(ind)) { + if(wBS.get(ind)) { newW(ind) = cleanMerge(newW(ind), w) } else { newW(ind) = w - wBS(ind) = true + wBS.set(ind) } } } // end of for loop over tuples // Some vertices in this vertex set may not have a corresponding // tuple in the join and so a None value should be returned. val otherOption = newW.view.zipWithIndex - .map{ (x: (W, Int)) => if(wBS(x._2)) Option(x._1) else None } + .map{ (x: (W, Int)) => if(wBS.get(x._2)) Option(x._1) else None } // the final values is the zip of the values in this RDD along with // the values in the other val newValues = thisValues.view.zip(otherOption) @@ -456,10 +465,13 @@ class VertexSetRDD[@specialized V: ClassManifest]( */ val newValues = new Array[(Seq[V], Seq[W])](thisValues.size) val newBS = thisBS | otherBS - for( ind <- newBS ) { - val a = if (thisBS(ind)) Seq(thisValues(ind)) else Seq.empty[V] - val b = if (otherBS(ind)) Seq(otherValues(ind)) else Seq.empty[W] + + var ind = newBS.nextSetBit(0) + while(ind >= 0) { + val a = if (thisBS.get(ind)) Seq(thisValues(ind)) else Seq.empty[V] + val b = if (otherBS.get(ind)) Seq(otherValues(ind)) else Seq.empty[W] newValues(ind) = (a, b) + ind = newBS.nextSetBit(ind+1) } Iterator((newValues.toIndexedSeq, newBS)) } @@ -511,17 +523,17 @@ class VertexSetRDD[@specialized V: ClassManifest]( // Get the left key val a = if (thisIndex.contains(k)) { val ind = thisIndex.get(k) - if(thisBS(ind)) Seq(thisValues(ind)) else Seq.empty[V] + if(thisBS.get(ind)) Seq(thisValues(ind)) else Seq.empty[V] } else Seq.empty[V] // Get the right key val b = if (otherIndex.contains(k)) { val ind = otherIndex.get(k) - if (otherBS(ind)) Seq(otherValues(ind)) else Seq.empty[W] + if (otherBS.get(ind)) Seq(otherValues(ind)) else Seq.empty[W] } else Seq.empty[W] // If at least one key was present then we generate a tuple. if (!a.isEmpty || !b.isEmpty) { newValues(ind) = (a, b) - newBS(ind) = true + newBS.set(ind) } } Iterator((newValues.toIndexedSeq, newBS)) @@ -554,28 +566,28 @@ class VertexSetRDD[@specialized V: ClassManifest]( val newBS = new BitSet(thisValues.size) // populate the newValues with the values in this VertexSetRDD for ((k,i) <- thisIndex) { - if (thisBS(i)) { + if (thisBS.get(i)) { newValues(i) = (Seq(thisValues(i)), ArrayBuffer.empty[W]) - newBS(i) = true + newBS.set(i) } } // Now iterate through the other tuples updating the map for ((k,w) <- otherTuplesIter){ if (newIndex.contains(k)) { val ind = newIndex.get(k) - if(newBS(ind)) { + if(newBS.get(ind)) { newValues(ind)._2.asInstanceOf[ArrayBuffer[W]].append(w) } else { // If the other key was in the index but not in the values // of this indexed RDD then create a new values entry for it - newBS(ind) = true + newBS.set(ind) newValues(ind) = (Seq.empty[V], ArrayBuffer(w)) } } else { // update the index val ind = newIndex.size newIndex.put(k, ind) - newBS(ind) = true + newBS.set(ind) // Update the values newValues.append( (Seq.empty[V], ArrayBuffer(w) ) ) } @@ -592,6 +604,8 @@ class VertexSetRDD[@specialized V: ClassManifest]( } } } // end of cogroup + + } // End of VertexSetRDD @@ -633,18 +647,18 @@ object VertexSetRDD { val groups = preAgg.mapPartitions( iter => { val indexMap = new VertexIdToIndexMap() val values = new ArrayBuffer[V] - val bs = new BitSet for ((k,v) <- iter) { if(!indexMap.contains(k)) { val ind = indexMap.size indexMap.put(k, ind) values.append(v) - bs(ind) = true } else { val ind = indexMap.get(k) values(ind) = reduceFunc(values(ind), v) } } + val bs = new BitSet(indexMap.size) + bs.setUntil(indexMap.size) Iterator( (indexMap, (values.toIndexedSeq, bs)) ) }, true).cache // extract the index and the values @@ -732,16 +746,17 @@ object VertexSetRDD { val values = new Array[C](index.size) val bs = new BitSet(index.size) for ((k,c) <- tblIter) { + // @todo this extra check may be costing us a lot! if (!index.contains(k)) { throw new SparkException("Error: Trying to bind an external index " + "to an RDD which contains keys that are not in the index.") } val ind = index(k) - if (bs(ind)) { + if (bs.get(ind)) { values(ind) = mergeCombiners(values(ind), c) } else { values(ind) = c - bs(ind) = true + bs.set(ind) } } Iterator((values, bs)) From d74ad4ebc910e62f2598b7a7323fdc678fa179ca Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Thu, 31 Oct 2013 18:01:34 -0700 Subject: [PATCH 169/531] Adding ability to access local BitSet and to safely get a value at a given position --- .../scala/org/apache/spark/util/hash/OpenHashSet.scala | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/util/hash/OpenHashSet.scala b/core/src/main/scala/org/apache/spark/util/hash/OpenHashSet.scala index 7aa3f6220cee3..d083ab26ac868 100644 --- a/core/src/main/scala/org/apache/spark/util/hash/OpenHashSet.scala +++ b/core/src/main/scala/org/apache/spark/util/hash/OpenHashSet.scala @@ -81,6 +81,8 @@ class OpenHashSet[@specialized(Long, Int) T: ClassManifest]( protected var _data = classManifest[T].newArray(_capacity) protected var _bitset = new BitSet(_capacity) + def getBitSet = _bitset + /** Number of elements in the set. */ def size: Int = _size @@ -147,6 +149,13 @@ class OpenHashSet[@specialized(Long, Int) T: ClassManifest]( /** Return the value at the specified position. */ def getValue(pos: Int): T = _data(pos) + /** Return the value at the specified position. */ + def getValueSafe(pos: Int): T = { + assert(_bitset.get(pos)) + _data(pos) + } + + /** * Return the next position with an element stored, starting from the given position inclusively. */ From 4ad58e2b9a6e21b5b23ae2b0c62633e085ef3a61 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Thu, 31 Oct 2013 18:09:42 -0700 Subject: [PATCH 170/531] This commit makes three changes to the (PrimitiveKey)OpenHashMap 1) _keySet --renamed--> keySet 2) keySet and _values are made externally accessible 3) added an update function which merges duplicate values --- .../apache/spark/util/hash/OpenHashMap.scala | 49 ++++++++++++++----- .../util/hash/PrimitiveKeyOpenHashMap.scala | 47 +++++++++++++----- 2 files changed, 71 insertions(+), 25 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/hash/OpenHashMap.scala b/core/src/main/scala/org/apache/spark/util/hash/OpenHashMap.scala index a376d1015a314..af282d5651292 100644 --- a/core/src/main/scala/org/apache/spark/util/hash/OpenHashMap.scala +++ b/core/src/main/scala/org/apache/spark/util/hash/OpenHashMap.scala @@ -27,14 +27,21 @@ package org.apache.spark.util.hash */ private[spark] class OpenHashMap[K >: Null : ClassManifest, @specialized(Long, Int, Double) V: ClassManifest]( - initialCapacity: Int) + var keySet: OpenHashSet[K], var _values: Array[V]) extends Iterable[(K, V)] with Serializable { - def this() = this(64) + /** + * Allocate an OpenHashMap with a fixed initial capacity + */ + def this(initialCapacity: Int = 64) = + this(new OpenHashSet[K](initialCapacity), new Array[V](initialCapacity)) + + /** + * Allocate an OpenHashMap with a fixed initial capacity + */ + def this(keySet: OpenHashSet[K]) = this(keySet, new Array[V](keySet.capacity)) - protected var _keySet = new OpenHashSet[K](initialCapacity) - private var _values = new Array[V](_keySet.capacity) @transient private var _oldValues: Array[V] = null @@ -42,14 +49,14 @@ class OpenHashMap[K >: Null : ClassManifest, @specialized(Long, Int, Double) V: private var haveNullValue = false private var nullValue: V = null.asInstanceOf[V] - override def size: Int = if (haveNullValue) _keySet.size + 1 else _keySet.size + override def size: Int = if (haveNullValue) keySet.size + 1 else keySet.size /** Get the value for a given key */ def apply(k: K): V = { if (k == null) { nullValue } else { - val pos = _keySet.getPos(k) + val pos = keySet.getPos(k) if (pos < 0) { null.asInstanceOf[V] } else { @@ -64,9 +71,26 @@ class OpenHashMap[K >: Null : ClassManifest, @specialized(Long, Int, Double) V: haveNullValue = true nullValue = v } else { - val pos = _keySet.fastAdd(k) & OpenHashSet.POSITION_MASK + val pos = keySet.fastAdd(k) & OpenHashSet.POSITION_MASK _values(pos) = v - _keySet.rehashIfNeeded(k, grow, move) + keySet.rehashIfNeeded(k, grow, move) + _oldValues = null + } + } + + /** Set the value for a key */ + def update(k: K, v: V, mergeF: (V,V) => V) { + if (k == null) { + if(haveNullValue) { + nullValue = mergeF(nullValue, v) + } else { + haveNullValue = true + nullValue = v + } + } else { + val pos = keySet.fastAdd(k) & OpenHashSet.POSITION_MASK + _values(pos) = mergeF(_values(pos), v) + keySet.rehashIfNeeded(k, grow, move) _oldValues = null } } @@ -87,11 +111,11 @@ class OpenHashMap[K >: Null : ClassManifest, @specialized(Long, Int, Double) V: } nullValue } else { - val pos = _keySet.fastAdd(k) + val pos = keySet.fastAdd(k) if ((pos & OpenHashSet.EXISTENCE_MASK) != 0) { val newValue = defaultValue _values(pos & OpenHashSet.POSITION_MASK) = newValue - _keySet.rehashIfNeeded(k, grow, move) + keySet.rehashIfNeeded(k, grow, move) newValue } else { _values(pos) = mergeValue(_values(pos)) @@ -113,9 +137,9 @@ class OpenHashMap[K >: Null : ClassManifest, @specialized(Long, Int, Double) V: } pos += 1 } - pos = _keySet.nextPos(pos) + pos = keySet.nextPos(pos) if (pos >= 0) { - val ret = (_keySet.getValue(pos), _values(pos)) + val ret = (keySet.getValue(pos), _values(pos)) pos += 1 ret } else { @@ -146,3 +170,4 @@ class OpenHashMap[K >: Null : ClassManifest, @specialized(Long, Int, Double) V: _values(newPos) = _oldValues(oldPos) } } + diff --git a/core/src/main/scala/org/apache/spark/util/hash/PrimitiveKeyOpenHashMap.scala b/core/src/main/scala/org/apache/spark/util/hash/PrimitiveKeyOpenHashMap.scala index 14c136720788a..cbfb2361b419e 100644 --- a/core/src/main/scala/org/apache/spark/util/hash/PrimitiveKeyOpenHashMap.scala +++ b/core/src/main/scala/org/apache/spark/util/hash/PrimitiveKeyOpenHashMap.scala @@ -28,35 +28,56 @@ package org.apache.spark.util.hash private[spark] class PrimitiveKeyOpenHashMap[@specialized(Long, Int) K: ClassManifest, @specialized(Long, Int, Double) V: ClassManifest]( - initialCapacity: Int) + var keySet: OpenHashSet[K], var _values: Array[V]) extends Iterable[(K, V)] with Serializable { - def this() = this(64) + /** + * Allocate an OpenHashMap with a fixed initial capacity + */ + def this(initialCapacity: Int = 64) = + this(new OpenHashSet[K](initialCapacity), new Array[V](initialCapacity)) - require(classManifest[K] == classManifest[Long] || classManifest[K] == classManifest[Int]) + /** + * Allocate an OpenHashMap with a fixed initial capacity + */ + def this(keySet: OpenHashSet[K]) = this(keySet, new Array[V](keySet.capacity)) - protected var _keySet = new OpenHashSet[K](initialCapacity) - private var _values = new Array[V](_keySet.capacity) + require(classManifest[K] == classManifest[Long] || classManifest[K] == classManifest[Int]) private var _oldValues: Array[V] = null - override def size = _keySet.size + override def size = keySet.size /** Get the value for a given key */ def apply(k: K): V = { - val pos = _keySet.getPos(k) + val pos = keySet.getPos(k) _values(pos) } /** Set the value for a key */ def update(k: K, v: V) { - val pos = _keySet.fastAdd(k) & OpenHashSet.POSITION_MASK + val pos = keySet.fastAdd(k) & OpenHashSet.POSITION_MASK _values(pos) = v - _keySet.rehashIfNeeded(k, grow, move) + keySet.rehashIfNeeded(k, grow, move) + _oldValues = null + } + + + /** Set the value for a key */ + def update(k: K, v: V, mergeF: (V,V) => V) { + val pos = keySet.fastAdd(k) + val ind = pos & OpenHashSet.POSITION_MASK + if ((pos & OpenHashSet.EXISTENCE_MASK) != 0) { // if first add + _values(ind) = v + } else { + _values(ind) = mergeF(_values(ind), v) + } + keySet.rehashIfNeeded(k, grow, move) _oldValues = null } + /** * If the key doesn't exist yet in the hash map, set its value to defaultValue; otherwise, * set its value to mergeValue(oldValue). @@ -64,11 +85,11 @@ class PrimitiveKeyOpenHashMap[@specialized(Long, Int) K: ClassManifest, * @return the newly updated value. */ def changeValue(k: K, defaultValue: => V, mergeValue: (V) => V): V = { - val pos = _keySet.fastAdd(k) + val pos = keySet.fastAdd(k) if ((pos & OpenHashSet.EXISTENCE_MASK) != 0) { val newValue = defaultValue _values(pos & OpenHashSet.POSITION_MASK) = newValue - _keySet.rehashIfNeeded(k, grow, move) + keySet.rehashIfNeeded(k, grow, move) newValue } else { _values(pos) = mergeValue(_values(pos)) @@ -82,9 +103,9 @@ class PrimitiveKeyOpenHashMap[@specialized(Long, Int) K: ClassManifest, /** Get the next value we should return from next(), or null if we're finished iterating */ def computeNextPair(): (K, V) = { - pos = _keySet.nextPos(pos) + pos = keySet.nextPos(pos) if (pos >= 0) { - val ret = (_keySet.getValue(pos), _values(pos)) + val ret = (keySet.getValue(pos), _values(pos)) pos += 1 ret } else { From 8381aeffb34fecba53b943763ef65f35ef52289a Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Thu, 31 Oct 2013 18:13:02 -0700 Subject: [PATCH 171/531] This commit introduces the OpenHashSet and OpenHashMap as indexing primitives. Large parts of the VertexSetRDD were restructured to take advantage of: 1) the OpenHashSet as an index map 2) view based lazy mapValues and mapValuesWithVertices 3) the cogroup code is currently disabled (since it is not used in any of the tests) The GraphImpl was updated to also use the OpenHashSet and PrimitiveOpenHashMap wherever possible: 1) the LocalVidMaps (used to track replicated vertices) are now implemented using the OpenHashSet 2) an OpenHashMap is temporarily constructed to combine the local OpenHashSet with the local (replicated) vertex attribute arrays 3) because the OpenHashSet constructor grabs a class manifest all operations that construct OpenHashSets have been moved to the GraphImpl Singleton to prevent implicit variable capture within closures. --- .../spark/graph/GraphKryoRegistrator.scala | 6 +- .../org/apache/spark/graph/VertexSetRDD.scala | 214 ++++++++--------- .../apache/spark/graph/impl/GraphImpl.scala | 217 +++++++++--------- .../org/apache/spark/graph/package.scala | 10 +- .../org/apache/spark/graph/GraphSuite.scala | 11 +- 5 files changed, 225 insertions(+), 233 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphKryoRegistrator.scala b/graph/src/main/scala/org/apache/spark/graph/GraphKryoRegistrator.scala index 821063e1f811b..62f445127c568 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphKryoRegistrator.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphKryoRegistrator.scala @@ -1,14 +1,11 @@ package org.apache.spark.graph -import org.apache.spark.util.hash.BitSet - - import com.esotericsoftware.kryo.Kryo import org.apache.spark.graph.impl.MessageToPartition import org.apache.spark.serializer.KryoRegistrator import org.apache.spark.graph.impl._ -import scala.collection.mutable.BitSet +import org.apache.spark.util.hash.BitSet class GraphKryoRegistrator extends KryoRegistrator { @@ -20,7 +17,6 @@ class GraphKryoRegistrator extends KryoRegistrator { kryo.register(classOf[EdgePartition[Object]]) kryo.register(classOf[BitSet]) kryo.register(classOf[VertexIdToIndexMap]) - // This avoids a large number of hash table lookups. kryo.setReferences(false) } diff --git a/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala b/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala index 8acc89a95b8f1..b3647c083ed5d 100644 --- a/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala @@ -31,6 +31,8 @@ import org.apache.spark.Partitioner._ import org.apache.spark.storage.StorageLevel import org.apache.spark.util.hash.BitSet +import org.apache.spark.util.hash.OpenHashSet +import org.apache.spark.util.hash.PrimitiveKeyOpenHashMap @@ -160,15 +162,8 @@ class VertexSetRDD[@specialized V: ClassManifest]( * Provide the RDD[(K,V)] equivalent output. */ override def compute(part: Partition, context: TaskContext): Iterator[(Vid, V)] = { - tuples.compute(part, context).flatMap { case (indexMap, (values, bs) ) => - // Walk the index to construct the key, value pairs - indexMap.iterator - // Extract rows with key value pairs and indicators - .map{ case (k, ind) => (bs.get(ind), k, ind) } - // Remove tuples that aren't actually present in the array - .filter( _._1 ) - // Extract the pair (removing the indicator from the tuple) - .map( x => (x._2, values(x._3) ) ) + tuples.compute(part, context).flatMap { case (indexMap, (values, bs) ) => + bs.iterator.map(ind => (indexMap.getValueSafe(ind), values(ind))) } } // end of compute @@ -195,11 +190,15 @@ class VertexSetRDD[@specialized V: ClassManifest]( assert(valuesIter.hasNext() == false) // Allocate the array to store the results into val newBS = new BitSet(oldValues.size) - // Populate the new Values - for( (k,i) <- index ) { - if( bs.get(i) && cleanPred( (k, oldValues(i)) ) ) { - newBS.set(i) + // Iterate over the active bits in the old bitset and + // evaluate the predicate + var ind = bs.nextSetBit(0) + while(ind >= 0) { + val k = index.getValueSafe(ind) + if( cleanPred( (k, oldValues(ind)) ) ) { + newBS.set(ind) } + ind = bs.nextSetBit(ind+1) } Array((oldValues, newBS)).iterator } @@ -223,27 +222,10 @@ class VertexSetRDD[@specialized V: ClassManifest]( val newValuesRDD: RDD[ (IndexedSeq[U], BitSet) ] = valuesRDD.mapPartitions(iter => iter.map{ case (values, bs: BitSet) => - - /** - * @todo Consider using a view rather than creating a new - * array. This is already being done for join operations. - * It could reduce memory overhead but require additional - * recomputation. - */ - val newValues = new Array[U](values.size) - var ind = bs.nextSetBit(0) - while(ind >= 0) { - // if(ind >= newValues.size) { - // println(ind) - // println(newValues.size) - // bs.iterator.foreach(print(_)) - // } - // assert(ind < newValues.size) - // assert(ind < values.size) - newValues(ind) = cleanF(values(ind)) - ind = bs.nextSetBit(ind+1) - } - (newValues.toIndexedSeq, bs) + val newValues: IndexedSeq[U] = values.view.zipWithIndex.map{ + (x: (V, Int)) => if(bs.get(x._2)) cleanF(x._1) else null.asInstanceOf[U] + }.toIndexedSeq // @todo check the toIndexedSeq is free + (newValues, bs) }, preservesPartitioning = true) new VertexSetRDD[U](index, newValuesRDD) } // end of mapValues @@ -271,18 +253,14 @@ class VertexSetRDD[@specialized V: ClassManifest]( assert(keysIter.hasNext() == false) val (oldValues, bs: BitSet) = valuesIter.next() assert(valuesIter.hasNext() == false) - /** - * @todo Consider using a view rather than creating a new array. - * This is already being done for join operations. It could reduce - * memory overhead but require additional recomputation. - */ - // Allocate the array to store the results into - val newValues: Array[U] = new Array[U](oldValues.size) - // Populate the new Values - for( (k,i) <- index ) { - if (bs.get(i)) { newValues(i) = cleanF(k, oldValues(i)) } - } - Array((newValues.toIndexedSeq, bs)).iterator + // Cosntruct a view of the map transformation + val newValues: IndexedSeq[U] = oldValues.view.zipWithIndex.map{ + (x: (V, Int)) => + if(bs.get(x._2)) { + cleanF(index.getValueSafe(x._2), x._1) + } else null.asInstanceOf[U] + }.toIndexedSeq // @todo check the toIndexedSeq is free + Iterator((newValues, bs)) } new VertexSetRDD[U](index, newValues) } // end of mapValuesWithKeys @@ -314,8 +292,10 @@ class VertexSetRDD[@specialized V: ClassManifest]( val (otherValues, otherBS: BitSet) = otherIter.next() assert(!otherIter.hasNext) val newBS: BitSet = thisBS & otherBS - val newValues = thisValues.view.zip(otherValues) - Iterator((newValues.toIndexedSeq, newBS)) + val newValues: IndexedSeq[(V,W)] = + thisValues.view.zip(otherValues).toIndexedSeq // @todo check the toIndexedSeq is free + // Iterator((newValues.toIndexedSeq, newBS)) + Iterator((newValues, newBS)) } new VertexSetRDD(index, newValuesRDD) } @@ -348,10 +328,15 @@ class VertexSetRDD[@specialized V: ClassManifest]( assert(!thisIter.hasNext) val (otherValues, otherBS: BitSet) = otherIter.next() assert(!otherIter.hasNext) - val otherOption = otherValues.view.zipWithIndex - .map{ (x: (W, Int)) => if(otherBS.get(x._2)) Option(x._1) else None } - val newValues = thisValues.view.zip(otherOption) - Iterator((newValues.toIndexedSeq, thisBS)) + val newValues: IndexedSeq[(V, Option[W])] = thisValues.view.zip(otherValues) + .zipWithIndex.map { + // @todo not sure about the efficiency of this case statement + // though it is assumed that the return value is short lived + case ((v, w), ind) => (v, if (otherBS.get(ind)) Option(w) else None) + } + .toIndexedSeq // @todo check the toIndexedSeq is free + Iterator((newValues, thisBS)) + // Iterator((newValues.toIndexedSeq, thisBS)) } new VertexSetRDD(index, newValuesRDD) } // end of leftZipJoin @@ -378,7 +363,6 @@ class VertexSetRDD[@specialized V: ClassManifest]( def leftJoin[W: ClassManifest]( other: RDD[(Vid,W)], merge: (W,W) => W = (a:W, b:W) => a): VertexSetRDD[(V, Option[W]) ] = { - val cleanMerge = index.rdd.context.clean(merge) // Test if the other vertex is a VertexSetRDD to choose the optimal // join strategy other match { @@ -396,7 +380,7 @@ class VertexSetRDD[@specialized V: ClassManifest]( if (other.partitioner == partitioner) other else other.partitionBy(partitioner.get) // Compute the new values RDD - val newValues: RDD[ (IndexedSeq[(V,Option[W])], BitSet) ] = + val newValuesRDD: RDD[ (IndexedSeq[(V,Option[W])], BitSet) ] = index.rdd.zipPartitions(valuesRDD, otherShuffled) { (thisIndexIter: Iterator[VertexIdToIndexMap], thisIter: Iterator[(IndexedSeq[V], BitSet)], @@ -407,33 +391,37 @@ class VertexSetRDD[@specialized V: ClassManifest]( val (thisValues, thisBS) = thisIter.next() assert(!thisIter.hasNext) // Create a new array to store the values in the resulting VertexSet - val newW = new Array[W](thisValues.size) + val otherValues = new Array[W](thisValues.size) // track which values are matched with values in other - val wBS = new BitSet(thisValues.size) - // Loop over all the tuples that have vertices in this VertexSet. - for( (k, w) <- tuplesIter if index.contains(k) ) { - val ind = index.get(k) - // Not all the vertex ids in the index are in this VertexSet. - // If there is a vertex in this set then record the other value - if(thisBS.get(ind)) { - if(wBS.get(ind)) { - newW(ind) = cleanMerge(newW(ind), w) - } else { - newW(ind) = w - wBS.set(ind) + val otherBS = new BitSet(thisValues.size) + for ((k,w) <- tuplesIter) { + // Get the location of the key in the index + val pos = index.getPos(k) + // Only if the key is already in the index + if ((pos & OpenHashSet.EXISTENCE_MASK) == 0) { + // Get the actual index + val ind = pos & OpenHashSet.POSITION_MASK + // If this value has already been seen then merge + if (otherBS.get(ind)) { + otherValues(ind) = merge(otherValues(ind), w) + } else { // otherwise just store the new value + otherBS.set(ind) + otherValues(ind) = w } } - } // end of for loop over tuples + } // Some vertices in this vertex set may not have a corresponding // tuple in the join and so a None value should be returned. - val otherOption = newW.view.zipWithIndex - .map{ (x: (W, Int)) => if(wBS.get(x._2)) Option(x._1) else None } - // the final values is the zip of the values in this RDD along with - // the values in the other - val newValues = thisValues.view.zip(otherOption) - Iterator((newValues.toIndexedSeq, thisBS)) + val newValues: IndexedSeq[(V, Option[W])] = thisValues.view.zip(otherValues) + .zipWithIndex.map { + // @todo not sure about the efficiency of this case statement + // though it is assumed that the return value is short lived + case ((v, w), ind) => (v, if (otherBS.get(ind)) Option(w) else None) + } + .toIndexedSeq // @todo check the toIndexedSeq is free + Iterator((newValues, thisBS)) } // end of newValues - new VertexSetRDD(index, newValues) + new VertexSetRDD(index, newValuesRDD) } } } // end of leftJoin @@ -443,6 +431,7 @@ class VertexSetRDD[@specialized V: ClassManifest]( * For each key k in `this` or `other`, return a resulting RDD that contains a * tuple with the list of values for that key in `this` as well as `other`. */ + /* def cogroup[W: ClassManifest](other: RDD[(Vid, W)], partitioner: Partitioner): VertexSetRDD[(Seq[V], Seq[W])] = { //RDD[(K, (Seq[V], Seq[W]))] = { @@ -489,16 +478,17 @@ class VertexSetRDD[@specialized V: ClassManifest]( assert(!thisIter.hasNext) val otherIndex = otherIter.next() assert(!otherIter.hasNext) - val newIndex = new VertexIdToIndexMap() - // @todo Merge only the keys that correspond to non-null values // Merge the keys - newIndex.putAll(thisIndex) - newIndex.putAll(otherIndex) - // We need to rekey the index - var ctr = 0 - for (e <- newIndex.entrySet) { - e.setValue(ctr) - ctr += 1 + val newIndex = new VertexIdToIndexMap(thisIndex.capacity + otherIndex.capacity) + var ind = thisIndex.nextPos(0) + while(ind >= 0) { + newIndex.fastAdd(thisIndex.getValue(ind)) + ind = thisIndex.nextPos(ind+1) + } + var ind = otherIndex.nextPos(0) + while(ind >= 0) { + newIndex.fastAdd(otherIndex.getValue(ind)) + ind = otherIndex.nextPos(ind+1) } List(newIndex).iterator }).cache() @@ -604,7 +594,7 @@ class VertexSetRDD[@specialized V: ClassManifest]( } } } // end of cogroup - + */ } // End of VertexSetRDD @@ -649,21 +639,14 @@ object VertexSetRDD { } val groups = preAgg.mapPartitions( iter => { - val indexMap = new VertexIdToIndexMap() - val values = new ArrayBuffer[V] + val hashMap = new PrimitiveKeyOpenHashMap[Vid, V] for ((k,v) <- iter) { - if(!indexMap.contains(k)) { - val ind = indexMap.size - indexMap.put(k, ind) - values.append(v) - } else { - val ind = indexMap.get(k) - values(ind) = reduceFunc(values(ind), v) - } + hashMap.update(k, v, reduceFunc) } - val bs = new BitSet(indexMap.size) - bs.setUntil(indexMap.size) - Iterator( (indexMap, (values.toIndexedSeq, bs)) ) + val index = hashMap.keySet + val values: IndexedSeq[V] = hashMap._values + val bs = index.getBitSet + Iterator( (index, (values, bs)) ) }, true).cache // extract the index and the values val index = groups.mapPartitions(_.map{ case (kMap, vAr) => kMap }, true) @@ -747,20 +730,24 @@ object VertexSetRDD { // There is only one map val index = indexIter.next() assert(!indexIter.hasNext()) - val values = new Array[C](index.size) - val bs = new BitSet(index.size) + val values = new Array[C](index.capacity) + val bs = new BitSet(index.capacity) for ((k,c) <- tblIter) { - // @todo this extra check may be costing us a lot! - if (!index.contains(k)) { + // Get the location of the key in the index + val pos = index.getPos(k) + if ((pos & OpenHashSet.EXISTENCE_MASK) != 0) { throw new SparkException("Error: Trying to bind an external index " + "to an RDD which contains keys that are not in the index.") - } - val ind = index(k) - if (bs.get(ind)) { - values(ind) = mergeCombiners(values(ind), c) } else { - values(ind) = c - bs.set(ind) + // Get the actual index + val ind = pos & OpenHashSet.POSITION_MASK + // If this value has already been seen then merge + if (bs.get(ind)) { + values(ind) = mergeCombiners(values(ind), c) + } else { // otherwise just store the new value + bs.set(ind) + values(ind) = c + } } } Iterator((values, bs)) @@ -792,14 +779,9 @@ object VertexSetRDD { } val index = shuffledTbl.mapPartitions( iter => { - val indexMap = new VertexIdToIndexMap() - for ( (k,_) <- iter ){ - if(!indexMap.contains(k)){ - val ind = indexMap.size - indexMap.put(k, ind) - } - } - Iterator(indexMap) + val index = new VertexIdToIndexMap + for ( (k,_) <- iter ){ index.add(k) } + Iterator(index) }, true).cache new VertexSetIndex(index) } diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index 016811db3660c..b80713dbf4382 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -5,7 +5,6 @@ import scala.collection.JavaConversions._ import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.ArrayBuilder -import scala.collection.mutable.BitSet import org.apache.spark.SparkContext._ @@ -21,6 +20,12 @@ import org.apache.spark.graph._ import org.apache.spark.graph.impl.GraphImpl._ import org.apache.spark.graph.impl.MessageToPartitionRDDFunctions._ +import org.apache.spark.util.hash.BitSet +import org.apache.spark.util.hash.OpenHashSet +import org.apache.spark.util.hash.PrimitiveKeyOpenHashMap + + + /** * The Iterator type returned when constructing edge triplets */ @@ -31,15 +36,16 @@ class EdgeTripletIterator[VD: ClassManifest, ED: ClassManifest]( private var pos = 0 private val et = new EdgeTriplet[VD, ED] + private val vmap = new PrimitiveKeyOpenHashMap[Vid, VD](vidToIndex, vertexArray) override def hasNext: Boolean = pos < edgePartition.size override def next() = { et.srcId = edgePartition.srcIds(pos) // assert(vmap.containsKey(e.src.id)) - et.srcAttr = vertexArray(vidToIndex(et.srcId)) + et.srcAttr = vmap(et.srcId) et.dstId = edgePartition.dstIds(pos) // assert(vmap.containsKey(e.dst.id)) - et.dstAttr = vertexArray(vidToIndex(et.dstId)) + et.dstAttr = vmap(et.dstId) et.attr = edgePartition.data(pos) pos += 1 et @@ -51,10 +57,10 @@ class EdgeTripletIterator[VD: ClassManifest, ED: ClassManifest]( for (i <- (0 until edgePartition.size)) { currentEdge.srcId = edgePartition.srcIds(i) // assert(vmap.containsKey(e.src.id)) - currentEdge.srcAttr = vertexArray(vidToIndex(currentEdge.srcId)) + currentEdge.srcAttr = vmap(currentEdge.srcId) currentEdge.dstId = edgePartition.dstIds(i) // assert(vmap.containsKey(e.dst.id)) - currentEdge.dstAttr = vertexArray(vidToIndex(currentEdge.dstId)) + currentEdge.dstAttr = vmap(currentEdge.dstId) currentEdge.attr = edgePartition.data(i) lb += currentEdge } @@ -63,23 +69,6 @@ class EdgeTripletIterator[VD: ClassManifest, ED: ClassManifest]( } // end of Edge Triplet Iterator - -object EdgeTripletBuilder { - def makeTriplets[VD: ClassManifest, ED: ClassManifest]( - localVidMap: RDD[(Pid, VertexIdToIndexMap)], - vTableReplicatedValues: RDD[(Pid, Array[VD]) ], - eTable: RDD[(Pid, EdgePartition[ED])]): RDD[EdgeTriplet[VD, ED]] = { - localVidMap.zipPartitions(vTableReplicatedValues, eTable) { - (vidMapIter, replicatedValuesIter, eTableIter) => - val (_, vidToIndex) = vidMapIter.next() - val (_, vertexArray) = replicatedValuesIter.next() - val (_, edgePartition) = eTableIter.next() - new EdgeTripletIterator(vidToIndex, vertexArray, edgePartition) - } - } -} - - /** * A Graph RDD that supports computation on graphs. */ @@ -90,6 +79,10 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( @transient val eTable: RDD[(Pid, EdgePartition[ED])] ) extends Graph[VD, ED] { + def this() = this(null, null, null, null) + + + /** * (localVidMap: VertexSetRDD[Pid, VertexIdToIndexMap]) is a version of the * vertex data after it is replicated. Within each partition, it holds a map @@ -115,7 +108,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( /** Return a RDD that brings edges with its source and destination vertices together. */ @transient override val triplets: RDD[EdgeTriplet[VD, ED]] = - EdgeTripletBuilder.makeTriplets(localVidMap, vTableReplicatedValues, eTable) + makeTriplets(localVidMap, vTableReplicatedValues, eTable) override def cache(): Graph[VD, ED] = { @@ -219,24 +212,8 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( } - override def mapTriplets[ED2: ClassManifest](f: EdgeTriplet[VD, ED] => ED2): - Graph[VD, ED2] = { - val newETable = eTable.zipPartitions(localVidMap, vTableReplicatedValues){ - (edgePartitionIter, vidToIndexIter, vertexArrayIter) => - val (pid, edgePartition) = edgePartitionIter.next() - val (_, vidToIndex) = vidToIndexIter.next() - val (_, vertexArray) = vertexArrayIter.next() - val et = new EdgeTriplet[VD, ED] - val newEdgePartition = edgePartition.map{e => - et.set(e) - et.srcAttr = vertexArray(vidToIndex(e.srcId)) - et.dstAttr = vertexArray(vidToIndex(e.dstId)) - f(et) - } - Iterator((pid, newEdgePartition)) - } - new GraphImpl(vTable, vid2pid, localVidMap, newETable) - } + override def mapTriplets[ED2: ClassManifest](f: EdgeTriplet[VD, ED] => ED2): Graph[VD, ED2] = + GraphImpl.mapTriplets(this, f) override def subgraph(epred: EdgeTriplet[VD,ED] => Boolean = (x => true), @@ -330,57 +307,8 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( override def mapReduceTriplets[A: ClassManifest]( mapFunc: EdgeTriplet[VD, ED] => Array[(Vid, A)], reduceFunc: (A, A) => A) - : VertexSetRDD[A] = { - - ClosureCleaner.clean(mapFunc) - ClosureCleaner.clean(reduceFunc) - - // Map and preaggregate - val preAgg = eTable.zipPartitions(localVidMap, vTableReplicatedValues){ - (edgePartitionIter, vidToIndexIter, vertexArrayIter) => - val (pid, edgePartition) = edgePartitionIter.next() - val (_, vidToIndex) = vidToIndexIter.next() - val (_, vertexArray) = vertexArrayIter.next() - // We can reuse the vidToIndex map for aggregation here as well. - /** @todo Since this has the downside of not allowing "messages" to arbitrary - * vertices we should consider just using a fresh map. - */ - val msgArray = new Array[A](vertexArray.size) - val msgBS = new BitSet(vertexArray.size) - // Iterate over the partition - val et = new EdgeTriplet[VD, ED] - edgePartition.foreach{e => - et.set(e) - et.srcAttr = vertexArray(vidToIndex(e.srcId)) - et.dstAttr = vertexArray(vidToIndex(e.dstId)) - mapFunc(et).foreach{ case (vid, msg) => - // verify that the vid is valid - assert(vid == et.srcId || vid == et.dstId) - val ind = vidToIndex(vid) - // Populate the aggregator map - if(msgBS(ind)) { - msgArray(ind) = reduceFunc(msgArray(ind), msg) - } else { - msgArray(ind) = msg - msgBS(ind) = true - } - } - } - // Return the aggregate map - vidToIndex.long2IntEntrySet().fastIterator() - // Remove the entries that did not receive a message - .filter{ entry => msgBS(entry.getValue()) } - // Construct the actual pairs - .map{ entry => - val vid = entry.getLongKey() - val ind = entry.getValue() - val msg = msgArray(ind) - (vid, msg) - } - }.partitionBy(vTable.index.rdd.partitioner.get) - // do the final reduction reusing the index map - VertexSetRDD(preAgg, vTable.index, reduceFunc) - } + : VertexSetRDD[A] = + GraphImpl.mapReduceTriplets(this, mapFunc, reduceFunc) override def outerJoinVertices[U: ClassManifest, VD2: ClassManifest] @@ -436,7 +364,6 @@ object GraphImpl { } - /** * Create the edge table RDD, which is much more efficient for Java heap storage than the * normal edges data structure (RDD[(Vid, Vid, ED)]). @@ -494,16 +421,9 @@ object GraphImpl { RDD[(Pid, VertexIdToIndexMap)] = { eTable.mapPartitions( _.map{ case (pid, epart) => val vidToIndex = new VertexIdToIndexMap - var i = 0 epart.foreach{ e => - if(!vidToIndex.contains(e.srcId)) { - vidToIndex.put(e.srcId, i) - i += 1 - } - if(!vidToIndex.contains(e.dstId)) { - vidToIndex.put(e.dstId, i) - i += 1 - } + vidToIndex.add(e.srcId) + vidToIndex.add(e.dstId) } (pid, vidToIndex) }, preservesPartitioning = true).cache() @@ -528,9 +448,9 @@ object GraphImpl { val (pid, vidToIndex) = mapIter.next() assert(!mapIter.hasNext) // Populate the vertex array using the vidToIndex map - val vertexArray = new Array[VD](vidToIndex.size) + val vertexArray = new Array[VD](vidToIndex.capacity) for (msg <- msgsIter) { - val ind = vidToIndex(msg.data._1) + val ind = vidToIndex.getPos(msg.data._1) & OpenHashSet.POSITION_MASK vertexArray(ind) = msg.data._2 } Iterator((pid, vertexArray)) @@ -540,6 +460,95 @@ object GraphImpl { } + def makeTriplets[VD: ClassManifest, ED: ClassManifest]( + localVidMap: RDD[(Pid, VertexIdToIndexMap)], + vTableReplicatedValues: RDD[(Pid, Array[VD]) ], + eTable: RDD[(Pid, EdgePartition[ED])]): RDD[EdgeTriplet[VD, ED]] = { + localVidMap.zipPartitions(vTableReplicatedValues, eTable) { + (vidMapIter, replicatedValuesIter, eTableIter) => + val (_, vidToIndex) = vidMapIter.next() + val (_, vertexArray) = replicatedValuesIter.next() + val (_, edgePartition) = eTableIter.next() + new EdgeTripletIterator(vidToIndex, vertexArray, edgePartition) + } + } + + + def mapTriplets[VD: ClassManifest, ED: ClassManifest, ED2: ClassManifest]( + g: GraphImpl[VD, ED], + f: EdgeTriplet[VD, ED] => ED2): Graph[VD, ED2] = { + val newETable = g.eTable.zipPartitions(g.localVidMap, g.vTableReplicatedValues){ + (edgePartitionIter, vidToIndexIter, vertexArrayIter) => + val (pid, edgePartition) = edgePartitionIter.next() + val (_, vidToIndex) = vidToIndexIter.next() + val (_, vertexArray) = vertexArrayIter.next() + val et = new EdgeTriplet[VD, ED] + val vmap = new PrimitiveKeyOpenHashMap[Vid, VD](vidToIndex, vertexArray) + val newEdgePartition = edgePartition.map{e => + et.set(e) + et.srcAttr = vmap(e.srcId) + et.dstAttr = vmap(e.dstId) + f(et) + } + Iterator((pid, newEdgePartition)) + } + new GraphImpl(g.vTable, g.vid2pid, g.localVidMap, newETable) + } + + + def mapReduceTriplets[VD: ClassManifest, ED: ClassManifest, A: ClassManifest]( + g: GraphImpl[VD, ED], + mapFunc: EdgeTriplet[VD, ED] => Array[(Vid, A)], + reduceFunc: (A, A) => A): VertexSetRDD[A] = { + + ClosureCleaner.clean(mapFunc) + ClosureCleaner.clean(reduceFunc) + + // Map and preaggregate + val preAgg = g.eTable.zipPartitions(g.localVidMap, g.vTableReplicatedValues){ + (edgePartitionIter, vidToIndexIter, vertexArrayIter) => + val (pid, edgePartition) = edgePartitionIter.next() + val (_, vidToIndex) = vidToIndexIter.next() + val (_, vertexArray) = vertexArrayIter.next() + assert(!edgePartitionIter.hasNext) + assert(!vidToIndexIter.hasNext) + assert(!vertexArrayIter.hasNext) + assert(vidToIndex.capacity == vertexArray.size) + val vmap = new PrimitiveKeyOpenHashMap[Vid, VD](vidToIndex, vertexArray) + // We can reuse the vidToIndex map for aggregation here as well. + /** @todo Since this has the downside of not allowing "messages" to arbitrary + * vertices we should consider just using a fresh map. + */ + val msgArray = new Array[A](vertexArray.size) + val msgBS = new BitSet(vertexArray.size) + // Iterate over the partition + val et = new EdgeTriplet[VD, ED] + edgePartition.foreach{e => + et.set(e) + et.srcAttr = vmap(e.srcId) + et.dstAttr = vmap(e.dstId) + mapFunc(et).foreach{ case (vid, msg) => + // verify that the vid is valid + assert(vid == et.srcId || vid == et.dstId) + // Get the index of the key + val ind = vidToIndex.getPos(vid) & OpenHashSet.POSITION_MASK + // Populate the aggregator map + if(msgBS.get(ind)) { + msgArray(ind) = reduceFunc(msgArray(ind), msg) + } else { + msgArray(ind) = msg + msgBS.set(ind) + } + } + } + // construct an iterator of tuples Iterator[(Vid, A)] + msgBS.iterator.map( ind => (vidToIndex.getValue(ind), msgArray(ind)) ) + }.partitionBy(g.vTable.index.rdd.partitioner.get) + // do the final reduction reusing the index map + VertexSetRDD(preAgg, g.vTable.index, reduceFunc) + } + + protected def edgePartitionFunction1D(src: Vid, dst: Vid, numParts: Pid): Pid = { val mixingPrime: Vid = 1125899906842597L (math.abs(src) * mixingPrime).toInt % numParts diff --git a/graph/src/main/scala/org/apache/spark/graph/package.scala b/graph/src/main/scala/org/apache/spark/graph/package.scala index 4627c3566ca19..37a4fb4a5e149 100644 --- a/graph/src/main/scala/org/apache/spark/graph/package.scala +++ b/graph/src/main/scala/org/apache/spark/graph/package.scala @@ -1,5 +1,10 @@ package org.apache.spark +import org.apache.spark.util.hash.BitSet +import org.apache.spark.util.hash.OpenHashSet +import org.apache.spark.util.hash.PrimitiveKeyOpenHashMap + + package object graph { type Vid = Long @@ -8,8 +13,9 @@ package object graph { type VertexHashMap[T] = it.unimi.dsi.fastutil.longs.Long2ObjectOpenHashMap[T] type VertexSet = it.unimi.dsi.fastutil.longs.LongOpenHashSet type VertexArrayList = it.unimi.dsi.fastutil.longs.LongArrayList - // @todo replace with rxin's fast hashmap - type VertexIdToIndexMap = it.unimi.dsi.fastutil.longs.Long2IntOpenHashMap + + // type VertexIdToIndexMap = it.unimi.dsi.fastutil.longs.Long2IntOpenHashMap + type VertexIdToIndexMap = OpenHashSet[Vid] /** * Return the default null-like value for a data type T. diff --git a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala index f2b3d5bdfe268..2067b1613ef8d 100644 --- a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala @@ -77,16 +77,15 @@ class GraphSuite extends FunSuite with LocalSparkContext { withSpark(new SparkContext("local", "test")) { sc => val a = sc.parallelize((0 to 100).map(x => (x.toLong, x.toLong)), 5) val b = VertexSetRDD(a).mapValues(x => -x) - assert(b.leftJoin(a) - .mapValues(x => x._1 + x._2.get).map(x=> x._2).reduce(_+_) === 0) + assert(b.count === 101) + assert(b.leftJoin(a).mapValues(x => x._1 + x._2.get).map(x=> x._2).reduce(_+_) === 0) val c = VertexSetRDD(a, b.index) - assert(b.leftJoin(c) - .mapValues(x => x._1 + x._2.get).map(x=> x._2).reduce(_+_) === 0) + assert(b.leftJoin(c).mapValues(x => x._1 + x._2.get).map(x=> x._2).reduce(_+_) === 0) val d = c.filter(q => ((q._2 % 2) == 0)) val e = a.filter(q => ((q._2 % 2) == 0)) assert(d.count === e.count) - assert(b.zipJoin(c).mapValues(x => x._1 + x._2) - .map(x => x._2).reduce(_+_) === 0) + assert(b.zipJoin(c).mapValues(x => x._1 + x._2).map(x => x._2).reduce(_+_) === 0) + } } From 0a61cafba8262b3d45330f5bb18e999d67553fd6 Mon Sep 17 00:00:00 2001 From: Dan Crankshaw Date: Thu, 31 Oct 2013 19:54:06 -0700 Subject: [PATCH 172/531] Added logging to Graph, GraphLab, and Pregel. --- graph/src/main/scala/org/apache/spark/graph/Graph.scala | 3 ++- graph/src/main/scala/org/apache/spark/graph/GraphLab.scala | 3 ++- graph/src/main/scala/org/apache/spark/graph/Pregel.scala | 3 ++- 3 files changed, 6 insertions(+), 3 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/Graph.scala b/graph/src/main/scala/org/apache/spark/graph/Graph.scala index 89e1b4ea01a81..f0e96896de370 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Graph.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Graph.scala @@ -3,6 +3,7 @@ package org.apache.spark.graph import org.apache.spark.rdd.RDD import org.apache.spark.util.ClosureCleaner +import org.apache.spark.Logging @@ -24,7 +25,7 @@ import org.apache.spark.util.ClosureCleaner * @tparam VD the vertex attribute type * @tparam ED the edge attribute type */ -abstract class Graph[VD: ClassManifest, ED: ClassManifest] { +abstract class Graph[VD: ClassManifest, ED: ClassManifest] extends Logging { /** * Get the vertices and their data. diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala b/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala index b8503ab7fdb6c..39dc33acf0f6e 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala @@ -2,11 +2,12 @@ package org.apache.spark.graph import scala.collection.JavaConversions._ import org.apache.spark.rdd.RDD +import org.apache.spark.Logging /** * This object implements the GraphLab gather-apply-scatter api. */ -object GraphLab { +object GraphLab extends Logging { /** * Execute the GraphLab Gather-Apply-Scatter API diff --git a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala index 1750b7f8dcabc..c6b532462462f 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala @@ -1,6 +1,7 @@ package org.apache.spark.graph import org.apache.spark.rdd.RDD +import org.apache.spark.Logging /** @@ -41,7 +42,7 @@ import org.apache.spark.rdd.RDD * }}} * */ -object Pregel { +object Pregel extends Logging { /** From 63311d9c729d40c1846aeeef808e3b6b5c4479ec Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Thu, 31 Oct 2013 20:12:30 -0700 Subject: [PATCH 173/531] renamed update to setMerge --- .../src/main/scala/org/apache/spark/util/hash/OpenHashMap.scala | 2 +- .../org/apache/spark/util/hash/PrimitiveKeyOpenHashMap.scala | 2 +- graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/hash/OpenHashMap.scala b/core/src/main/scala/org/apache/spark/util/hash/OpenHashMap.scala index af282d5651292..4eb52158b02d4 100644 --- a/core/src/main/scala/org/apache/spark/util/hash/OpenHashMap.scala +++ b/core/src/main/scala/org/apache/spark/util/hash/OpenHashMap.scala @@ -79,7 +79,7 @@ class OpenHashMap[K >: Null : ClassManifest, @specialized(Long, Int, Double) V: } /** Set the value for a key */ - def update(k: K, v: V, mergeF: (V,V) => V) { + def setMerge(k: K, v: V, mergeF: (V,V) => V) { if (k == null) { if(haveNullValue) { nullValue = mergeF(nullValue, v) diff --git a/core/src/main/scala/org/apache/spark/util/hash/PrimitiveKeyOpenHashMap.scala b/core/src/main/scala/org/apache/spark/util/hash/PrimitiveKeyOpenHashMap.scala index cbfb2361b419e..1bf2554fb7730 100644 --- a/core/src/main/scala/org/apache/spark/util/hash/PrimitiveKeyOpenHashMap.scala +++ b/core/src/main/scala/org/apache/spark/util/hash/PrimitiveKeyOpenHashMap.scala @@ -65,7 +65,7 @@ class PrimitiveKeyOpenHashMap[@specialized(Long, Int) K: ClassManifest, /** Set the value for a key */ - def update(k: K, v: V, mergeF: (V,V) => V) { + def setMerge(k: K, v: V, mergeF: (V,V) => V) { val pos = keySet.fastAdd(k) val ind = pos & OpenHashSet.POSITION_MASK if ((pos & OpenHashSet.EXISTENCE_MASK) != 0) { // if first add diff --git a/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala b/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala index b3647c083ed5d..7211ff3705c45 100644 --- a/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala @@ -641,7 +641,7 @@ object VertexSetRDD { val groups = preAgg.mapPartitions( iter => { val hashMap = new PrimitiveKeyOpenHashMap[Vid, V] for ((k,v) <- iter) { - hashMap.update(k, v, reduceFunc) + hashMap.setMerge(k, v, reduceFunc) } val index = hashMap.keySet val values: IndexedSeq[V] = hashMap._values From e7d37472b83b3bc8e232e790b2df230e35c0a5af Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Thu, 31 Oct 2013 21:09:39 -0700 Subject: [PATCH 174/531] After some testing I realized that the IndexedSeq is still instantiating the array (not maintaining a view) so I have replaced all IndexedSeq[V] with (Int => V) --- .../org/apache/spark/graph/VertexSetRDD.scala | 83 +++++++------------ 1 file changed, 32 insertions(+), 51 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala b/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala index 7211ff3705c45..f26e286003033 100644 --- a/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala @@ -62,7 +62,6 @@ class VertexSetIndex(private[spark] val rdd: RDD[VertexIdToIndexMap]) { } // end of VertexSetIndex - /** * An VertexSetRDD[V] extends the RDD[(Vid,V)] by ensuring that there * is only one entry for each vertex and by pre-indexing the entries @@ -99,7 +98,7 @@ class VertexSetIndex(private[spark] val rdd: RDD[VertexIdToIndexMap]) { */ class VertexSetRDD[@specialized V: ClassManifest]( @transient val index: VertexSetIndex, - @transient val valuesRDD: RDD[ (IndexedSeq[V], BitSet) ]) + @transient val valuesRDD: RDD[ ( (Int => V), BitSet) ]) extends RDD[(Vid, V)](index.rdd.context, List(new OneToOneDependency(index.rdd), new OneToOneDependency(valuesRDD)) ) { @@ -183,13 +182,13 @@ class VertexSetRDD[@specialized V: ClassManifest]( val cleanPred = index.rdd.context.clean(pred) val newValues = index.rdd.zipPartitions(valuesRDD){ (keysIter: Iterator[VertexIdToIndexMap], - valuesIter: Iterator[(IndexedSeq[V], BitSet)]) => + valuesIter: Iterator[(Int => V, BitSet)]) => val index = keysIter.next() assert(keysIter.hasNext() == false) val (oldValues, bs) = valuesIter.next() assert(valuesIter.hasNext() == false) // Allocate the array to store the results into - val newBS = new BitSet(oldValues.size) + val newBS = new BitSet(index.capacity) // Iterate over the active bits in the old bitset and // evaluate the predicate var ind = bs.nextSetBit(0) @@ -218,15 +217,13 @@ class VertexSetRDD[@specialized V: ClassManifest]( * VertexSetRDD retains the same index. */ def mapValues[U: ClassManifest](f: V => U): VertexSetRDD[U] = { - val cleanF = index.rdd.context.clean(f) - val newValuesRDD: RDD[ (IndexedSeq[U], BitSet) ] = + val newValuesRDD: RDD[ (Int => U, BitSet) ] = valuesRDD.mapPartitions(iter => iter.map{ case (values, bs: BitSet) => - val newValues: IndexedSeq[U] = values.view.zipWithIndex.map{ - (x: (V, Int)) => if(bs.get(x._2)) cleanF(x._1) else null.asInstanceOf[U] - }.toIndexedSeq // @todo check the toIndexedSeq is free + val newValues: (Int => U) = + (ind: Int) => if (bs.get(ind)) f(values(ind)) else null.asInstanceOf[U] (newValues, bs) - }, preservesPartitioning = true) + }, preservesPartitioning = true) new VertexSetRDD[U](index, newValuesRDD) } // end of mapValues @@ -244,22 +241,19 @@ class VertexSetRDD[@specialized V: ClassManifest]( * VertexSetRDD retains the same index. */ def mapValuesWithKeys[U: ClassManifest](f: (Vid, V) => U): VertexSetRDD[U] = { - val cleanF = index.rdd.context.clean(f) - val newValues: RDD[ (IndexedSeq[U], BitSet) ] = + val newValues: RDD[ (Int => U, BitSet) ] = index.rdd.zipPartitions(valuesRDD){ (keysIter: Iterator[VertexIdToIndexMap], - valuesIter: Iterator[(IndexedSeq[V], BitSet)]) => + valuesIter: Iterator[(Int => V, BitSet)]) => val index = keysIter.next() assert(keysIter.hasNext() == false) val (oldValues, bs: BitSet) = valuesIter.next() assert(valuesIter.hasNext() == false) // Cosntruct a view of the map transformation - val newValues: IndexedSeq[U] = oldValues.view.zipWithIndex.map{ - (x: (V, Int)) => - if(bs.get(x._2)) { - cleanF(index.getValueSafe(x._2), x._1) - } else null.asInstanceOf[U] - }.toIndexedSeq // @todo check the toIndexedSeq is free + val newValues: (Int => U) = (ind: Int) => { + if (bs.get(ind)) { f(index.getValueSafe(ind), oldValues(ind)) } + else { null.asInstanceOf[U] } + } Iterator((newValues, bs)) } new VertexSetRDD[U](index, newValues) @@ -283,18 +277,16 @@ class VertexSetRDD[@specialized V: ClassManifest]( if(index != other.index) { throw new SparkException("A zipJoin can only be applied to RDDs with the same index!") } - val newValuesRDD: RDD[ (IndexedSeq[(V,W)], BitSet) ] = + val newValuesRDD: RDD[ (Int => (V,W), BitSet) ] = valuesRDD.zipPartitions(other.valuesRDD){ - (thisIter: Iterator[(IndexedSeq[V], BitSet)], - otherIter: Iterator[(IndexedSeq[W], BitSet)]) => + (thisIter: Iterator[(Int => V, BitSet)], + otherIter: Iterator[(Int => W, BitSet)]) => val (thisValues, thisBS: BitSet) = thisIter.next() assert(!thisIter.hasNext) val (otherValues, otherBS: BitSet) = otherIter.next() assert(!otherIter.hasNext) val newBS: BitSet = thisBS & otherBS - val newValues: IndexedSeq[(V,W)] = - thisValues.view.zip(otherValues).toIndexedSeq // @todo check the toIndexedSeq is free - // Iterator((newValues.toIndexedSeq, newBS)) + val newValues: Int => (V,W) = (ind: Int) => (thisValues(ind), otherValues(ind)) Iterator((newValues, newBS)) } new VertexSetRDD(index, newValuesRDD) @@ -320,23 +312,17 @@ class VertexSetRDD[@specialized V: ClassManifest]( if(index != other.index) { throw new SparkException("A zipJoin can only be applied to RDDs with the same index!") } - val newValuesRDD: RDD[ (IndexedSeq[(V,Option[W])], BitSet) ] = + val newValuesRDD: RDD[ (Int => (V,Option[W]), BitSet) ] = valuesRDD.zipPartitions(other.valuesRDD){ - (thisIter: Iterator[(IndexedSeq[V], BitSet)], - otherIter: Iterator[(IndexedSeq[W], BitSet)]) => + (thisIter: Iterator[(Int => V, BitSet)], + otherIter: Iterator[(Int => W, BitSet)]) => val (thisValues, thisBS: BitSet) = thisIter.next() assert(!thisIter.hasNext) val (otherValues, otherBS: BitSet) = otherIter.next() assert(!otherIter.hasNext) - val newValues: IndexedSeq[(V, Option[W])] = thisValues.view.zip(otherValues) - .zipWithIndex.map { - // @todo not sure about the efficiency of this case statement - // though it is assumed that the return value is short lived - case ((v, w), ind) => (v, if (otherBS.get(ind)) Option(w) else None) - } - .toIndexedSeq // @todo check the toIndexedSeq is free + val newValues: Int => (V, Option[W]) = (ind: Int) => + (thisValues(ind), if (otherBS.get(ind)) Option(otherValues(ind)) else None) Iterator((newValues, thisBS)) - // Iterator((newValues.toIndexedSeq, thisBS)) } new VertexSetRDD(index, newValuesRDD) } // end of leftZipJoin @@ -380,10 +366,10 @@ class VertexSetRDD[@specialized V: ClassManifest]( if (other.partitioner == partitioner) other else other.partitionBy(partitioner.get) // Compute the new values RDD - val newValuesRDD: RDD[ (IndexedSeq[(V,Option[W])], BitSet) ] = + val newValuesRDD: RDD[ (Int => (V,Option[W]), BitSet) ] = index.rdd.zipPartitions(valuesRDD, otherShuffled) { (thisIndexIter: Iterator[VertexIdToIndexMap], - thisIter: Iterator[(IndexedSeq[V], BitSet)], + thisIter: Iterator[(Int => V, BitSet)], tuplesIter: Iterator[(Vid,W)]) => // Get the Index and values for this RDD val index = thisIndexIter.next() @@ -391,9 +377,9 @@ class VertexSetRDD[@specialized V: ClassManifest]( val (thisValues, thisBS) = thisIter.next() assert(!thisIter.hasNext) // Create a new array to store the values in the resulting VertexSet - val otherValues = new Array[W](thisValues.size) + val otherValues = new Array[W](index.capacity) // track which values are matched with values in other - val otherBS = new BitSet(thisValues.size) + val otherBS = new BitSet(index.capacity) for ((k,w) <- tuplesIter) { // Get the location of the key in the index val pos = index.getPos(k) @@ -412,13 +398,8 @@ class VertexSetRDD[@specialized V: ClassManifest]( } // Some vertices in this vertex set may not have a corresponding // tuple in the join and so a None value should be returned. - val newValues: IndexedSeq[(V, Option[W])] = thisValues.view.zip(otherValues) - .zipWithIndex.map { - // @todo not sure about the efficiency of this case statement - // though it is assumed that the return value is short lived - case ((v, w), ind) => (v, if (otherBS.get(ind)) Option(w) else None) - } - .toIndexedSeq // @todo check the toIndexedSeq is free + val newValues: Int => (V, Option[W]) = (ind: Int) => + (thisValues(ind), if (otherBS.get(ind)) Option(otherValues(ind)) else None) Iterator((newValues, thisBS)) } // end of newValues new VertexSetRDD(index, newValuesRDD) @@ -644,13 +625,13 @@ object VertexSetRDD { hashMap.setMerge(k, v, reduceFunc) } val index = hashMap.keySet - val values: IndexedSeq[V] = hashMap._values + val values: Int => V = (ind: Int) => hashMap._values(ind) val bs = index.getBitSet Iterator( (index, (values, bs)) ) }, true).cache // extract the index and the values val index = groups.mapPartitions(_.map{ case (kMap, vAr) => kMap }, true) - val values: RDD[(IndexedSeq[V], BitSet)] = + val values: RDD[(Int => V, BitSet)] = groups.mapPartitions(_.map{ case (kMap,vAr) => vAr }, true) new VertexSetRDD[V](new VertexSetIndex(index), values) } // end of apply @@ -726,7 +707,7 @@ object VertexSetRDD { } // Use the index to build the new values table - val values: RDD[ (IndexedSeq[C], BitSet) ] = index.rdd.zipPartitions(partitioned)( (indexIter, tblIter) => { + val values: RDD[ (Int => C, BitSet) ] = index.rdd.zipPartitions(partitioned)( (indexIter, tblIter) => { // There is only one map val index = indexIter.next() assert(!indexIter.hasNext()) @@ -750,7 +731,7 @@ object VertexSetRDD { } } } - Iterator((values, bs)) + Iterator(((ind: Int) => values(ind), bs)) }) new VertexSetRDD(index, values) } // end of apply From db89ac4bc8cbabaa13b36b75f4be2d96c29cb83a Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Thu, 31 Oct 2013 21:19:26 -0700 Subject: [PATCH 175/531] Changing var to val for keySet in OpenHashMaps --- .../src/main/scala/org/apache/spark/util/hash/OpenHashMap.scala | 2 +- .../org/apache/spark/util/hash/PrimitiveKeyOpenHashMap.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/hash/OpenHashMap.scala b/core/src/main/scala/org/apache/spark/util/hash/OpenHashMap.scala index 4eb52158b02d4..e53551ced62ec 100644 --- a/core/src/main/scala/org/apache/spark/util/hash/OpenHashMap.scala +++ b/core/src/main/scala/org/apache/spark/util/hash/OpenHashMap.scala @@ -27,7 +27,7 @@ package org.apache.spark.util.hash */ private[spark] class OpenHashMap[K >: Null : ClassManifest, @specialized(Long, Int, Double) V: ClassManifest]( - var keySet: OpenHashSet[K], var _values: Array[V]) + val keySet: OpenHashSet[K], var _values: Array[V]) extends Iterable[(K, V)] with Serializable { diff --git a/core/src/main/scala/org/apache/spark/util/hash/PrimitiveKeyOpenHashMap.scala b/core/src/main/scala/org/apache/spark/util/hash/PrimitiveKeyOpenHashMap.scala index 1bf2554fb7730..08fc74e5da6ba 100644 --- a/core/src/main/scala/org/apache/spark/util/hash/PrimitiveKeyOpenHashMap.scala +++ b/core/src/main/scala/org/apache/spark/util/hash/PrimitiveKeyOpenHashMap.scala @@ -28,7 +28,7 @@ package org.apache.spark.util.hash private[spark] class PrimitiveKeyOpenHashMap[@specialized(Long, Int) K: ClassManifest, @specialized(Long, Int, Double) V: ClassManifest]( - var keySet: OpenHashSet[K], var _values: Array[V]) + val keySet: OpenHashSet[K], var _values: Array[V]) extends Iterable[(K, V)] with Serializable { From 3c37928fab0801fa1e2662d873dac4b4f93c547d Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Mon, 4 Nov 2013 20:10:15 -0800 Subject: [PATCH 176/531] This commit adds a new graphx-shell which is essentially the same as the spark shell but with GraphX packages automatically imported and with Kryo serialization enabled for GraphX types. In addition the graphx-shell has a nifty new logo. To make these changes minimally invasive in the SparkILoop.scala I added some additional environment variables: SPARK_BANNER_TEXT: If set this string is displayed instead of the spark logo SPARK_SHELL_INIT_BLOCK: if set this expression is evaluated in the spark shell after the spark context is created. --- graphx-shell | 124 +++++++++++++ .../org/apache/spark/repl/SparkILoop.scala | 174 +++++++++--------- 2 files changed, 215 insertions(+), 83 deletions(-) create mode 100755 graphx-shell diff --git a/graphx-shell b/graphx-shell new file mode 100755 index 0000000000000..4dd6c68ace888 --- /dev/null +++ b/graphx-shell @@ -0,0 +1,124 @@ +#!/usr/bin/env bash + +# +# 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. +# + +# +# Shell script for starting the Spark Shell REPL +# Note that it will set MASTER to spark://${SPARK_MASTER_IP}:${SPARK_MASTER_PORT} +# if those two env vars are set in spark-env.sh but MASTER is not. +# Options: +# -c Set the number of cores for REPL to use +# + +# Enter posix mode for bash +set -o posix + + +# Update the the banner logo +export SPARK_BANNER_TEXT="Welcome to + ______ __ _ __ + / ____/________ _____ / /_ | |/ / + / / __/ ___/ __ \`/ __ \/ __ \| / + / /_/ / / / /_/ / /_/ / / / / | + \____/_/ \__,_/ .___/_/ /_/_/|_| + /_/ Alpha Release + +Powered by: + ____ __ + / __/__ ___ _____/ /__ + _\ \/ _ \/ _ \`/ __/ '_/ + /___/ .__/\_,_/_/ /_/\_\ + /_/ version 0.9.0 + +Example: + + scala> val graph = GraphLoader.textFile(sc, \"hdfs://links\") + scala> graph.numVertices + scala> graph.numEdges + scala> val pageRankGraph = Analytics.pagerank(graph, 10) // 10 iterations + scala> val maxPr = pageRankGraph.vertices.map{ case (vid, pr) => pr }.max + scala> println(maxPr) + +" + +export SPARK_SHELL_INIT_BLOCK="import org.apache.spark.graph._;" + +# Set the serializer to use Kryo for graphx objects +SPARK_JAVA_OPTS+=" -Dspark.serializer=org.apache.spark.serializer.KryoSerializer " +SPARK_JAVA_OPTS+="-Dspark.kryo.registrator=org.apache.spark.graph.GraphKryoRegistrator " +SPARK_JAVA_OPTS+="-Dspark.kryoserializer.buffer.mb=10 " + + + +FWDIR="`dirname $0`" + +for o in "$@"; do + if [ "$1" = "-c" -o "$1" = "--cores" ]; then + shift + if [ -n "$1" ]; then + OPTIONS="-Dspark.cores.max=$1" + shift + fi + fi +done + +# Set MASTER from spark-env if possible +if [ -z "$MASTER" ]; then + if [ -e "$FWDIR/conf/spark-env.sh" ]; then + . "$FWDIR/conf/spark-env.sh" + fi + if [[ "x" != "x$SPARK_MASTER_IP" && "y" != "y$SPARK_MASTER_PORT" ]]; then + MASTER="spark://${SPARK_MASTER_IP}:${SPARK_MASTER_PORT}" + export MASTER + fi +fi + +# Copy restore-TTY-on-exit functions from Scala script so spark-shell exits properly even in +# binary distribution of Spark where Scala is not installed +exit_status=127 +saved_stty="" + +# restore stty settings (echo in particular) +function restoreSttySettings() { + stty $saved_stty + saved_stty="" +} + +function onExit() { + if [[ "$saved_stty" != "" ]]; then + restoreSttySettings + fi + exit $exit_status +} + +# to reenable echo if we are interrupted before completing. +trap onExit INT + +# save terminal settings +saved_stty=$(stty -g 2>/dev/null) +# clear on error so we don't later try to restore them +if [[ ! $? ]]; then + saved_stty="" +fi + +$FWDIR/spark-class $OPTIONS org.apache.spark.repl.Main "$@" + +# record the exit status lest it be overwritten: +# then reenable echo and propagate the code. +exit_status=$? +onExit diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala index 0ced284da68f5..efdd90c47f7c8 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala @@ -45,7 +45,7 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: def this(in0: BufferedReader, out: PrintWriter, master: String) = this(Some(in0), out, Some(master)) def this(in0: BufferedReader, out: PrintWriter) = this(Some(in0), out, None) def this() = this(None, new PrintWriter(Console.out, true), None) - + var in: InteractiveReader = _ // the input stream from which commands come var settings: Settings = _ var intp: SparkIMain = _ @@ -56,16 +56,16 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: Power[g.type](this, g) } */ - + // TODO // object opt extends AestheticSettings - // + // @deprecated("Use `intp` instead.", "2.9.0") def interpreter = intp - + @deprecated("Use `intp` instead.", "2.9.0") def interpreter_= (i: SparkIMain): Unit = intp = i - + def history = in.history /** The context class loader at the time this object was created */ @@ -75,7 +75,7 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: private val signallable = /*if (isReplDebug) Signallable("Dump repl state.")(dumpCommand()) else*/ null - + // classpath entries added via :cp var addedClasspath: String = "" @@ -87,10 +87,10 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: /** Record a command for replay should the user request a :replay */ def addReplay(cmd: String) = replayCommandStack ::= cmd - + /** Try to install sigint handler: ignore failure. Signal handler * will interrupt current line execution if any is in progress. - * + * * Attempting to protect the repl from accidental exit, we only honor * a single ctrl-C if the current buffer is empty: otherwise we look * for a second one within a short time. @@ -124,7 +124,7 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: Thread.currentThread.setContextClassLoader(originalClassLoader) } } - + class SparkILoopInterpreter extends SparkIMain(settings, out) { override lazy val formatting = new Formatting { def prompt = SparkILoop.this.prompt @@ -135,7 +135,7 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: |// She's gone rogue, captain! Have to take her out! |// Calling Thread.stop on runaway %s with offending code: |// scala> %s""".stripMargin - + echo(template.format(line.thread, line.code)) // XXX no way to suppress the deprecation warning line.thread.stop() @@ -151,7 +151,7 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: def createInterpreter() { if (addedClasspath != "") settings.classpath append addedClasspath - + intp = new SparkILoopInterpreter intp.setContextClassLoader() installSigIntHandler() @@ -168,10 +168,10 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: private def helpSummary() = { val usageWidth = commands map (_.usageMsg.length) max val formatStr = "%-" + usageWidth + "s %s %s" - + echo("All commands can be abbreviated, e.g. :he instead of :help.") echo("Those marked with a * have more detailed help, e.g. :help imports.\n") - + commands foreach { cmd => val star = if (cmd.hasLongHelp) "*" else " " echo(formatStr.format(cmd.usageMsg, star, cmd.help)) @@ -182,7 +182,7 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: case Nil => echo(cmd + ": no such command. Type :help for help.") case xs => echo(cmd + " is ambiguous: did you mean " + xs.map(":" + _.name).mkString(" or ") + "?") } - Result(true, None) + Result(true, None) } private def matchingCommands(cmd: String) = commands filter (_.name startsWith cmd) private def uniqueCommand(cmd: String): Option[LoopCommand] = { @@ -193,31 +193,35 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: case xs => xs find (_.name == cmd) } } - + /** Print a welcome message */ def printWelcome() { - echo("""Welcome to - ____ __ + val prop = System.getenv("SPARK_BANNER_TEXT") + val bannerText = + if (prop != null) prop else + """Welcome to + ____ __ / __/__ ___ _____/ /__ _\ \/ _ \/ _ `/ __/ '_/ /___/ .__/\_,_/_/ /_/\_\ version 0.9.0-SNAPSHOT - /_/ -""") + /_/ + """ + echo(bannerText) import Properties._ val welcomeMsg = "Using Scala %s (%s, Java %s)".format( - versionString, javaVmName, javaVersion) + versionString, javaVmName, javaVersion) echo(welcomeMsg) } - + /** Show the history */ lazy val historyCommand = new LoopCommand("history", "show the history (optional num is commands to show)") { override def usage = "[num]" def defaultLines = 20 - + def apply(line: String): Result = { if (history eq NoHistory) return "No history available." - + val xs = words(line) val current = history.index val count = try xs.head.toInt catch { case _: Exception => defaultLines } @@ -237,21 +241,21 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: out print msg out.flush() } - + /** Search the history */ def searchHistory(_cmdline: String) { val cmdline = _cmdline.toLowerCase val offset = history.index - history.size + 1 - + for ((line, index) <- history.asStrings.zipWithIndex ; if line.toLowerCase contains cmdline) echo("%d %s".format(index + offset, line)) } - + private var currentPrompt = Properties.shellPromptString def setPrompt(prompt: String) = currentPrompt = prompt /** Prompt to print when awaiting input */ def prompt = currentPrompt - + import LoopCommand.{ cmd, nullary } /** Standard commands **/ @@ -273,7 +277,7 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: nullary("silent", "disable/enable automatic printing of results", verbosity), cmd("type", "", "display the type of an expression without evaluating it", typeCommand) ) - + /** Power user commands */ lazy val powerCommands: List[LoopCommand] = List( //nullary("dump", "displays a view of the interpreter's internal state", dumpCommand), @@ -298,10 +302,10 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: |An argument of clear will remove the wrapper if any is active. |Note that wrappers do not compose (a new one replaces the old |one) and also that the :phase command uses the same machinery, - |so setting :wrap will clear any :phase setting. + |so setting :wrap will clear any :phase setting. """.stripMargin.trim) ) - + /* private def dumpCommand(): Result = { echo("" + power) @@ -309,7 +313,7 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: in.redrawLine() } */ - + private val typeTransforms = List( "scala.collection.immutable." -> "immutable.", "scala.collection.mutable." -> "mutable.", @@ -317,7 +321,7 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: "java.lang." -> "jl.", "scala.runtime." -> "runtime." ) - + private def importsCommand(line: String): Result = { val tokens = words(line) val handlers = intp.languageWildcardHandlers ++ intp.importHandlers @@ -333,7 +337,7 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: val implicitMsg = if (imps.isEmpty) "" else imps.size + " are implicit" val foundMsg = if (found.isEmpty) "" else found.mkString(" // imports: ", ", ", "") val statsMsg = List(typeMsg, termMsg, implicitMsg) filterNot (_ == "") mkString ("(", ", ", ")") - + intp.reporter.printMessage("%2d) %-30s %s%s".format( idx + 1, handler.importString, @@ -342,12 +346,12 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: )) } } - + private def implicitsCommand(line: String): Result = { val intp = SparkILoop.this.intp import intp._ import global.Symbol - + def p(x: Any) = intp.reporter.printMessage("" + x) // If an argument is given, only show a source with that @@ -360,14 +364,14 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: else (args exists (source.name.toString contains _)) } } - + if (filtered.isEmpty) return "No implicits have been imported other than those in Predef." - + filtered foreach { case (source, syms) => p("/* " + syms.size + " implicit members imported from " + source.fullName + " */") - + // This groups the members by where the symbol is defined val byOwner = syms groupBy (_.owner) val sortedOwners = byOwner.toList sortBy { case (owner, _) => intp.afterTyper(source.info.baseClasses indexOf owner) } @@ -388,10 +392,10 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: xss map (xs => xs sortBy (_.name.toString)) } - - val ownerMessage = if (owner == source) " defined in " else " inherited from " + + val ownerMessage = if (owner == source) " defined in " else " inherited from " p(" /* " + members.size + ownerMessage + owner.fullName + " */") - + memberGroups foreach { group => group foreach (s => p(" " + intp.symbolDefString(s))) p("") @@ -400,7 +404,7 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: p("") } } - + protected def newJavap() = new Javap(intp.classLoader, new SparkIMain.ReplStrippingWriter(intp)) { override def tryClass(path: String): Array[Byte] = { // Look for Foo first, then Foo$, but if Foo$ is given explicitly, @@ -417,20 +421,20 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: private lazy val javap = try newJavap() catch { case _: Exception => null } - + private def typeCommand(line: String): Result = { intp.typeOfExpression(line) match { case Some(tp) => tp.toString case _ => "Failed to determine type." } } - + private def javapCommand(line: String): Result = { if (javap == null) return ":javap unavailable on this platform." if (line == "") return ":javap [-lcsvp] [path1 path2 ...]" - + javap(words(line)) foreach { res => if (res.isError) return "Failed: " + res.value else res.show() @@ -504,25 +508,25 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: } else { val what = phased.parse(name) - if (what.isEmpty || !phased.set(what)) + if (what.isEmpty || !phased.set(what)) "'" + name + "' does not appear to represent a valid phase." else { intp.setExecutionWrapper(pathToPhaseWrapper) val activeMessage = if (what.toString.length == name.length) "" + what else "%s (%s)".format(what, name) - + "Active phase is now: " + activeMessage } } } */ - + /** Available commands */ def commands: List[LoopCommand] = standardCommands /* ++ ( if (isReplPower) powerCommands else Nil )*/ - + val replayQuestionMessage = """|The repl compiler has crashed spectacularly. Shall I replay your |session? I can re-run all lines except the last one. @@ -579,10 +583,10 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: } /** interpret all lines from a specified file */ - def interpretAllFrom(file: File) { + def interpretAllFrom(file: File) { val oldIn = in val oldReplay = replayCommandStack - + try file applyReader { reader => in = SimpleReader(reader, out, false) echo("Loading " + file + "...") @@ -604,26 +608,26 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: echo("") } } - + /** fork a shell and run a command */ lazy val shCommand = new LoopCommand("sh", "run a shell command (result is implicitly => List[String])") { override def usage = "" def apply(line: String): Result = line match { case "" => showUsage() - case _ => + case _ => val toRun = classOf[ProcessResult].getName + "(" + string2codeQuoted(line) + ")" intp interpret toRun () } } - + def withFile(filename: String)(action: File => Unit) { val f = File(filename) - + if (f.exists) action(f) else echo("That file does not exist") } - + def loadCommand(arg: String) = { var shouldReplay: Option[String] = None withFile(arg)(f => { @@ -657,7 +661,7 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: } else echo("The path '" + f + "' doesn't seem to exist.") } - + def powerCmd(): Result = { if (isReplPower) "Already in power mode." else enablePowerMode() @@ -667,13 +671,13 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: //power.unleash() //echo(power.banner) } - + def verbosity() = { val old = intp.printResults intp.printResults = !old echo("Switched " + (if (old) "off" else "on") + " result printing.") } - + /** Run one command submitted by the user. Two values are returned: * (1) whether to keep running, (2) the line to record for replay, * if any. */ @@ -688,11 +692,11 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: else if (intp.global == null) Result(false, None) // Notice failure to create compiler else Result(true, interpretStartingWith(line)) } - + private def readWhile(cond: String => Boolean) = { Iterator continually in.readLine("") takeWhile (x => x != null && cond(x)) } - + def pasteCommand(): Result = { echo("// Entering paste mode (ctrl-D to finish)\n") val code = readWhile(_ => true) mkString "\n" @@ -700,17 +704,17 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: intp interpret code () } - + private object paste extends Pasted { val ContinueString = " | " val PromptString = "scala> " - + def interpret(line: String): Unit = { echo(line.trim) intp interpret line echo("") } - + def transcript(start: String) = { // Printing this message doesn't work very well because it's buried in the // transcript they just pasted. Todo: a short timer goes off when @@ -731,7 +735,7 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: def interpretStartingWith(code: String): Option[String] = { // signal completion non-completion input has been received in.completion.resetVerbosity() - + def reallyInterpret = { val reallyResult = intp.interpret(code) (reallyResult, reallyResult match { @@ -741,7 +745,7 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: if (in.interactive && code.endsWith("\n\n")) { echo("You typed two blank lines. Starting a new command.") None - } + } else in.readLine(ContinueString) match { case null => // we know compilation is going to fail since we're at EOF and the @@ -755,10 +759,10 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: } }) } - + /** Here we place ourselves between the user and the interpreter and examine * the input they are ostensibly submitting. We intervene in several cases: - * + * * 1) If the line starts with "scala> " it is assumed to be an interpreter paste. * 2) If the line starts with "." (but not ".." or "./") it is treated as an invocation * on the previous result. @@ -787,7 +791,7 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: val (code, result) = reallyInterpret //if (power != null && code == IR.Error) // runCompletion - + result } else runCompletion match { @@ -808,7 +812,7 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: } case _ => } - + /** Tries to create a JLineReader, falling back to SimpleReader: * unless settings or properties are such that it should start * with SimpleReader. @@ -837,6 +841,10 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: org.apache.spark.repl.Main.interp.out.flush(); """) command("import org.apache.spark.SparkContext._") + val prop = System.getenv("SPARK_SHELL_INIT_BLOCK") + if (prop != null) { + command(prop) + } } echo("Type in expressions to have them evaluated.") echo("Type :help for more information.") @@ -884,7 +892,7 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: this.settings = settings createInterpreter() - + // sets in to some kind of reader depending on environmental cues in = in0 match { case Some(reader) => SimpleReader(reader, out, true) @@ -895,10 +903,10 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: // it is broken on startup; go ahead and exit if (intp.reporter.hasErrors) return false - - try { + + try { // this is about the illusion of snappiness. We call initialize() - // which spins off a separate thread, then print the prompt and try + // which spins off a separate thread, then print the prompt and try // our best to look ready. Ideally the user will spend a // couple seconds saying "wow, it starts so fast!" and by the time // they type a command the compiler is ready to roll. @@ -920,19 +928,19 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: def neededHelp(): String = (if (command.settings.help.value) command.usageMsg + "\n" else "") + (if (command.settings.Xhelp.value) command.xusageMsg + "\n" else "") - + // if they asked for no help and command is valid, we call the real main neededHelp() match { case "" => command.ok && process(command.settings) case help => echoNoNL(help) ; true } } - + @deprecated("Use `process` instead", "2.9.0") def main(args: Array[String]): Unit = { if (isReplDebug) System.out.println(new java.util.Date) - + process(args) } @deprecated("Use `process` instead", "2.9.0") @@ -948,7 +956,7 @@ object SparkILoop { // like if you'd just typed it into the repl. def runForTranscript(code: String, settings: Settings): String = { import java.io.{ BufferedReader, StringReader, OutputStreamWriter } - + stringFromStream { ostream => Console.withOut(ostream) { val output = new PrintWriter(new OutputStreamWriter(ostream), true) { @@ -977,19 +985,19 @@ object SparkILoop { } } } - + /** Creates an interpreter loop with default settings and feeds * the given code to it as input. */ def run(code: String, sets: Settings = new Settings): String = { import java.io.{ BufferedReader, StringReader, OutputStreamWriter } - + stringFromStream { ostream => Console.withOut(ostream) { val input = new BufferedReader(new StringReader(code)) val output = new PrintWriter(new OutputStreamWriter(ostream), true) val repl = new SparkILoop(input, output) - + if (sets.classpath.isDefault) sets.classpath.value = sys.props("java.class.path") @@ -1017,7 +1025,7 @@ object SparkILoop { repl.settings.embeddedDefaults[T] repl.createInterpreter() repl.in = SparkJLineReader(repl) - + // rebind exit so people don't accidentally call sys.exit by way of predef repl.quietRun("""def exit = println("Type :quit to resume program execution.")""") args foreach (p => repl.bind(p.name, p.tpe, p.value)) @@ -1025,5 +1033,5 @@ object SparkILoop { echo("\nDebug repl exiting.") repl.closeInterpreter() - } + } } From 2dc9ec23874eb7174183134bb2ae6050b4ff270d Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Tue, 5 Nov 2013 01:15:12 -0800 Subject: [PATCH 177/531] Reverting to Array based (materialized) output of all VertexSetRDD operations. --- .../org/apache/spark/graph/GraphOps.scala | 11 +- .../org/apache/spark/graph/VertexSetRDD.scala | 178 +++++++++--------- .../apache/spark/graph/impl/GraphImpl.scala | 11 +- .../apache/spark/graph/AnalyticsSuite.scala | 36 ++-- .../org/apache/spark/graph/GraphSuite.scala | 6 +- 5 files changed, 121 insertions(+), 121 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala b/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala index 8480ff29d34d0..8c7f4c25e2952 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala @@ -154,11 +154,7 @@ class GraphOps[VD: ClassManifest, ED: ClassManifest](graph: Graph[VD, ED]) { (vid, edge) => Some(Array(edge.otherVertexId(vid))), (a, b) => a ++ b, edgeDirection) - - graph.vertices.leftZipJoin(nbrs).mapValues{ - case (_, Some(nbrs)) => nbrs - case (_, None) => Array.empty[Vid] - } + graph.vertices.leftZipJoin(nbrs) { (vid, vdata, nbrsOpt) => nbrsOpt.getOrElse(Array.empty[Vid]) } } // end of collectNeighborIds @@ -183,10 +179,7 @@ class GraphOps[VD: ClassManifest, ED: ClassManifest](graph: Graph[VD, ED]) { (a, b) => a ++ b, edgeDirection) - graph.vertices.leftZipJoin(nbrs).mapValues{ - case (_, Some(nbrs)) => nbrs - case (_, None) => Array.empty[(Vid, VD)] - } + graph.vertices.leftZipJoin(nbrs) { (vid, vdata, nbrsOpt) => nbrsOpt.getOrElse(Array.empty[(Vid, VD)]) } } // end of collectNeighbor diff --git a/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala b/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala index f26e286003033..5cb05998aaaf8 100644 --- a/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala @@ -98,7 +98,7 @@ class VertexSetIndex(private[spark] val rdd: RDD[VertexIdToIndexMap]) { */ class VertexSetRDD[@specialized V: ClassManifest]( @transient val index: VertexSetIndex, - @transient val valuesRDD: RDD[ ( (Int => V), BitSet) ]) + @transient val valuesRDD: RDD[ ( Array[V], BitSet) ]) extends RDD[(Vid, V)](index.rdd.context, List(new OneToOneDependency(index.rdd), new OneToOneDependency(valuesRDD)) ) { @@ -182,7 +182,7 @@ class VertexSetRDD[@specialized V: ClassManifest]( val cleanPred = index.rdd.context.clean(pred) val newValues = index.rdd.zipPartitions(valuesRDD){ (keysIter: Iterator[VertexIdToIndexMap], - valuesIter: Iterator[(Int => V, BitSet)]) => + valuesIter: Iterator[(Array[V], BitSet)]) => val index = keysIter.next() assert(keysIter.hasNext() == false) val (oldValues, bs) = valuesIter.next() @@ -217,11 +217,12 @@ class VertexSetRDD[@specialized V: ClassManifest]( * VertexSetRDD retains the same index. */ def mapValues[U: ClassManifest](f: V => U): VertexSetRDD[U] = { - val newValuesRDD: RDD[ (Int => U, BitSet) ] = + val cleanF = index.rdd.context.clean(f) + val newValuesRDD: RDD[ (Array[U], BitSet) ] = valuesRDD.mapPartitions(iter => iter.map{ - case (values, bs: BitSet) => - val newValues: (Int => U) = - (ind: Int) => if (bs.get(ind)) f(values(ind)) else null.asInstanceOf[U] + case (values, bs: BitSet) => + val newValues = new Array[U](values.size) + bs.iterator.foreach { ind => newValues(ind) = cleanF(values(ind)) } (newValues, bs) }, preservesPartitioning = true) new VertexSetRDD[U](index, newValuesRDD) @@ -241,19 +242,18 @@ class VertexSetRDD[@specialized V: ClassManifest]( * VertexSetRDD retains the same index. */ def mapValuesWithKeys[U: ClassManifest](f: (Vid, V) => U): VertexSetRDD[U] = { - val newValues: RDD[ (Int => U, BitSet) ] = + val cleanF = index.rdd.context.clean(f) + val newValues: RDD[ (Array[U], BitSet) ] = index.rdd.zipPartitions(valuesRDD){ (keysIter: Iterator[VertexIdToIndexMap], - valuesIter: Iterator[(Int => V, BitSet)]) => + valuesIter: Iterator[(Array[V], BitSet)]) => val index = keysIter.next() assert(keysIter.hasNext() == false) - val (oldValues, bs: BitSet) = valuesIter.next() + val (values, bs: BitSet) = valuesIter.next() assert(valuesIter.hasNext() == false) // Cosntruct a view of the map transformation - val newValues: (Int => U) = (ind: Int) => { - if (bs.get(ind)) { f(index.getValueSafe(ind), oldValues(ind)) } - else { null.asInstanceOf[U] } - } + val newValues = new Array[U](index.capacity) + bs.iterator.foreach { ind => newValues(ind) = cleanF(index.getValueSafe(ind), values(ind)) } Iterator((newValues, bs)) } new VertexSetRDD[U](index, newValues) @@ -261,6 +261,8 @@ class VertexSetRDD[@specialized V: ClassManifest]( /** + * @todo update docs to reflect function argument + * * Inner join this VertexSet with another VertexSet which has the * same Index. This function will fail if both VertexSets do not * share the same index. The resulting vertex set will only contain @@ -273,20 +275,25 @@ class VertexSetRDD[@specialized V: ClassManifest]( * and the other VertexSet and with tuple attributes. * */ - def zipJoin[W: ClassManifest](other: VertexSetRDD[W]): VertexSetRDD[(V,W)] = { + def zipJoin[W: ClassManifest, Z: ClassManifest](other: VertexSetRDD[W])(f: (Vid, V, W) => Z): + VertexSetRDD[Z] = { + val cleanF = index.rdd.context.clean(f) if(index != other.index) { throw new SparkException("A zipJoin can only be applied to RDDs with the same index!") } - val newValuesRDD: RDD[ (Int => (V,W), BitSet) ] = - valuesRDD.zipPartitions(other.valuesRDD){ - (thisIter: Iterator[(Int => V, BitSet)], - otherIter: Iterator[(Int => W, BitSet)]) => + val newValuesRDD: RDD[ (Array[Z], BitSet) ] = + index.rdd.zipPartitions(valuesRDD, other.valuesRDD) { (indexIter, thisIter, otherIter) => + val index = indexIter.next() + assert(!indexIter.hasNext) val (thisValues, thisBS: BitSet) = thisIter.next() assert(!thisIter.hasNext) val (otherValues, otherBS: BitSet) = otherIter.next() assert(!otherIter.hasNext) val newBS: BitSet = thisBS & otherBS - val newValues: Int => (V,W) = (ind: Int) => (thisValues(ind), otherValues(ind)) + val newValues = new Array[Z](index.capacity) + newBS.iterator.foreach { ind => + newValues(ind) = cleanF(index.getValueSafe(ind), thisValues(ind), otherValues(ind)) + } Iterator((newValues, newBS)) } new VertexSetRDD(index, newValuesRDD) @@ -294,6 +301,37 @@ class VertexSetRDD[@specialized V: ClassManifest]( /** + * @todo document + * + * @param other + * @param f + * @tparam W + * @tparam Z + * @return + */ + def zipJoinFlatMap[W: ClassManifest, Z: ClassManifest](other: VertexSetRDD[W])(f: (Vid, V,W) => Iterator[Z]): + RDD[Z] = { + val cleanF = index.rdd.context.clean(f) + if(index != other.index) { + throw new SparkException("A zipJoin can only be applied to RDDs with the same index!") + } + index.rdd.zipPartitions(valuesRDD, other.valuesRDD) { (indexIter, thisIter, otherIter) => + val index = indexIter.next() + assert(!indexIter.hasNext) + val (thisValues, thisBS: BitSet) = thisIter.next() + assert(!thisIter.hasNext) + val (otherValues, otherBS: BitSet) = otherIter.next() + assert(!otherIter.hasNext) + val newBS: BitSet = thisBS & otherBS + val newValues = new Array[Z](index.capacity) + newBS.iterator.flatMap { ind => cleanF(index.getValueSafe(ind), thisValues(ind), otherValues(ind)) } + } + } + + + /** + * @todo update docs to reflect function argument + * Left join this VertexSet with another VertexSet which has the * same Index. This function will fail if both VertexSets do not * share the same index. The resulting vertex set contains an entry @@ -308,20 +346,25 @@ class VertexSetRDD[@specialized V: ClassManifest]( * other VertexSet. * */ - def leftZipJoin[W: ClassManifest](other: VertexSetRDD[W]): VertexSetRDD[(V,Option[W])] = { + def leftZipJoin[W: ClassManifest, Z: ClassManifest](other: VertexSetRDD[W])(f: (Vid, V, Option[W]) => Z): + VertexSetRDD[Z] = { if(index != other.index) { throw new SparkException("A zipJoin can only be applied to RDDs with the same index!") } - val newValuesRDD: RDD[ (Int => (V,Option[W]), BitSet) ] = - valuesRDD.zipPartitions(other.valuesRDD){ - (thisIter: Iterator[(Int => V, BitSet)], - otherIter: Iterator[(Int => W, BitSet)]) => + val cleanF = index.rdd.context.clean(f) + val newValuesRDD: RDD[(Array[Z], BitSet)] = + index.rdd.zipPartitions(valuesRDD, other.valuesRDD) { (indexIter, thisIter, otherIter) => + val index = indexIter.next() + assert(!indexIter.hasNext) val (thisValues, thisBS: BitSet) = thisIter.next() assert(!thisIter.hasNext) val (otherValues, otherBS: BitSet) = otherIter.next() assert(!otherIter.hasNext) - val newValues: Int => (V, Option[W]) = (ind: Int) => - (thisValues(ind), if (otherBS.get(ind)) Option(otherValues(ind)) else None) + val newValues = new Array[Z](index.capacity) + thisBS.iterator.foreach { ind => + val otherV = if (otherBS.get(ind)) Option(otherValues(ind)) else None + newValues(ind) = cleanF(index.getValueSafe(ind), thisValues(ind), otherV) + } Iterator((newValues, thisBS)) } new VertexSetRDD(index, newValuesRDD) @@ -346,68 +389,29 @@ class VertexSetRDD[@specialized V: ClassManifest]( * other VertexSet. * */ - def leftJoin[W: ClassManifest]( - other: RDD[(Vid,W)], merge: (W,W) => W = (a:W, b:W) => a): - VertexSetRDD[(V, Option[W]) ] = { + def leftJoin[W: ClassManifest, Z: ClassManifest](other: RDD[(Vid,W)]) + (f: (Vid, V, Option[W]) => Z, merge: (W,W) => W = (a:W, b:W) => a ): + VertexSetRDD[Z] = { + val cleanF = index.rdd.context.clean(f) + val cleanMerge = index.rdd.context.clean(merge) // Test if the other vertex is a VertexSetRDD to choose the optimal // join strategy other match { // If the other set is a VertexSetRDD and shares the same index as // this vertex set then we use the much more efficient leftZipJoin case other: VertexSetRDD[_] if index == other.index => { - leftZipJoin(other) - } + leftZipJoin(other)(cleanF) + // @todo handle case where other is a VertexSetRDD with a different index + } case _ => { - // Otherwise we treat the other RDD as a collectiong of - // vertex-attribute pairs. - // If necessary shuffle the other RDD using the partitioner - // for this VertexSet - val otherShuffled = - if (other.partitioner == partitioner) other - else other.partitionBy(partitioner.get) - // Compute the new values RDD - val newValuesRDD: RDD[ (Int => (V,Option[W]), BitSet) ] = - index.rdd.zipPartitions(valuesRDD, otherShuffled) { - (thisIndexIter: Iterator[VertexIdToIndexMap], - thisIter: Iterator[(Int => V, BitSet)], - tuplesIter: Iterator[(Vid,W)]) => - // Get the Index and values for this RDD - val index = thisIndexIter.next() - assert(!thisIndexIter.hasNext) - val (thisValues, thisBS) = thisIter.next() - assert(!thisIter.hasNext) - // Create a new array to store the values in the resulting VertexSet - val otherValues = new Array[W](index.capacity) - // track which values are matched with values in other - val otherBS = new BitSet(index.capacity) - for ((k,w) <- tuplesIter) { - // Get the location of the key in the index - val pos = index.getPos(k) - // Only if the key is already in the index - if ((pos & OpenHashSet.EXISTENCE_MASK) == 0) { - // Get the actual index - val ind = pos & OpenHashSet.POSITION_MASK - // If this value has already been seen then merge - if (otherBS.get(ind)) { - otherValues(ind) = merge(otherValues(ind), w) - } else { // otherwise just store the new value - otherBS.set(ind) - otherValues(ind) = w - } - } - } - // Some vertices in this vertex set may not have a corresponding - // tuple in the join and so a None value should be returned. - val newValues: Int => (V, Option[W]) = (ind: Int) => - (thisValues(ind), if (otherBS.get(ind)) Option(otherValues(ind)) else None) - Iterator((newValues, thisBS)) - } // end of newValues - new VertexSetRDD(index, newValuesRDD) + val indexedOther: VertexSetRDD[W] = VertexSetRDD(other, index, cleanMerge) + leftZipJoin(indexedOther)(cleanF) } } } // end of leftJoin + /** * For each key k in `this` or `other`, return a resulting RDD that contains a * tuple with the list of values for that key in `this` as well as `other`. @@ -609,29 +613,30 @@ object VertexSetRDD { */ def apply[V: ClassManifest]( rdd: RDD[(Vid,V)], reduceFunc: (V, V) => V): VertexSetRDD[V] = { + val cReduceFunc = rdd.context.clean(reduceFunc) // Preaggregate and shuffle if necessary val preAgg = rdd.partitioner match { case Some(p) => rdd case None => val partitioner = new HashPartitioner(rdd.partitions.size) // Preaggregation. - val aggregator = new Aggregator[Vid, V, V](v => v, reduceFunc, reduceFunc) + val aggregator = new Aggregator[Vid, V, V](v => v, cReduceFunc, cReduceFunc) rdd.mapPartitions(aggregator.combineValuesByKey, true).partitionBy(partitioner) } val groups = preAgg.mapPartitions( iter => { val hashMap = new PrimitiveKeyOpenHashMap[Vid, V] for ((k,v) <- iter) { - hashMap.setMerge(k, v, reduceFunc) + hashMap.setMerge(k, v, cReduceFunc) } val index = hashMap.keySet - val values: Int => V = (ind: Int) => hashMap._values(ind) + val values = hashMap._values val bs = index.getBitSet Iterator( (index, (values, bs)) ) }, true).cache // extract the index and the values val index = groups.mapPartitions(_.map{ case (kMap, vAr) => kMap }, true) - val values: RDD[(Int => V, BitSet)] = + val values: RDD[(Array[V], BitSet)] = groups.mapPartitions(_.map{ case (kMap,vAr) => vAr }, true) new VertexSetRDD[V](new VertexSetIndex(index), values) } // end of apply @@ -690,6 +695,9 @@ object VertexSetRDD { createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => C): VertexSetRDD[C] = { + val cCreateCombiner = index.rdd.context.clean(createCombiner) + val cMergeValue = index.rdd.context.clean(mergeValue) + val cMergeCombiners = index.rdd.context.clean(mergeCombiners) // Get the index Partitioner val partitioner = index.rdd.partitioner match { case Some(p) => p @@ -699,15 +707,15 @@ object VertexSetRDD { val partitioned = if (rdd.partitioner != Some(partitioner)) { // Preaggregation. - val aggregator = new Aggregator[Vid, V, C](createCombiner, mergeValue, - mergeCombiners) + val aggregator = new Aggregator[Vid, V, C](cCreateCombiner, cMergeValue, + cMergeCombiners) rdd.mapPartitions(aggregator.combineValuesByKey).partitionBy(partitioner) } else { rdd.mapValues(x => createCombiner(x)) } // Use the index to build the new values table - val values: RDD[ (Int => C, BitSet) ] = index.rdd.zipPartitions(partitioned)( (indexIter, tblIter) => { + val values: RDD[ (Array[C], BitSet) ] = index.rdd.zipPartitions(partitioned)( (indexIter, tblIter) => { // There is only one map val index = indexIter.next() assert(!indexIter.hasNext()) @@ -724,14 +732,14 @@ object VertexSetRDD { val ind = pos & OpenHashSet.POSITION_MASK // If this value has already been seen then merge if (bs.get(ind)) { - values(ind) = mergeCombiners(values(ind), c) + values(ind) = cMergeCombiners(values(ind), c) } else { // otherwise just store the new value bs.set(ind) values(ind) = c } } } - Iterator(((ind: Int) => values(ind), bs)) + Iterator((values, bs)) }) new VertexSetRDD(index, values) } // end of apply diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index b80713dbf4382..83ff2d734c8f6 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -315,8 +315,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( (updates: RDD[(Vid, U)])(updateF: (Vid, VD, Option[U]) => VD2) : Graph[VD2, ED] = { ClosureCleaner.clean(updateF) - val newVTable = vTable.leftJoin(updates).mapValuesWithKeys( - (vid, vu) => updateF(vid, vu._1, vu._2) ) + val newVTable = vTable.leftJoin(updates)(updateF) new GraphImpl(newVTable, vid2pid, localVidMap, eTable) } @@ -437,11 +436,9 @@ object GraphImpl { RDD[(Pid, Array[VD])] = { // Join vid2pid and vTable, generate a shuffle dependency on the joined // result, and get the shuffle id so we can use it on the slave. - val msgsByPartition = vTable.zipJoin(vid2pid) - .flatMap { case (vid, (vdata, pids)) => - pids.iterator.map { pid => MessageToPartition(pid, (vid, vdata)) } - } - .partitionBy(replicationMap.partitioner.get).cache() + val msgsByPartition = vTable.zipJoinFlatMap(vid2pid) { (vid, vdata, pids) => + pids.iterator.map { pid => MessageToPartition(pid, (vid, vdata)) } + }.partitionBy(replicationMap.partitioner.get).cache() replicationMap.zipPartitions(msgsByPartition){ (mapIter, msgsIter) => diff --git a/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala b/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala index 8d0b2e0b02b75..0fb101a08c55a 100644 --- a/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala @@ -4,6 +4,7 @@ import org.scalatest.FunSuite import org.apache.spark.SparkContext import org.apache.spark.SparkContext._ +import org.apache.spark.rdd._ import org.apache.spark.graph.LocalSparkContext._ @@ -58,8 +59,9 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { val prGraph1 = Analytics.pagerank(starGraph, 1, resetProb) val prGraph2 = Analytics.pagerank(starGraph, 2, resetProb) - val notMatching = prGraph1.vertices.zipJoin(prGraph2.vertices) - .map{ case (vid, (pr1, pr2)) => if (pr1 != pr2) { 1 } else { 0 } }.sum + val notMatching = prGraph1.vertices.zipJoin(prGraph2.vertices) { (vid, pr1, pr2) => + if (pr1 != pr2) { 1 } else { 0 } + }.map { case (vid, test) => test }.sum assert(notMatching === 0) prGraph2.vertices.foreach(println(_)) val errors = prGraph2.vertices.map{ case (vid, pr) => @@ -70,10 +72,12 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { assert(errors.sum === 0) val prGraph3 = Analytics.deltaPagerank(starGraph, 0, resetProb) - val errors2 = prGraph2.vertices.leftJoin(prGraph3.vertices).map{ - case (_, (pr1, Some(pr2))) if(pr1 == pr2) => 0 - case _ => 1 - }.sum + val errors2 = prGraph2.vertices.leftJoin(prGraph3.vertices){ (vid, pr1, pr2Opt) => + pr2Opt match { + case Some(pr2) if(pr1 == pr2) => 0 + case _ => 1 + } + }.map { case (vid, test) => test }.sum assert(errors2 === 0) } } // end of test Star PageRank @@ -86,19 +90,17 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { val resetProb = 0.15 val prGraph1 = Analytics.pagerank(gridGraph, 50, resetProb).cache() val prGraph2 = Analytics.deltaPagerank(gridGraph, 0.0001, resetProb).cache() - val error = prGraph1.vertices.zipJoin(prGraph2.vertices).map { - case (id, (a, b)) => (a - b) * (a - b) - }.sum - prGraph1.vertices.zipJoin(prGraph2.vertices) - .map{ case (id, (a,b)) => (id, (a,b, a-b))}.foreach(println(_)) + val error = prGraph1.vertices.zipJoin(prGraph2.vertices) { case (id, a, b) => (a - b) * (a - b) } + .map { case (id, error) => error }.sum + prGraph1.vertices.zipJoin(prGraph2.vertices) { (id, a, b) => (a, b, a-b) }.foreach(println(_)) println(error) assert(error < 1.0e-5) - val pr3 = sc.parallelize(GridPageRank(10,10, 50, resetProb)) - val error2 = prGraph1.vertices.leftJoin(pr3).map { - case (id, (a, Some(b))) => (a - b) * (a - b) - case _ => 0 - }.sum - prGraph1.vertices.leftJoin(pr3).foreach(println( _ )) + val pr3: RDD[(Vid, Double)] = sc.parallelize(GridPageRank(10,10, 50, resetProb)) + val error2 = prGraph1.vertices.leftJoin(pr3) { (id, a, bOpt) => + val b: Double = bOpt.get + (a - b) * (a - b) + }.map { case (id, error) => error }.sum + prGraph1.vertices.leftJoin(pr3) { (id, a, b) => (a, b) }.foreach( println(_) ) println(error2) assert(error2 < 1.0e-5) } diff --git a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala index 2067b1613ef8d..ec548bda16069 100644 --- a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala @@ -78,13 +78,13 @@ class GraphSuite extends FunSuite with LocalSparkContext { val a = sc.parallelize((0 to 100).map(x => (x.toLong, x.toLong)), 5) val b = VertexSetRDD(a).mapValues(x => -x) assert(b.count === 101) - assert(b.leftJoin(a).mapValues(x => x._1 + x._2.get).map(x=> x._2).reduce(_+_) === 0) + assert(b.leftJoin(a){ (id, a, bOpt) => a + bOpt.get }.map(x=> x._2).reduce(_+_) === 0) val c = VertexSetRDD(a, b.index) - assert(b.leftJoin(c).mapValues(x => x._1 + x._2.get).map(x=> x._2).reduce(_+_) === 0) + assert(b.leftJoin(c){ (id, b, cOpt) => b + cOpt.get }.map(x=> x._2).reduce(_+_) === 0) val d = c.filter(q => ((q._2 % 2) == 0)) val e = a.filter(q => ((q._2 % 2) == 0)) assert(d.count === e.count) - assert(b.zipJoin(c).mapValues(x => x._1 + x._2).map(x => x._2).reduce(_+_) === 0) + assert(b.zipJoin(c)((id, b, c) => b + c).map(x => x._2).reduce(_+_) === 0) } } From 95f1f5315ec4366f3122aa36047608f9e33ee030 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 7 Nov 2013 16:22:05 -0800 Subject: [PATCH 178/531] Added GraphX to classpath. --- bin/compute-classpath.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index 6f1a3f8d64bbc..af22d85afc0a6 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -40,6 +40,7 @@ if [ -f "$FWDIR"/assembly/target/scala-$SCALA_VERSION/spark-assembly*hadoop*-dep CLASSPATH="$CLASSPATH:$FWDIR/repl/target/scala-$SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$FWDIR/mllib/target/scala-$SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$FWDIR/bagel/target/scala-$SCALA_VERSION/classes" + CLASSPATH="$CLASSPATH:$FWDIR/graph/target/scala-$SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$FWDIR/streaming/target/scala-$SCALA_VERSION/classes" DEPS_ASSEMBLY_JAR=`ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/spark-assembly*hadoop*-deps.jar` From 6fadff2b92e5f47750a9438035556e8ae9ce49bd Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 7 Nov 2013 16:54:33 -0800 Subject: [PATCH 179/531] Converted for loops to while loops in EdgePartition. --- .../scala/org/apache/spark/graph/Graph.scala | 3 -- .../spark/graph/impl/EdgePartition.scala | 40 +++++++++---------- 2 files changed, 18 insertions(+), 25 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/Graph.scala b/graph/src/main/scala/org/apache/spark/graph/Graph.scala index 89e1b4ea01a81..acfdc4378b0a0 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Graph.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Graph.scala @@ -1,9 +1,6 @@ package org.apache.spark.graph - import org.apache.spark.rdd.RDD -import org.apache.spark.util.ClosureCleaner - /** diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala b/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala index dbfccde8b91d5..750075533af4e 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala @@ -1,6 +1,5 @@ package org.apache.spark.graph.impl -import scala.collection.mutable.ArrayBuilder import org.apache.spark.graph._ @@ -8,47 +7,46 @@ import org.apache.spark.graph._ * A partition of edges in 3 large columnar arrays. */ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED: ClassManifest]( - val srcIds: Array[Vid], - val dstIds: Array[Vid], - val data: Array[ED] - ){ - - // private var _data: Array[ED] = _ - // private var _dataBuilder = ArrayBuilder.make[ED] - - // var srcIds = new VertexArrayList - // var dstIds = new VertexArrayList + val srcIds: Array[Vid], + val dstIds: Array[Vid], + val data: Array[ED]) +{ def reverse: EdgePartition[ED] = new EdgePartition(dstIds, srcIds, data) def map[ED2: ClassManifest](f: Edge[ED] => ED2): EdgePartition[ED2] = { val newData = new Array[ED2](data.size) val edge = new Edge[ED]() - for(i <- 0 until data.size){ + val size = data.size + var i = 0 + while (i < size) { edge.srcId = srcIds(i) edge.dstId = dstIds(i) edge.attr = data(i) - newData(i) = f(edge) + newData(i) = f(edge) + i += 1 } new EdgePartition(srcIds, dstIds, newData) } def foreach(f: Edge[ED] => Unit) { val edge = new Edge[ED] - for(i <- 0 until data.size){ - edge.srcId = srcIds(i) - edge.dstId = dstIds(i) + val size = data.size + var i = 0 + while (i < size) { + edge.srcId = srcIds(i) + edge.dstId = dstIds(i) edge.attr = data(i) - f(edge) + f(edge) + i += 1 } } - def size: Int = srcIds.size def iterator = new Iterator[Edge[ED]] { - private val edge = new Edge[ED] - private var pos = 0 + private[this] val edge = new Edge[ED] + private[this] var pos = 0 override def hasNext: Boolean = pos < EdgePartition.this.size @@ -61,5 +59,3 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) } } } - - From e9308e0e75b20cae7beda6c88a7b99445a748e6d Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Thu, 7 Nov 2013 18:04:06 -0800 Subject: [PATCH 180/531] Changing Pregel API to operate directly on edge triplets in SendMessage rather than (Vid, EdgeTriplet) pairs. --- .../org/apache/spark/graph/Analytics.scala | 62 ++++---- .../org/apache/spark/graph/GraphOps.scala | 36 +++-- .../scala/org/apache/spark/graph/Pregel.scala | 150 ++---------------- .../apache/spark/graph/impl/GraphImpl.scala | 138 ++++++++-------- 4 files changed, 137 insertions(+), 249 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala index a67cc44f6ec8d..6beaea07fa060 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala @@ -18,7 +18,7 @@ object Analytics extends Logging { * Run PageRank for a fixed number of iterations returning a graph * with vertex attributes containing the PageRank and edge * attributes the normalized edge weight. - * + * * The following PageRank fixed point is computed for each vertex. * * {{{ @@ -35,7 +35,7 @@ object Analytics extends Logging { * 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. @@ -52,7 +52,7 @@ object Analytics extends Logging { * */ def pagerank[VD: Manifest, ED: Manifest]( - graph: Graph[VD, ED], numIter: Int, resetProb: Double = 0.15): + graph: Graph[VD, ED], numIter: Int, resetProb: Double = 0.15): Graph[Double, Double] = { /** @@ -76,13 +76,13 @@ object Analytics extends Logging { // version of Pregel def vertexProgram(id: Vid, attr: Double, msgSum: Double): Double = resetProb + (1.0 - resetProb) * msgSum - def sendMessage(id: Vid, edge: EdgeTriplet[Double, Double]): Option[Double] = - Some(edge.srcAttr * edge.attr) + def sendMessage(edge: EdgeTriplet[Double, Double]) = + Array((edge.dstId, edge.srcAttr * edge.attr)) def messageCombiner(a: Double, b: Double): Double = a + b // The initial message received by all vertices in PageRank - val initialMessage = 0.0 + val initialMessage = 0.0 - // Execute pregel for a fixed number of iterations. + // Execute pregel for a fixed number of iterations. Pregel(pagerankGraph, initialMessage, numIter)( vertexProgram, sendMessage, messageCombiner) } @@ -107,7 +107,7 @@ object Analytics extends Logging { * 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. @@ -124,11 +124,11 @@ object Analytics extends Logging { * PageRank and each edge containing the normalized weight. */ def deltaPagerank[VD: Manifest, ED: Manifest]( - graph: Graph[VD, ED], tol: Double, resetProb: Double = 0.15): + graph: Graph[VD, ED], tol: Double, resetProb: Double = 0.15): Graph[Double, Double] = { /** - * Initialize the pagerankGraph with each edge attribute + * Initialize the pagerankGraph with each edge attribute * having weight 1/outDegree and each vertex with attribute 1.0. */ val pagerankGraph: Graph[(Double, Double), Double] = graph @@ -136,7 +136,7 @@ object Analytics extends Logging { .outerJoinVertices(graph.outDegrees){ (vid, vdata, deg) => deg.getOrElse(0) } - // Set the weight on the edges based on the degree + // Set the weight on the edges based on the degree .mapTriplets( e => 1.0 / e.srcAttr ) // Set the vertex attributes to (initalPR, delta = 0) .mapVertices( (id, attr) => (0.0, 0.0) ) @@ -151,16 +151,16 @@ object Analytics extends Logging { val newPR = oldPR + (1.0 - resetProb) * msgSum (newPR, newPR - oldPR) } - def sendMessage(id: Vid, edge: EdgeTriplet[(Double, Double), Double]): Option[Double] = { + def sendMessage(edge: EdgeTriplet[(Double, Double), Double]) = { if (edge.srcAttr._2 > tol) { - Some(edge.srcAttr._2 * edge.attr) - } else { None } - } + Array((edge.dstId, edge.srcAttr._2 * edge.attr)) + } else { Array.empty[(Vid, Double)] } + } def messageCombiner(a: Double, b: Double): Double = a + b // The initial message received by all vertices in PageRank val initialMessage = resetProb / (1.0 - resetProb) - // Execute a dynamic version of Pregel. + // Execute a dynamic version of Pregel. Pregel(pagerankGraph, initialMessage)( vertexProgram, sendMessage, messageCombiner) .mapVertices( (vid, attr) => attr._1 ) @@ -182,26 +182,28 @@ object Analytics extends Logging { * @return a graph with vertex attributes containing the smallest * vertex in each connected component */ - def connectedComponents[VD: Manifest, ED: Manifest](graph: Graph[VD, ED]): + def connectedComponents[VD: Manifest, ED: Manifest](graph: Graph[VD, ED]): Graph[Vid, ED] = { val ccGraph = graph.mapVertices { case (vid, _) => vid } - def sendMessage(id: Vid, edge: EdgeTriplet[Vid, ED]): Option[Vid] = { - val thisAttr = edge.vertexAttr(id) - val otherAttr = edge.otherVertexAttr(id) - if(thisAttr < otherAttr) { Some(thisAttr) } - else { None } + def sendMessage(edge: EdgeTriplet[Vid, ED]) = { + if (edge.srcAttr < edge.dstAttr) { + Array((edge.dstId, edge.srcAttr)) + } else if (edge.srcAttr > edge.dstAttr) { + Array((edge.srcId, edge.dstAttr)) + } else { + Array.empty[(Vid, Vid)] + } } - val initialMessage = Long.MaxValue - Pregel(ccGraph, initialMessage, EdgeDirection.Both)( + Pregel(ccGraph, initialMessage)( (id, attr, msg) => math.min(attr, msg), - sendMessage, + sendMessage, (a,b) => math.min(a,b) ) - } // end of connectedComponents + } // end of connectedComponents + - def main(args: Array[String]) = { val host = args(0) @@ -213,7 +215,7 @@ object Analytics extends Logging { case _ => throw new IllegalArgumentException("Invalid argument: " + arg) } } - + def setLogLevels(level: org.apache.log4j.Level, loggers: TraversableOnce[String]) = { loggers.map{ loggerName => @@ -265,7 +267,7 @@ object Analytics extends Logging { val sc = new SparkContext(host, "PageRank(" + fname + ")") - val graph = GraphLoader.textFile(sc, fname, a => 1.0F, + val graph = GraphLoader.textFile(sc, fname, a => 1.0F, minEdgePartitions = numEPart, minVertexPartitions = numVPart).cache() val startTime = System.currentTimeMillis @@ -314,7 +316,7 @@ object Analytics extends Logging { val sc = new SparkContext(host, "ConnectedComponents(" + fname + ")") //val graph = GraphLoader.textFile(sc, fname, a => 1.0F) - val graph = GraphLoader.textFile(sc, fname, a => 1.0F, + val graph = GraphLoader.textFile(sc, fname, a => 1.0F, minEdgePartitions = numEPart, minVertexPartitions = numVPart).cache() val cc = Analytics.connectedComponents(graph) //val cc = if(isDynamic) Analytics.dynamicConnectedComponents(graph, numIter) diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala b/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala index 8c7f4c25e2952..5fd8cd699106e 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala @@ -11,21 +11,21 @@ import org.apache.spark.util.ClosureCleaner * the graph type and is implicitly constructed for each Graph object. * All operations in `GraphOps` are expressed in terms of the * efficient GraphX API. - * + * * @tparam VD the vertex attribute type - * @tparam ED the edge attribute type + * @tparam ED the edge attribute type * */ class GraphOps[VD: ClassManifest, ED: ClassManifest](graph: Graph[VD, ED]) { /** - * Compute the number of edges in the graph. + * Compute the number of edges in the graph. */ lazy val numEdges: Long = graph.edges.count() /** - * Compute the number of vertices in the graph. + * Compute the number of vertices in the graph. */ lazy val numVertices: Long = graph.vertices.count() @@ -39,7 +39,7 @@ class GraphOps[VD: ClassManifest, ED: ClassManifest](graph: Graph[VD, ED]) { /** - * Compute the out-degree of each vertex in the Graph returning an RDD. + * Compute the out-degree of each vertex in the Graph returning an RDD. * @note Vertices with no out edges are not returned in the resulting RDD. */ lazy val outDegrees: VertexSetRDD[Int] = degreesRDD(EdgeDirection.Out) @@ -60,7 +60,13 @@ class GraphOps[VD: ClassManifest, ED: ClassManifest](graph: Graph[VD, ED]) { * neighboring vertex attributes. */ private def degreesRDD(edgeDirection: EdgeDirection): VertexSetRDD[Int] = { - graph.aggregateNeighbors((vid, edge) => Some(1), _+_, edgeDirection) + if (edgeDirection == EdgeDirection.In) { + graph.mapReduceTriplets(et => Array((et.dstId,1)), _+_) + } else if (edgeDirection == EdgeDirection.Out) { + graph.mapReduceTriplets(et => Array((et.srcId,1)), _+_) + } else { // EdgeDirection.both + graph.mapReduceTriplets(et => Array((et.srcId,1), (et.dstId,1)), _+_) + } } @@ -89,7 +95,7 @@ class GraphOps[VD: ClassManifest, ED: ClassManifest](graph: Graph[VD, ED]) { * * @example We can use this function to compute the average follower * age for each user - * + * * {{{ * val graph: Graph[Int,Int] = loadGraph() * val averageFollowerAge: RDD[(Int, Int)] = @@ -113,15 +119,15 @@ class GraphOps[VD: ClassManifest, ED: ClassManifest](graph: Graph[VD, ED]) { ClosureCleaner.clean(mapFunc) ClosureCleaner.clean(reduceFunc) - // Define a new map function over edge triplets + // Define a new map function over edge triplets val mf = (et: EdgeTriplet[VD,ED]) => { // Compute the message to the dst vertex - val dst = + 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 = + val src = if (dir == EdgeDirection.Out || dir == EdgeDirection.Both) { mapFunc(et.srcId, et) } else { Option.empty[A] } @@ -130,7 +136,7 @@ class GraphOps[VD: ClassManifest, ED: ClassManifest](graph: Graph[VD, ED]) { case (None, None) => Array.empty[(Vid, A)] case (Some(srcA),None) => Array((et.srcId, srcA)) case (None, Some(dstA)) => Array((et.dstId, dstA)) - case (Some(srcA), Some(dstA)) => + case (Some(srcA), Some(dstA)) => Array((et.srcId, srcA), (et.dstId, dstA)) } } @@ -141,14 +147,14 @@ class GraphOps[VD: ClassManifest, ED: ClassManifest](graph: Graph[VD, ED]) { /** - * Return the Ids of the neighboring vertices. + * Return the Ids of the neighboring vertices. * * @param edgeDirection the direction along which to collect * neighboring vertices * * @return the vertex set of neighboring ids for each vertex. */ - def collectNeighborIds(edgeDirection: EdgeDirection) : + def collectNeighborIds(edgeDirection: EdgeDirection) : VertexSetRDD[Array[Vid]] = { val nbrs = graph.aggregateNeighbors[Array[Vid]]( (vid, edge) => Some(Array(edge.otherVertexId(vid))), @@ -171,10 +177,10 @@ class GraphOps[VD: ClassManifest, ED: ClassManifest](graph: Graph[VD, ED]) { * @return the vertex set of neighboring vertex attributes for each * vertex. */ - def collectNeighbors(edgeDirection: EdgeDirection) : + def collectNeighbors(edgeDirection: EdgeDirection) : VertexSetRDD[ Array[(Vid, VD)] ] = { val nbrs = graph.aggregateNeighbors[Array[(Vid,VD)]]( - (vid, edge) => + (vid, edge) => Some(Array( (edge.otherVertexId(vid), edge.otherVertexAttr(vid)) )), (a, b) => a ++ b, edgeDirection) diff --git a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala index 1750b7f8dcabc..501e593e917ea 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala @@ -34,8 +34,8 @@ import org.apache.spark.rdd.RDD * def sendMessage(id: Vid, edge: EdgeTriplet[Double, Double]): Option[Double] = * Some(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. + * val initialMessage = 0.0 + * // Execute pregel for a fixed number of iterations. * Pregel(pagerankGraph, initialMessage, numIter)( * vertexProgram, sendMessage, messageCombiner) * }}} @@ -64,7 +64,7 @@ object Pregel { * @tparam ED the edge data type * @tparam A the Pregel message type * - * @param graph the input graph. + * @param graph the input graph. * * @param initialMsg the message each vertex will receive at the on * the first iteration. @@ -93,78 +93,17 @@ object Pregel { def apply[VD: ClassManifest, ED: ClassManifest, A: ClassManifest] (graph: Graph[VD, ED], initialMsg: A, numIter: Int)( vprog: (Vid, VD, A) => VD, - sendMsg: (Vid, EdgeTriplet[VD, ED]) => Option[A], + sendMsg: EdgeTriplet[VD, ED] => Array[(Vid,A)], mergeMsg: (A, A) => A) : Graph[VD, ED] = { - apply(graph, initialMsg, numIter, EdgeDirection.Out)(vprog, sendMsg, mergeMsg) - } // end of Apply - - - /** - * 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 a fixed number (`numIter`) of 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 numIter the number of iterations to run this computation. - * - * @param sendDir the edge direction along which the `sendMsg` - * function is invoked. - * - * @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 each - * edge in the direction `sendDir` adjacent to 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 apply[VD: ClassManifest, ED: ClassManifest, A: ClassManifest] - (graph: Graph[VD, ED], initialMsg: A, numIter: Int, sendDir: EdgeDirection)( - vprog: (Vid, VD, A) => VD, - sendMsg: (Vid, EdgeTriplet[VD, ED]) => Option[A], - mergeMsg: (A, A) => A) - : Graph[VD, ED] = { - - def mapF(vid: Vid, edge: EdgeTriplet[VD,ED]) = sendMsg(edge.otherVertexId(vid), edge) // Receive the first set of messages var g = graph.mapVertices( (vid, vdata) => vprog(vid, vdata, initialMsg)) - + var i = 0 while (i < numIter) { // compute the messages - val messages = g.aggregateNeighbors(mapF, mergeMsg, sendDir.reverse) + val messages = g.mapReduceTriplets(sendMsg, mergeMsg) // receive the messages g = g.joinVertices(messages)(vprog) // count the iteration @@ -195,7 +134,7 @@ object Pregel { * @tparam ED the edge data type * @tparam A the Pregel message type * - * @param graph the input graph. + * @param graph the input graph. * * @param initialMsg the message each vertex will receive at the on * the first iteration. @@ -224,66 +163,7 @@ object Pregel { def apply[VD: ClassManifest, ED: ClassManifest, A: ClassManifest] (graph: Graph[VD, ED], initialMsg: A)( vprog: (Vid, VD, A) => VD, - sendMsg: (Vid, EdgeTriplet[VD, ED]) => Option[A], - mergeMsg: (A, A) => A) - : Graph[VD, ED] = { - apply(graph, initialMsg, EdgeDirection.Out)(vprog, sendMsg, mergeMsg) - } // end of apply - - - /** - * 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. - * - * @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 numIter the number of iterations to run this computation. - * - * @param sendDir the edge direction along which the `sendMsg` - * function is invoked. - * - * @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 each - * edge in the direction `sendDir` adjacent to 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 apply[VD: ClassManifest, ED: ClassManifest, A: ClassManifest] - (graph: Graph[VD, ED], initialMsg: A, sendDir: EdgeDirection)( - vprog: (Vid, VD, A) => VD, - sendMsg: (Vid, EdgeTriplet[VD, ED]) => Option[A], + sendMsg: EdgeTriplet[VD, ED] => Array[(Vid,A)], mergeMsg: (A, A) => A) : Graph[VD, ED] = { @@ -294,7 +174,7 @@ object Pregel { } } - def sendMsgFun(vid: Vid, edge: EdgeTriplet[(VD,Boolean), ED]): Option[A] = { + def sendMsgFun(edge: EdgeTriplet[(VD,Boolean), ED]): Array[(Vid, A)] = { if(edge.srcAttr._2) { val et = new EdgeTriplet[VD, ED] et.srcId = edge.srcId @@ -302,22 +182,22 @@ object Pregel { et.dstId = edge.dstId et.dstAttr = edge.dstAttr._1 et.attr = edge.attr - sendMsg(edge.otherVertexId(vid), et) - } else { None } + sendMsg(et) + } else { Array.empty[(Vid,A)] } } - var g = graph.mapVertices( (vid, vdata) => (vprog(vid, vdata, initialMsg), true) ) + var g = graph.mapVertices( (vid, vdata) => (vprog(vid, vdata, initialMsg), true) ) // compute the messages - var messages = g.aggregateNeighbors(sendMsgFun, mergeMsg, sendDir.reverse).cache + var messages = g.mapReduceTriplets(sendMsgFun, mergeMsg).cache var activeMessages = messages.count - // Loop + // Loop var i = 0 while (activeMessages > 0) { // receive the messages g = g.outerJoinVertices(messages)(vprogFun) val oldMessages = messages // compute the messages - messages = g.aggregateNeighbors(sendMsgFun, mergeMsg, sendDir.reverse).cache + messages = g.mapReduceTriplets(sendMsgFun, mergeMsg).cache activeMessages = messages.count // after counting we can unpersist the old messages oldMessages.unpersist(blocking=false) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index b88c952feb65d..b178e3bb42bb6 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -7,7 +7,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.SparkContext._ -import org.apache.spark.HashPartitioner +import org.apache.spark.HashPartitioner import org.apache.spark.util.ClosureCleaner import org.apache.spark.graph._ @@ -28,7 +28,7 @@ class EdgeTripletIterator[VD: ClassManifest, ED: ClassManifest]( private var pos = 0 private val et = new EdgeTriplet[VD, ED] private val vmap = new PrimitiveKeyOpenHashMap[Vid, VD](vidToIndex, vertexArray) - + override def hasNext: Boolean = pos < edgePartition.size override def next() = { et.srcId = edgePartition.srcIds(pos) @@ -113,16 +113,16 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( override def statistics: Map[String, Any] = { val numVertices = this.numVertices val numEdges = this.numEdges - val replicationRatio = + val replicationRatio = vid2pid.map(kv => kv._2.size).sum / vTable.count - val loadArray = + val loadArray = eTable.map{ case (pid, epart) => epart.data.size }.collect.map(x => x.toDouble / numEdges) val minLoad = loadArray.min val maxLoad = loadArray.max Map( "Num Vertices" -> numVertices, "Num Edges" -> numEdges, - "Replication" -> replicationRatio, "Load Array" -> loadArray, - "Min Load" -> minLoad, "Max Load" -> maxLoad) + "Replication" -> replicationRatio, "Load Array" -> loadArray, + "Min Load" -> minLoad, "Max Load" -> maxLoad) } @@ -145,10 +145,10 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( 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)) + deps.foreach(d => traverseLineage(d.rdd, indent + " | ", visited)) } } - + println("eTable ------------------------------------------") traverseLineage(eTable, " ") var visited = Map(eTable.id -> "eTable") @@ -169,11 +169,11 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( traverseLineage(vid2pid, " ", visited) visited += (vid2pid.id -> "vid2pid") visited += (vid2pid.valuesRDD.id -> "vid2pid.values") - + println("\n\nlocalVidMap -------------------------------------") traverseLineage(localVidMap, " ", visited) visited += (localVidMap.id -> "localVidMap") - + println("\n\nvTableReplicatedValues --------------------------") traverseLineage(vTableReplicatedValues, " ", visited) visited += (vTableReplicatedValues.id -> "vTableReplicatedValues") @@ -185,7 +185,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( override def reverse: Graph[VD, ED] = { - val newEtable = eTable.mapPartitions( _.map{ case (pid, epart) => (pid, epart.reverse) }, + val newEtable = eTable.mapPartitions( _.map{ case (pid, epart) => (pid, epart.reverse) }, preservesPartitioning = true) new GraphImpl(vTable, vid2pid, localVidMap, newEtable) } @@ -207,7 +207,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( GraphImpl.mapTriplets(this, f) - override def subgraph(epred: EdgeTriplet[VD,ED] => Boolean = (x => true), + override def subgraph(epred: EdgeTriplet[VD,ED] => Boolean = (x => true), vpred: (Vid, VD) => Boolean = ((a,b) => true) ): Graph[VD, ED] = { /** @todo The following code behaves deterministically on each @@ -215,7 +215,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( * this version */ // val predGraph = mapVertices(v => (v.data, vpred(v))) - // val newETable = predGraph.triplets.filter(t => + // val newETable = predGraph.triplets.filter(t => // if(v.src.data._2 && v.dst.data._2) { // val src = Vertex(t.src.id, t.src.data._1) // val dst = Vertex(t.dst.id, t.dst.data._1) @@ -226,7 +226,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( // .map(v => (v.id, v.data._1)).indexed() // Reuse the partitioner (but not the index) from this graph - val newVTable = + val newVTable = VertexSetRDD(vertices.filter(v => vpred(v._1, v._2)).partitionBy(vTable.index.partitioner)) @@ -237,9 +237,9 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( ) .map( t => Edge(t.srcId, t.dstId, t.attr) )) - // Construct the Vid2Pid map. Here we assume that the filter operation - // behaves deterministically. - // @todo reindex the vertex and edge tables + // Construct the Vid2Pid map. Here we assume that the filter operation + // behaves deterministically. + // @todo reindex the vertex and edge tables val newVid2Pid = createVid2Pid(newETable, newVTable.index) val newVidMap = createLocalVidMap(newETable) @@ -298,7 +298,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( override def mapReduceTriplets[A: ClassManifest]( mapFunc: EdgeTriplet[VD, ED] => Array[(Vid, A)], reduceFunc: (A, A) => A) - : VertexSetRDD[A] = + : VertexSetRDD[A] = GraphImpl.mapReduceTriplets(this, mapFunc, reduceFunc) @@ -322,30 +322,30 @@ object GraphImpl { def apply[VD: ClassManifest, ED: ClassManifest]( vertices: RDD[(Vid, VD)], edges: RDD[Edge[ED]], - defaultVertexAttr: VD): + defaultVertexAttr: VD): GraphImpl[VD,ED] = { apply(vertices, edges, defaultVertexAttr, (a:VD, b:VD) => a) } def apply[VD: ClassManifest, ED: ClassManifest]( - vertices: RDD[(Vid, VD)], + vertices: RDD[(Vid, VD)], edges: RDD[Edge[ED]], defaultVertexAttr: VD, mergeFunc: (VD, VD) => VD): GraphImpl[VD,ED] = { - val vtable = VertexSetRDD(vertices, mergeFunc) - /** - * @todo Verify that there are no edges that contain vertices + val vtable = VertexSetRDD(vertices, mergeFunc) + /** + * @todo Verify that there are no edges that contain vertices * that are not in vTable. This should probably be resolved: * * edges.flatMap{ e => Array((e.srcId, null), (e.dstId, null)) } * .cogroup(vertices).map{ - * case (vid, _, attr) => + * case (vid, _, attr) => * if (attr.isEmpty) (vid, defaultValue) * else (vid, attr) * } - * + * */ val etable = createETable(edges) val vid2pid = createVid2Pid(etable, vtable.index) @@ -366,7 +366,7 @@ object GraphImpl { : RDD[(Pid, EdgePartition[ED])] = { // Get the number of partitions val numPartitions = edges.partitions.size - val ceilSqrt: Pid = math.ceil(math.sqrt(numPartitions)).toInt + val ceilSqrt: Pid = math.ceil(math.sqrt(numPartitions)).toInt edges.map { e => // Random partitioning based on the source vertex id. // val part: Pid = edgePartitionFunction1D(e.srcId, e.dstId, numPartitions) @@ -399,7 +399,7 @@ object GraphImpl { edgePartition.foreach(e => {vSet.add(e.srcId); vSet.add(e.dstId)}) vSet.iterator.map { vid => (vid.toLong, pid) } } - VertexSetRDD[Pid, ArrayBuffer[Pid]](preAgg, vTableIndex, + VertexSetRDD[Pid, ArrayBuffer[Pid]](preAgg, vTableIndex, (p: Pid) => ArrayBuffer(p), (ab: ArrayBuffer[Pid], p:Pid) => {ab.append(p); ab}, (a: ArrayBuffer[Pid], b: ArrayBuffer[Pid]) => a ++ b) @@ -407,11 +407,11 @@ object GraphImpl { } - protected def createLocalVidMap[ED: ClassManifest](eTable: RDD[(Pid, EdgePartition[ED])]): + protected def createLocalVidMap[ED: ClassManifest](eTable: RDD[(Pid, EdgePartition[ED])]): RDD[(Pid, VertexIdToIndexMap)] = { eTable.mapPartitions( _.map{ case (pid, epart) => val vidToIndex = new VertexIdToIndexMap - epart.foreach{ e => + epart.foreach{ e => vidToIndex.add(e.srcId) vidToIndex.add(e.dstId) } @@ -421,17 +421,17 @@ object GraphImpl { protected def createVTableReplicated[VD: ClassManifest]( - vTable: VertexSetRDD[VD], + vTable: VertexSetRDD[VD], vid2pid: VertexSetRDD[Array[Pid]], - replicationMap: RDD[(Pid, VertexIdToIndexMap)]): + replicationMap: RDD[(Pid, VertexIdToIndexMap)]): RDD[(Pid, Array[VD])] = { - // Join vid2pid and vTable, generate a shuffle dependency on the joined + // Join vid2pid and vTable, generate a shuffle dependency on the joined // result, and get the shuffle id so we can use it on the slave. val msgsByPartition = vTable.zipJoinFlatMap(vid2pid) { (vid, vdata, pids) => pids.iterator.map { pid => MessageToPartition(pid, (vid, vdata)) } }.partitionBy(replicationMap.partitioner.get).cache() - - replicationMap.zipPartitions(msgsByPartition){ + + replicationMap.zipPartitions(msgsByPartition){ (mapIter, msgsIter) => val (pid, vidToIndex) = mapIter.next() assert(!mapIter.hasNext) @@ -448,12 +448,12 @@ object GraphImpl { } - def makeTriplets[VD: ClassManifest, ED: ClassManifest]( + def makeTriplets[VD: ClassManifest, ED: ClassManifest]( localVidMap: RDD[(Pid, VertexIdToIndexMap)], vTableReplicatedValues: RDD[(Pid, Array[VD]) ], eTable: RDD[(Pid, EdgePartition[ED])]): RDD[EdgeTriplet[VD, ED]] = { - localVidMap.zipPartitions(vTableReplicatedValues, eTable) { - (vidMapIter, replicatedValuesIter, eTableIter) => + eTable.zipPartitions(localVidMap, vTableReplicatedValues) { + (eTableIter, vidMapIter, replicatedValuesIter) => val (_, vidToIndex) = vidMapIter.next() val (_, vertexArray) = replicatedValuesIter.next() val (_, edgePartition) = eTableIter.next() @@ -463,9 +463,9 @@ object GraphImpl { def mapTriplets[VD: ClassManifest, ED: ClassManifest, ED2: ClassManifest]( - g: GraphImpl[VD, ED], + g: GraphImpl[VD, ED], f: EdgeTriplet[VD, ED] => ED2): Graph[VD, ED2] = { - val newETable = g.eTable.zipPartitions(g.localVidMap, g.vTableReplicatedValues){ + val newETable = g.eTable.zipPartitions(g.localVidMap, g.vTableReplicatedValues){ (edgePartitionIter, vidToIndexIter, vertexArrayIter) => val (pid, edgePartition) = edgePartitionIter.next() val (_, vidToIndex) = vidToIndexIter.next() @@ -492,8 +492,8 @@ object GraphImpl { ClosureCleaner.clean(mapFunc) ClosureCleaner.clean(reduceFunc) - // Map and preaggregate - val preAgg = g.eTable.zipPartitions(g.localVidMap, g.vTableReplicatedValues){ + // Map and preaggregate + val preAgg = g.eTable.zipPartitions(g.localVidMap, g.vTableReplicatedValues){ (edgePartitionIter, vidToIndexIter, vertexArrayIter) => val (pid, edgePartition) = edgePartitionIter.next() val (_, vidToIndex) = vidToIndexIter.next() @@ -511,7 +511,7 @@ object GraphImpl { val msgBS = new BitSet(vertexArray.size) // Iterate over the partition val et = new EdgeTriplet[VD, ED] - edgePartition.foreach{e => + edgePartition.foreach{e => et.set(e) et.srcAttr = vmap(e.srcId) et.dstAttr = vmap(e.dstId) @@ -523,7 +523,7 @@ object GraphImpl { // Populate the aggregator map if(msgBS.get(ind)) { msgArray(ind) = reduceFunc(msgArray(ind), msg) - } else { + } else { msgArray(ind) = msg msgBS.set(ind) } @@ -538,59 +538,59 @@ object GraphImpl { protected def edgePartitionFunction1D(src: Vid, dst: Vid, numParts: Pid): Pid = { - val mixingPrime: Vid = 1125899906842597L + val mixingPrime: Vid = 1125899906842597L (math.abs(src) * mixingPrime).toInt % numParts } /** - * This function implements a classic 2D-Partitioning of a sparse matrix. - * Suppose we have a graph with 11 vertices that we want to partition + * This function implements a classic 2D-Partitioning of a sparse matrix. + * Suppose we have a graph with 11 vertices that we want to partition * over 9 machines. We can use the following sparse matrix representation: * * __________________________________ - * v0 | P0 * | P1 | P2 * | + * v0 | P0 * | P1 | P2 * | * v1 | **** | * | | * v2 | ******* | ** | **** | - * v3 | ***** | * * | * | + * v3 | ***** | * * | * | * ---------------------------------- - * v4 | P3 * | P4 *** | P5 ** * | + * v4 | P3 * | P4 *** | P5 ** * | * v5 | * * | * | | * v6 | * | ** | **** | - * v7 | * * * | * * | * | + * v7 | * * * | * * | * | * ---------------------------------- - * v8 | P6 * | P7 * | P8 * *| + * v8 | P6 * | P7 * | P8 * *| * v9 | * | * * | | * v10 | * | ** | * * | - * v11 | * <-E | *** | ** | + * v11 | * <-E | *** | ** | * ---------------------------------- * - * The edge denoted by E connects v11 with v1 and is assigned to + * 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(numProc) by sqrt(numProc) blocks. Notice that edges - * adjacent to v11 can only be in the first colum 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 + * adjacent to v11 can only be in the first colum 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(numProc) machines. * - * Notice that P0 has many edges and as a consequence this + * 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 effectively shuffle the vertex locations. + * balance we first multiply each vertex id by a large prime + * to effectively 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. - * + * 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. + * * */ - protected def edgePartitionFunction2D(src: Vid, dst: Vid, + protected def edgePartitionFunction2D(src: Vid, dst: Vid, numParts: Pid, ceilSqrtNumParts: Pid): Pid = { - val mixingPrime: Vid = 1125899906842597L + val mixingPrime: Vid = 1125899906842597L val col: Pid = ((math.abs(src) * mixingPrime) % ceilSqrtNumParts).toInt val row: Pid = ((math.abs(dst) * mixingPrime) % ceilSqrtNumParts).toInt (col * ceilSqrtNumParts + row) % numParts @@ -598,7 +598,7 @@ object GraphImpl { /** - * Assign edges to an aribtrary machine corresponding to a + * Assign edges to an aribtrary machine corresponding to a * random vertex cut. */ protected def randomVertexCut(src: Vid, dst: Vid, numParts: Pid): Pid = { @@ -610,9 +610,9 @@ object GraphImpl { * @todo This will only partition edges to the upper diagonal * of the 2D processor space. */ - protected def canonicalEdgePartitionFunction2D(srcOrig: Vid, dstOrig: Vid, + protected def canonicalEdgePartitionFunction2D(srcOrig: Vid, dstOrig: Vid, numParts: Pid, ceilSqrtNumParts: Pid): Pid = { - val mixingPrime: Vid = 1125899906842597L + val mixingPrime: Vid = 1125899906842597L // Partitions by canonical edge direction val src = math.min(srcOrig, dstOrig) val dst = math.max(srcOrig, dstOrig) From 2406bf33e4381dc172d28311646954b08b614a6c Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 7 Nov 2013 19:18:58 -0800 Subject: [PATCH 181/531] Use custom serializer for aggregation messages when the data type is int/double. --- .../apache/spark/graph/impl/GraphImpl.scala | 61 ++------- .../spark/graph/impl/MessageToPartition.scala | 35 ++++- .../apache/spark/graph/impl/Serializers.scala | 125 ++++++++++++++++++ 3 files changed, 171 insertions(+), 50 deletions(-) create mode 100644 graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index b88c952feb65d..d0df35d4226f7 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -5,7 +5,6 @@ import scala.collection.JavaConversions._ import scala.collection.mutable import scala.collection.mutable.ArrayBuffer - import org.apache.spark.SparkContext._ import org.apache.spark.HashPartitioner import org.apache.spark.util.ClosureCleaner @@ -72,8 +71,6 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( def this() = this(null, null, null, null) - - /** * (localVidMap: VertexSetRDD[Pid, VertexIdToIndexMap]) is a version of the * vertex data after it is replicated. Within each partition, it holds a map @@ -86,22 +83,18 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( @transient val vTableReplicatedValues: RDD[(Pid, Array[VD])] = createVTableReplicated(vTable, vid2pid, localVidMap) - /** Return a RDD of vertices. */ @transient override val vertices = vTable - /** Return a RDD of edges. */ @transient override val edges: RDD[Edge[ED]] = { eTable.mapPartitions( iter => iter.next()._2.iterator , true ) } - /** Return a RDD that brings edges with its source and destination vertices together. */ @transient override val triplets: RDD[EdgeTriplet[VD, ED]] = makeTriplets(localVidMap, vTableReplicatedValues, eTable) - override def cache(): Graph[VD, ED] = { eTable.cache() vid2pid.cache() @@ -109,7 +102,6 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( this } - override def statistics: Map[String, Any] = { val numVertices = this.numVertices val numEdges = this.numEdges @@ -125,7 +117,6 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( "Min Load" -> minLoad, "Max Load" -> maxLoad) } - /** * Display the lineage information for this graph. */ @@ -183,14 +174,12 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( println(visited) } // end of print lineage - override def reverse: Graph[VD, ED] = { val newEtable = eTable.mapPartitions( _.map{ case (pid, epart) => (pid, epart.reverse) }, preservesPartitioning = true) new GraphImpl(vTable, vid2pid, localVidMap, newEtable) } - override def mapVertices[VD2: ClassManifest](f: (Vid, VD) => VD2): Graph[VD2, ED] = { val newVTable = vTable.mapValuesWithKeys((vid, data) => f(vid, data)) new GraphImpl(newVTable, vid2pid, localVidMap, eTable) @@ -202,11 +191,9 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( new GraphImpl(vTable, vid2pid, localVidMap, newETable) } - override def mapTriplets[ED2: ClassManifest](f: EdgeTriplet[VD, ED] => ED2): Graph[VD, ED2] = GraphImpl.mapTriplets(this, f) - override def subgraph(epred: EdgeTriplet[VD,ED] => Boolean = (x => true), vpred: (Vid, VD) => Boolean = ((a,b) => true) ): Graph[VD, ED] = { @@ -246,7 +233,6 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( new GraphImpl(newVTable, newVid2Pid, localVidMap, newETable) } - override def groupEdgeTriplets[ED2: ClassManifest]( f: Iterator[EdgeTriplet[VD,ED]] => ED2 ): Graph[VD,ED2] = { val newEdges: RDD[Edge[ED2]] = triplets.mapPartitions { partIter => @@ -271,7 +257,6 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( new GraphImpl(vTable, vid2pid, localVidMap, newETable) } - override def groupEdges[ED2: ClassManifest](f: Iterator[Edge[ED]] => ED2 ): Graph[VD,ED2] = { @@ -289,8 +274,6 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( new GraphImpl(vTable, vid2pid, localVidMap, newETable) } - - ////////////////////////////////////////////////////////////////////////////////////////////////// // Lower level transformation methods ////////////////////////////////////////////////////////////////////////////////////////////////// @@ -301,7 +284,6 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( : VertexSetRDD[A] = GraphImpl.mapReduceTriplets(this, mapFunc, reduceFunc) - override def outerJoinVertices[U: ClassManifest, VD2: ClassManifest] (updates: RDD[(Vid, U)])(updateF: (Vid, VD, Option[U]) => VD2) : Graph[VD2, ED] = { @@ -309,15 +291,9 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( val newVTable = vTable.leftJoin(updates)(updateF) new GraphImpl(newVTable, vid2pid, localVidMap, eTable) } - - } // end of class GraphImpl - - - - object GraphImpl { def apply[VD: ClassManifest, ED: ClassManifest]( @@ -327,7 +303,6 @@ object GraphImpl { apply(vertices, edges, defaultVertexAttr, (a:VD, b:VD) => a) } - def apply[VD: ClassManifest, ED: ClassManifest]( vertices: RDD[(Vid, VD)], edges: RDD[Edge[ED]], @@ -353,7 +328,6 @@ object GraphImpl { new GraphImpl(vtable, vid2pid, localVidMap, etable) } - /** * Create the edge table RDD, which is much more efficient for Java heap storage than the * normal edges data structure (RDD[(Vid, Vid, ED)]). @@ -389,7 +363,6 @@ object GraphImpl { }, preservesPartitioning = true).cache() } - protected def createVid2Pid[ED: ClassManifest]( eTable: RDD[(Pid, EdgePartition[ED])], vTableIndex: VertexSetIndex): VertexSetRDD[Array[Pid]] = { @@ -406,7 +379,6 @@ object GraphImpl { .mapValues(a => a.toArray).cache() } - protected def createLocalVidMap[ED: ClassManifest](eTable: RDD[(Pid, EdgePartition[ED])]): RDD[(Pid, VertexIdToIndexMap)] = { eTable.mapPartitions( _.map{ case (pid, epart) => @@ -419,7 +391,6 @@ object GraphImpl { }, preservesPartitioning = true).cache() } - protected def createVTableReplicated[VD: ClassManifest]( vTable: VertexSetRDD[VD], vid2pid: VertexSetRDD[Array[Pid]], @@ -428,9 +399,9 @@ object GraphImpl { // Join vid2pid and vTable, generate a shuffle dependency on the joined // result, and get the shuffle id so we can use it on the slave. val msgsByPartition = vTable.zipJoinFlatMap(vid2pid) { (vid, vdata, pids) => - pids.iterator.map { pid => MessageToPartition(pid, (vid, vdata)) } + pids.iterator.map { pid => new VertexMessage[VD](pid, vid, vdata) } }.partitionBy(replicationMap.partitioner.get).cache() - + replicationMap.zipPartitions(msgsByPartition){ (mapIter, msgsIter) => val (pid, vidToIndex) = mapIter.next() @@ -438,8 +409,8 @@ object GraphImpl { // Populate the vertex array using the vidToIndex map val vertexArray = new Array[VD](vidToIndex.capacity) for (msg <- msgsIter) { - val ind = vidToIndex.getPos(msg.data._1) & OpenHashSet.POSITION_MASK - vertexArray(ind) = msg.data._2 + val ind = vidToIndex.getPos(msg.vid) & OpenHashSet.POSITION_MASK + vertexArray(ind) = msg.data } Iterator((pid, vertexArray)) }.cache() @@ -447,7 +418,6 @@ object GraphImpl { // @todo assert edge table has partitioner } - def makeTriplets[VD: ClassManifest, ED: ClassManifest]( localVidMap: RDD[(Pid, VertexIdToIndexMap)], vTableReplicatedValues: RDD[(Pid, Array[VD]) ], @@ -461,7 +431,6 @@ object GraphImpl { } } - def mapTriplets[VD: ClassManifest, ED: ClassManifest, ED2: ClassManifest]( g: GraphImpl[VD, ED], f: EdgeTriplet[VD, ED] => ED2): Graph[VD, ED2] = { @@ -483,7 +452,6 @@ object GraphImpl { new GraphImpl(g.vTable, g.vid2pid, g.localVidMap, newETable) } - def mapReduceTriplets[VD: ClassManifest, ED: ClassManifest, A: ClassManifest]( g: GraphImpl[VD, ED], mapFunc: EdgeTriplet[VD, ED] => Array[(Vid, A)], @@ -495,33 +463,34 @@ object GraphImpl { // Map and preaggregate val preAgg = g.eTable.zipPartitions(g.localVidMap, g.vTableReplicatedValues){ (edgePartitionIter, vidToIndexIter, vertexArrayIter) => - val (pid, edgePartition) = edgePartitionIter.next() + val (_, edgePartition) = edgePartitionIter.next() val (_, vidToIndex) = vidToIndexIter.next() val (_, vertexArray) = vertexArrayIter.next() assert(!edgePartitionIter.hasNext) assert(!vidToIndexIter.hasNext) assert(!vertexArrayIter.hasNext) assert(vidToIndex.capacity == vertexArray.size) + // Reuse the vidToIndex map to run aggregation. val vmap = new PrimitiveKeyOpenHashMap[Vid, VD](vidToIndex, vertexArray) - // We can reuse the vidToIndex map for aggregation here as well. - /** @todo Since this has the downside of not allowing "messages" to arbitrary - * vertices we should consider just using a fresh map. - */ + // TODO(jegonzal): This doesn't allow users to send messages to arbitrary vertices. val msgArray = new Array[A](vertexArray.size) val msgBS = new BitSet(vertexArray.size) // Iterate over the partition val et = new EdgeTriplet[VD, ED] - edgePartition.foreach{e => + edgePartition.foreach { e => et.set(e) et.srcAttr = vmap(e.srcId) et.dstAttr = vmap(e.dstId) + // TODO(rxin): rewrite the foreach using a simple while loop to speed things up. + // Also given we are only allowing zero, one, or two messages, we can completely unroll + // the for loop. mapFunc(et).foreach{ case (vid, msg) => // verify that the vid is valid assert(vid == et.srcId || vid == et.dstId) // Get the index of the key val ind = vidToIndex.getPos(vid) & OpenHashSet.POSITION_MASK // Populate the aggregator map - if(msgBS.get(ind)) { + if (msgBS.get(ind)) { msgArray(ind) = reduceFunc(msgArray(ind), msg) } else { msgArray(ind) = msg @@ -536,14 +505,11 @@ object GraphImpl { VertexSetRDD(preAgg, g.vTable.index, reduceFunc) } - protected def edgePartitionFunction1D(src: Vid, dst: Vid, numParts: Pid): Pid = { val mixingPrime: Vid = 1125899906842597L (math.abs(src) * mixingPrime).toInt % numParts } - - /** * This function implements a classic 2D-Partitioning of a sparse matrix. * Suppose we have a graph with 11 vertices that we want to partition @@ -596,7 +562,6 @@ object GraphImpl { (col * ceilSqrtNumParts + row) % numParts } - /** * Assign edges to an aribtrary machine corresponding to a * random vertex cut. @@ -605,7 +570,6 @@ object GraphImpl { math.abs((src, dst).hashCode()) % numParts } - /** * @todo This will only partition edges to the upper diagonal * of the 2D processor space. @@ -622,4 +586,3 @@ object GraphImpl { } } // end of object GraphImpl - diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/MessageToPartition.scala b/graph/src/main/scala/org/apache/spark/graph/impl/MessageToPartition.scala index b7bbf257a4a56..9ac2c59bf844e 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/MessageToPartition.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/MessageToPartition.scala @@ -1,10 +1,24 @@ package org.apache.spark.graph.impl import org.apache.spark.Partitioner -import org.apache.spark.graph.Pid +import org.apache.spark.graph.{Pid, Vid} import org.apache.spark.rdd.{ShuffledRDD, RDD} +class VertexMessage[@specialized(Int, Long, Double, Boolean/*, AnyRef*/) T]( + @transient var partition: Pid, + var vid: Vid, + var data: T) + extends Product2[Pid, (Vid, T)] { + + override def _1 = partition + + override def _2 = (vid, data) + + override def canEqual(that: Any): Boolean = that.isInstanceOf[VertexMessage[_]] +} + + /** * A message used to send a specific value to a partition. * @param partition index of the target partition. @@ -30,6 +44,21 @@ object MessageToPartition { } +class VertexMessageRDDFunctions[T: ClassManifest](self: RDD[VertexMessage[T]]) { + def partitionBy(partitioner: Partitioner): RDD[VertexMessage[T]] = { + val rdd = new ShuffledRDD[Pid, (Vid, T), VertexMessage[T]](self, partitioner) + + // Set a custom serializer if the data is of int or double type. + if (classManifest[T] == ClassManifest.Int) { + rdd.setSerializer(classOf[IntVertexMessageSerializer].getName) + } else if (classManifest[T] == ClassManifest.Double) { + rdd.setSerializer(classOf[DoubleVertexMessageSerializer].getName) + } + rdd + } +} + + class MessageToPartitionRDDFunctions[T: ClassManifest](self: RDD[MessageToPartition[T]]) { /** @@ -46,4 +75,8 @@ object MessageToPartitionRDDFunctions { implicit def rdd2PartitionRDDFunctions[T: ClassManifest](rdd: RDD[MessageToPartition[T]]) = { new MessageToPartitionRDDFunctions(rdd) } + + implicit def rdd2vertexMessageRDDFunctions[T: ClassManifest](rdd: RDD[VertexMessage[T]]) = { + new VertexMessageRDDFunctions(rdd) + } } diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala b/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala new file mode 100644 index 0000000000000..0092aa7c6b538 --- /dev/null +++ b/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala @@ -0,0 +1,125 @@ +package org.apache.spark.graph.impl + +import java.io.{InputStream, OutputStream} +import java.nio.ByteBuffer + +import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance, Serializer} + + +/** A special shuffle serializer for VertexMessage[Int]. */ +class IntVertexMessageSerializer extends Serializer { + override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { + + override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { + def writeObject[T](t: T) = { + val msg = t.asInstanceOf[VertexMessage[Int]] + writeLong(msg.vid) + writeInt(msg.data) + this + } + } + + override def deserializeStream(s: InputStream) = new ShuffleDeserializationStream(s) { + override def readObject[T](): T = { + new VertexMessage[Int](0, readLong(), readInt()).asInstanceOf[T] + } + } + } +} + + +/** A special shuffle serializer for VertexMessage[Double]. */ +class DoubleVertexMessageSerializer extends Serializer { + override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { + + override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { + def writeObject[T](t: T) = { + val msg = t.asInstanceOf[VertexMessage[Double]] + writeLong(msg.vid) + writeDouble(msg.data) + this + } + } + + override def deserializeStream(s: InputStream) = new ShuffleDeserializationStream(s) { + def readObject[T](): T = { + new VertexMessage[Double](0, readLong(), readDouble()).asInstanceOf[T] + } + } + } +} + +//////////////////////////////////////////////////////////////////////////////// +// Helper classes to shorten the implementation of those special serializers. +//////////////////////////////////////////////////////////////////////////////// + +sealed abstract class ShuffleSerializationStream(s: OutputStream) extends SerializationStream { + // The implementation should override this one. + def writeObject[T](t: T): SerializationStream + + def writeInt(v: Int) { + s.write(v >> 24) + s.write(v >> 16) + s.write(v >> 8) + s.write(v) + } + + def writeLong(v: Long) { + s.write((v >>> 56).toInt) + s.write((v >>> 48).toInt) + s.write((v >>> 40).toInt) + s.write((v >>> 32).toInt) + s.write((v >>> 24).toInt) + s.write((v >>> 16).toInt) + s.write((v >>> 8).toInt) + s.write(v.toInt) + } + + def writeDouble(v: Double) { + writeLong(java.lang.Double.doubleToLongBits(v)) + } + + override def flush(): Unit = s.flush() + + override def close(): Unit = s.close() +} + + +sealed abstract class ShuffleDeserializationStream(s: InputStream) extends DeserializationStream { + // The implementation should override this one. + def readObject[T](): T + + def readInt(): Int = { + (s.read() & 0xFF) << 24 | (s.read() & 0xFF) << 16 | (s.read() & 0xFF) << 8 | (s.read() & 0xFF) + } + + def readLong(): Long = { + (s.read().toLong << 56) | + (s.read() & 0xFF).toLong << 48 | + (s.read() & 0xFF).toLong << 40 | + (s.read() & 0xFF).toLong << 32 | + (s.read() & 0xFF).toLong << 24 | + (s.read() & 0xFF) << 16 | + (s.read() & 0xFF) << 8 | + (s.read() & 0xFF) + } + + def readDouble(): Double = java.lang.Double.longBitsToDouble(readLong()) + + override def close(): Unit = s.close() +} + + +sealed trait ShuffleSerializerInstance extends SerializerInstance { + + override def serialize[T](t: T): ByteBuffer = throw new UnsupportedOperationException + + override def deserialize[T](bytes: ByteBuffer): T = throw new UnsupportedOperationException + + override def deserialize[T](bytes: ByteBuffer, loader: ClassLoader): T = + throw new UnsupportedOperationException + + // The implementation should override the following two. + override def serializeStream(s: OutputStream): SerializationStream + override def deserializeStream(s: InputStream): DeserializationStream +} From bac7be30cd9d58ee4bbc86fa78ba0cc90a84892e Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 7 Nov 2013 19:39:48 -0800 Subject: [PATCH 182/531] Made more specialized messages. --- .../apache/spark/graph/impl/GraphImpl.scala | 13 ++-- .../spark/graph/impl/MessageToPartition.scala | 57 ++++++++++++------ .../apache/spark/graph/impl/Serializers.scala | 60 ++++++++++++++++--- 3 files changed, 101 insertions(+), 29 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index a6c4cc4b66c32..c38780a265707 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -11,7 +11,7 @@ import org.apache.spark.util.ClosureCleaner import org.apache.spark.graph._ import org.apache.spark.graph.impl.GraphImpl._ -import org.apache.spark.graph.impl.MessageToPartitionRDDFunctions._ +import org.apache.spark.graph.impl.MsgRDDFunctions._ import org.apache.spark.rdd.RDD import org.apache.spark.util.collection.{BitSet, OpenHashSet, PrimitiveKeyOpenHashMap} @@ -349,7 +349,7 @@ object GraphImpl { //val part: Pid = canonicalEdgePartitionFunction2D(e.srcId, e.dstId, numPartitions, ceilSqrt) // Should we be using 3-tuple or an optimized class - MessageToPartition(part, (e.srcId, e.dstId, e.attr)) + new MessageToPartition(part, (e.srcId, e.dstId, e.attr)) } .partitionBy(new HashPartitioner(numPartitions)) .mapPartitionsWithIndex( (pid, iter) => { @@ -399,7 +399,10 @@ object GraphImpl { // Join vid2pid and vTable, generate a shuffle dependency on the joined // result, and get the shuffle id so we can use it on the slave. val msgsByPartition = vTable.zipJoinFlatMap(vid2pid) { (vid, vdata, pids) => - pids.iterator.map { pid => new VertexMessage[VD](pid, vid, vdata) } + // TODO(rxin): reuse VertexBroadcastMessage + pids.iterator.map { pid => + new VertexBroadcastMsg[VD](pid, vid, vdata) + } }.partitionBy(replicationMap.partitioner.get).cache() replicationMap.zipPartitions(msgsByPartition){ @@ -500,7 +503,9 @@ object GraphImpl { } } // construct an iterator of tuples Iterator[(Vid, A)] - msgBS.iterator.map( ind => (vidToIndex.getValue(ind), msgArray(ind)) ) + msgBS.iterator.map { ind => + new AggregationMsg[A](vidToIndex.getValue(ind), msgArray(ind)) + } }.partitionBy(g.vTable.index.rdd.partitioner.get) // do the final reduction reusing the index map VertexSetRDD(preAgg, g.vTable.index, reduceFunc) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/MessageToPartition.scala b/graph/src/main/scala/org/apache/spark/graph/impl/MessageToPartition.scala index 9ac2c59bf844e..3fc0b7c0f7588 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/MessageToPartition.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/MessageToPartition.scala @@ -5,7 +5,7 @@ import org.apache.spark.graph.{Pid, Vid} import org.apache.spark.rdd.{ShuffledRDD, RDD} -class VertexMessage[@specialized(Int, Long, Double, Boolean/*, AnyRef*/) T]( +class VertexBroadcastMsg[@specialized(Int, Long, Double, Boolean) T]( @transient var partition: Pid, var vid: Vid, var data: T) @@ -15,7 +15,18 @@ class VertexMessage[@specialized(Int, Long, Double, Boolean/*, AnyRef*/) T]( override def _2 = (vid, data) - override def canEqual(that: Any): Boolean = that.isInstanceOf[VertexMessage[_]] + override def canEqual(that: Any): Boolean = that.isInstanceOf[VertexBroadcastMsg[_]] +} + + +class AggregationMsg[@specialized(Int, Long, Double, Boolean) T](var vid: Vid, var data: T) + extends Product2[Vid, T] { + + override def _1 = vid + + override def _2 = data + + override def canEqual(that: Any): Boolean = that.isInstanceOf[AggregationMsg[_]] } @@ -36,30 +47,38 @@ class MessageToPartition[@specialized(Int, Long, Double, Char, Boolean/*, AnyRef override def canEqual(that: Any): Boolean = that.isInstanceOf[MessageToPartition[_]] } -/** - * Companion object for MessageToPartition. - */ -object MessageToPartition { - def apply[T](partition: Pid, value: T) = new MessageToPartition(partition, value) + +class VertexBroadcastMsgRDDFunctions[T: ClassManifest](self: RDD[VertexBroadcastMsg[T]]) { + def partitionBy(partitioner: Partitioner): RDD[VertexBroadcastMsg[T]] = { + val rdd = new ShuffledRDD[Pid, (Vid, T), VertexBroadcastMsg[T]](self, partitioner) + + // Set a custom serializer if the data is of int or double type. + if (classManifest[T] == ClassManifest.Int) { + rdd.setSerializer(classOf[IntVertexBroadcastMsgSerializer].getName) + } else if (classManifest[T] == ClassManifest.Double) { + rdd.setSerializer(classOf[DoubleVertexBroadcastMsgSerializer].getName) + } + rdd + } } -class VertexMessageRDDFunctions[T: ClassManifest](self: RDD[VertexMessage[T]]) { - def partitionBy(partitioner: Partitioner): RDD[VertexMessage[T]] = { - val rdd = new ShuffledRDD[Pid, (Vid, T), VertexMessage[T]](self, partitioner) +class AggregationMessageRDDFunctions[T: ClassManifest](self: RDD[AggregationMsg[T]]) { + def partitionBy(partitioner: Partitioner): RDD[AggregationMsg[T]] = { + val rdd = new ShuffledRDD[Vid, T, AggregationMsg[T]](self, partitioner) // Set a custom serializer if the data is of int or double type. if (classManifest[T] == ClassManifest.Int) { - rdd.setSerializer(classOf[IntVertexMessageSerializer].getName) + rdd.setSerializer(classOf[IntAggMsgSerializer].getName) } else if (classManifest[T] == ClassManifest.Double) { - rdd.setSerializer(classOf[DoubleVertexMessageSerializer].getName) + rdd.setSerializer(classOf[DoubleAggMsgSerializer].getName) } rdd } } -class MessageToPartitionRDDFunctions[T: ClassManifest](self: RDD[MessageToPartition[T]]) { +class MsgRDDFunctions[T: ClassManifest](self: RDD[MessageToPartition[T]]) { /** * Return a copy of the RDD partitioned using the specified partitioner. @@ -71,12 +90,16 @@ class MessageToPartitionRDDFunctions[T: ClassManifest](self: RDD[MessageToPartit } -object MessageToPartitionRDDFunctions { +object MsgRDDFunctions { implicit def rdd2PartitionRDDFunctions[T: ClassManifest](rdd: RDD[MessageToPartition[T]]) = { - new MessageToPartitionRDDFunctions(rdd) + new MsgRDDFunctions(rdd) + } + + implicit def rdd2vertexMessageRDDFunctions[T: ClassManifest](rdd: RDD[VertexBroadcastMsg[T]]) = { + new VertexBroadcastMsgRDDFunctions(rdd) } - implicit def rdd2vertexMessageRDDFunctions[T: ClassManifest](rdd: RDD[VertexMessage[T]]) = { - new VertexMessageRDDFunctions(rdd) + implicit def rdd2aggMessageRDDFunctions[T: ClassManifest](rdd: RDD[AggregationMsg[T]]) = { + new AggregationMessageRDDFunctions(rdd) } } diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala b/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala index 0092aa7c6b538..8b4c0868b1a6d 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala @@ -6,13 +6,13 @@ import java.nio.ByteBuffer import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance, Serializer} -/** A special shuffle serializer for VertexMessage[Int]. */ -class IntVertexMessageSerializer extends Serializer { +/** A special shuffle serializer for VertexBroadcastMessage[Int]. */ +class IntVertexBroadcastMsgSerializer extends Serializer { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { def writeObject[T](t: T) = { - val msg = t.asInstanceOf[VertexMessage[Int]] + val msg = t.asInstanceOf[VertexBroadcastMsg[Int]] writeLong(msg.vid) writeInt(msg.data) this @@ -21,20 +21,20 @@ class IntVertexMessageSerializer extends Serializer { override def deserializeStream(s: InputStream) = new ShuffleDeserializationStream(s) { override def readObject[T](): T = { - new VertexMessage[Int](0, readLong(), readInt()).asInstanceOf[T] + new VertexBroadcastMsg[Int](0, readLong(), readInt()).asInstanceOf[T] } } } } -/** A special shuffle serializer for VertexMessage[Double]. */ -class DoubleVertexMessageSerializer extends Serializer { +/** A special shuffle serializer for VertexBroadcastMessage[Double]. */ +class DoubleVertexBroadcastMsgSerializer extends Serializer { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { def writeObject[T](t: T) = { - val msg = t.asInstanceOf[VertexMessage[Double]] + val msg = t.asInstanceOf[VertexBroadcastMsg[Double]] writeLong(msg.vid) writeDouble(msg.data) this @@ -43,7 +43,51 @@ class DoubleVertexMessageSerializer extends Serializer { override def deserializeStream(s: InputStream) = new ShuffleDeserializationStream(s) { def readObject[T](): T = { - new VertexMessage[Double](0, readLong(), readDouble()).asInstanceOf[T] + new VertexBroadcastMsg[Double](0, readLong(), readDouble()).asInstanceOf[T] + } + } + } +} + + +/** A special shuffle serializer for AggregationMessage[Int]. */ +class IntAggMsgSerializer extends Serializer { + override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { + + override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { + def writeObject[T](t: T) = { + val msg = t.asInstanceOf[AggregationMsg[Int]] + writeLong(msg.vid) + writeInt(msg.data) + this + } + } + + override def deserializeStream(s: InputStream) = new ShuffleDeserializationStream(s) { + override def readObject[T](): T = { + new AggregationMsg[Int](readLong(), readInt()).asInstanceOf[T] + } + } + } +} + + +/** A special shuffle serializer for AggregationMessage[Double]. */ +class DoubleAggMsgSerializer extends Serializer { + override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { + + override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { + def writeObject[T](t: T) = { + val msg = t.asInstanceOf[AggregationMsg[Double]] + writeLong(msg.vid) + writeDouble(msg.data) + this + } + } + + override def deserializeStream(s: InputStream) = new ShuffleDeserializationStream(s) { + def readObject[T](): T = { + new AggregationMsg[Double](readLong(), readDouble()).asInstanceOf[T] } } } From 908e606473e4f85cc7c65334666a3d276c722029 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Thu, 7 Nov 2013 19:47:30 -0800 Subject: [PATCH 183/531] Additional optimizations --- .../spark/util/collection/OpenHashSet.scala | 10 ++ .../scala/org/apache/spark/graph/Graph.scala | 41 ++--- .../scala/org/apache/spark/graph/Pregel.scala | 4 +- .../org/apache/spark/graph/VertexSetRDD.scala | 140 +++++++++--------- .../apache/spark/graph/impl/GraphImpl.scala | 14 +- .../org/apache/spark/graph/package.scala | 5 +- 6 files changed, 116 insertions(+), 98 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala b/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala index f8d54a8f738ce..e86d7ef767798 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala @@ -157,6 +157,16 @@ class OpenHashSet[@specialized(Long, Int) T: ClassManifest]( /** Return the value at the specified position. */ def getValue(pos: Int): T = _data(pos) + def iterator() = new Iterator[T] { + var pos = nextPos(0) + override def hasNext: Boolean = pos != INVALID_POS + override def next(): T = { + val tmp = getValue(pos) + pos = nextPos(pos+1) + tmp + } + } + /** Return the value at the specified position. */ def getValueSafe(pos: Int): T = { assert(_bitset.get(pos)) diff --git a/graph/src/main/scala/org/apache/spark/graph/Graph.scala b/graph/src/main/scala/org/apache/spark/graph/Graph.scala index acfdc4378b0a0..f5b4c57f72902 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Graph.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Graph.scala @@ -1,7 +1,7 @@ package org.apache.spark.graph import org.apache.spark.rdd.RDD - +import org.apache.spark.storage.StorageLevel /** * The Graph abstractly represents a graph with arbitrary objects @@ -12,21 +12,21 @@ import org.apache.spark.rdd.RDD * operations return new graphs. * * @see GraphOps for additional graph member functions. - * + * * @note The majority of the graph operations are implemented in * `GraphOps`. All the convenience operations are defined in the * `GraphOps` class which may be shared across multiple graph * implementations. * * @tparam VD the vertex attribute type - * @tparam ED the edge attribute type + * @tparam ED the edge attribute type */ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { /** * Get the vertices and their data. * - * @note vertex ids are unique. + * @note vertex ids are unique. * @return An RDD containing the vertices in this graph * * @see Vertex for the vertex type. @@ -70,6 +70,11 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { */ val triplets: RDD[EdgeTriplet[VD, ED]] + + + def persist(newLevel: StorageLevel): Graph[VD, ED] + + /** * Return a graph that is cached when first created. This is used to * pin a graph in memory enabling multiple queries to reuse the same @@ -100,7 +105,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * @tparam VD2 the new vertex data type * * @example We might use this operation to change the vertex values - * from one type to another to initialize an algorithm. + * from one type to another to initialize an algorithm. * {{{ * val rawGraph: Graph[(), ()] = Graph.textFile("hdfs://file") * val root = 42 @@ -190,7 +195,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * @return the subgraph containing only the vertices and edges that * satisfy the predicates. */ - def subgraph(epred: EdgeTriplet[VD,ED] => Boolean = (x => true), + def subgraph(epred: EdgeTriplet[VD,ED] => Boolean = (x => true), vpred: (Vid, VD) => Boolean = ((v,d) => true) ): Graph[VD, ED] @@ -255,12 +260,12 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * @param reduceFunc the user defined reduce function which should * be commutative and assosciative and is used to combine the output * of the map phase. - * + * * @example We can use this function to compute the inDegree of each * vertex * {{{ * val rawGraph: Graph[(),()] = Graph.textFile("twittergraph") - * val inDeg: RDD[(Vid, Int)] = + * val inDeg: RDD[(Vid, Int)] = * mapReduceTriplets[Int](et => Array((et.dst.id, 1)), _ + _) * }}} * @@ -269,12 +274,12 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * Graph API in that enables neighborhood level computation. For * example this function can be used to count neighbors satisfying a * predicate or implement PageRank. - * + * */ def mapReduceTriplets[A: ClassManifest]( mapFunc: EdgeTriplet[VD, ED] => Array[(Vid, A)], reduceFunc: (A, A) => A) - : VertexSetRDD[A] + : VertexSetRDD[A] /** @@ -296,11 +301,11 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * @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[(),()] = Graph.textFile("webgraph") * val outDeg: RDD[(Vid, Int)] = rawGraph.outDegrees() - * val graph = rawGraph.outerJoinVertices(outDeg) { + * val graph = rawGraph.outerJoinVertices(outDeg) { * (vid, data, optDeg) => optDeg.getOrElse(0) * } * }}} @@ -337,7 +342,7 @@ object Graph { * (i.e., the undirected degree). * * @param rawEdges the RDD containing the set of edges in the graph - * + * * @return a graph with edge attributes containing the count of * duplicate edges and vertex attributes containing the total degree * of each vertex. @@ -368,10 +373,10 @@ object Graph { rawEdges.map { case (s, t) => Edge(s, t, 1) } } // Determine unique vertices - /** @todo Should this reduceByKey operation be indexed? */ - val vertices: RDD[(Vid, Int)] = + /** @todo Should this reduceByKey operation be indexed? */ + val vertices: RDD[(Vid, Int)] = edges.flatMap{ case Edge(s, t, cnt) => Array((s, 1), (t, 1)) }.reduceByKey(_ + _) - + // Return graph GraphImpl(vertices, edges, 0) } @@ -392,7 +397,7 @@ object Graph { * */ def apply[VD: ClassManifest, ED: ClassManifest]( - vertices: RDD[(Vid,VD)], + vertices: RDD[(Vid,VD)], edges: RDD[Edge[ED]]): Graph[VD, ED] = { val defaultAttr: VD = null.asInstanceOf[VD] Graph(vertices, edges, defaultAttr, (a:VD,b:VD) => a) @@ -416,7 +421,7 @@ object Graph { * */ def apply[VD: ClassManifest, ED: ClassManifest]( - vertices: RDD[(Vid,VD)], + vertices: RDD[(Vid,VD)], edges: RDD[Edge[ED]], defaultVertexAttr: VD, mergeFunc: (VD, VD) => VD): Graph[VD, ED] = { diff --git a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala index 501e593e917ea..3b4d3c0df2a51 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala @@ -98,14 +98,14 @@ object Pregel { : Graph[VD, ED] = { // Receive the first set of messages - var g = graph.mapVertices( (vid, vdata) => vprog(vid, vdata, initialMsg)) + var g = graph.mapVertices( (vid, vdata) => vprog(vid, vdata, initialMsg)).cache var i = 0 while (i < numIter) { // compute the messages val messages = g.mapReduceTriplets(sendMsg, mergeMsg) // receive the messages - g = g.joinVertices(messages)(vprog) + g = g.joinVertices(messages)(vprog).cache // count the iteration i += 1 } diff --git a/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala b/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala index 8611d2f0ce1a9..e8b8bb32280f3 100644 --- a/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala @@ -28,7 +28,7 @@ import org.apache.spark.util.collection.{BitSet, OpenHashSet, PrimitiveKeyOpenHa * The `VertexSetIndex` maintains the per-partition mapping from * vertex id to the corresponding location in the per-partition values * array. This class is meant to be an opaque type. - * + * */ class VertexSetIndex(private[spark] val rdd: RDD[VertexIdToIndexMap]) { /** @@ -55,7 +55,7 @@ class VertexSetIndex(private[spark] val rdd: RDD[VertexIdToIndexMap]) { * In addition to providing the basic RDD[(Vid,V)] functionality the * VertexSetRDD exposes an index member which can be used to "key" * other VertexSetRDDs - * + * * @tparam V the vertex attribute associated with each vertex in the * set. * @@ -84,7 +84,7 @@ class VertexSetIndex(private[spark] val rdd: RDD[VertexIdToIndexMap]) { class VertexSetRDD[@specialized V: ClassManifest]( @transient val index: VertexSetIndex, @transient val valuesRDD: RDD[ ( Array[V], BitSet) ]) - extends RDD[(Vid, V)](index.rdd.context, + extends RDD[(Vid, V)](index.rdd.context, List(new OneToOneDependency(index.rdd), new OneToOneDependency(valuesRDD)) ) { @@ -100,32 +100,32 @@ class VertexSetRDD[@specialized V: ClassManifest]( * An internal representation which joins the block indices with the values * This is used by the compute function to emulate RDD[(Vid, V)] */ - protected[spark] val tuples = + protected[spark] val tuples = new ZippedRDD(index.rdd.context, index.rdd, valuesRDD) /** - * The partitioner is defined by the index. + * The partitioner is defined by the index. */ override val partitioner = index.rdd.partitioner - + /** * The actual partitions are defined by the tuples. */ - override def getPartitions: Array[Partition] = tuples.getPartitions - + override def getPartitions: Array[Partition] = tuples.getPartitions + /** - * The preferred locations are computed based on the preferred - * locations of the tuples. + * The preferred locations are computed based on the preferred + * locations of the tuples. */ - override def getPreferredLocations(s: Partition): Seq[String] = + override def getPreferredLocations(s: Partition): Seq[String] = tuples.getPreferredLocations(s) /** - * Caching an VertexSetRDD causes the index and values to be cached separately. + * Caching an VertexSetRDD causes the index and values to be cached separately. */ override def persist(newLevel: StorageLevel): VertexSetRDD[V] = { index.persist(newLevel) @@ -143,7 +143,7 @@ class VertexSetRDD[@specialized V: ClassManifest]( /** - * Provide the RDD[(K,V)] equivalent output. + * Provide the RDD[(K,V)] equivalent output. */ override def compute(part: Partition, context: TaskContext): Iterator[(Vid, V)] = { tuples.compute(part, context).flatMap { case (indexMap, (values, bs) ) => @@ -154,19 +154,19 @@ class VertexSetRDD[@specialized V: ClassManifest]( /** * Restrict the vertex set to the set of vertices satisfying the - * given predicate. - * + * given predicate. + * * @param pred the user defined predicate * * @note The vertex set preserves the original index structure * which means that the returned RDD can be easily joined with - * the original vertex-set. Furthermore, the filter only - * modifies the bitmap index and so no new values are allocated. + * the original vertex-set. Furthermore, the filter only + * modifies the bitmap index and so no new values are allocated. */ override def filter(pred: Tuple2[Vid,V] => Boolean): VertexSetRDD[V] = { val cleanPred = index.rdd.context.clean(pred) - val newValues = index.rdd.zipPartitions(valuesRDD){ - (keysIter: Iterator[VertexIdToIndexMap], + val newValues = index.rdd.zipPartitions(valuesRDD){ + (keysIter: Iterator[VertexIdToIndexMap], valuesIter: Iterator[(Array[V], BitSet)]) => val index = keysIter.next() assert(keysIter.hasNext == false) @@ -174,7 +174,7 @@ class VertexSetRDD[@specialized V: ClassManifest]( assert(valuesIter.hasNext == false) // Allocate the array to store the results into val newBS = new BitSet(index.capacity) - // Iterate over the active bits in the old bitset and + // Iterate over the active bits in the old bitset and // evaluate the predicate var ind = bs.nextSetBit(0) while(ind >= 0) { @@ -193,7 +193,7 @@ class VertexSetRDD[@specialized V: ClassManifest]( /** * Pass each vertex attribute through a map function and retain the * original RDD's partitioning and index. - * + * * @tparam U the type returned by the map function * * @param f the function applied to each value in the RDD @@ -204,12 +204,12 @@ class VertexSetRDD[@specialized V: ClassManifest]( def mapValues[U: ClassManifest](f: V => U): VertexSetRDD[U] = { val cleanF = index.rdd.context.clean(f) val newValuesRDD: RDD[ (Array[U], BitSet) ] = - valuesRDD.mapPartitions(iter => iter.map{ + valuesRDD.mapPartitions(iter => iter.map{ case (values, bs: BitSet) => val newValues = new Array[U](values.size) bs.iterator.foreach { ind => newValues(ind) = cleanF(values(ind)) } (newValues, bs) - }, preservesPartitioning = true) + }, preservesPartitioning = true) new VertexSetRDD[U](index, newValuesRDD) } // end of mapValues @@ -217,7 +217,7 @@ class VertexSetRDD[@specialized V: ClassManifest]( /** * Pass each vertex attribute along with the vertex id through a map * function and retain the original RDD's partitioning and index. - * + * * @tparam U the type returned by the map function * * @param f the function applied to each vertex id and vertex @@ -229,8 +229,8 @@ class VertexSetRDD[@specialized V: ClassManifest]( def mapValuesWithKeys[U: ClassManifest](f: (Vid, V) => U): VertexSetRDD[U] = { val cleanF = index.rdd.context.clean(f) val newValues: RDD[ (Array[U], BitSet) ] = - index.rdd.zipPartitions(valuesRDD){ - (keysIter: Iterator[VertexIdToIndexMap], + index.rdd.zipPartitions(valuesRDD){ + (keysIter: Iterator[VertexIdToIndexMap], valuesIter: Iterator[(Array[V], BitSet)]) => val index = keysIter.next() assert(keysIter.hasNext == false) @@ -254,7 +254,7 @@ class VertexSetRDD[@specialized V: ClassManifest]( * vertices that are in both this and the other vertex set. * * @tparam W the attribute type of the other VertexSet - * + * * @param other the other VertexSet with which to join. * @return a VertexSetRDD containing only the vertices in both this * and the other VertexSet and with tuple attributes. @@ -324,7 +324,7 @@ class VertexSetRDD[@specialized V: ClassManifest]( * any vertex in this VertexSet then a `None` attribute is generated * * @tparam W the attribute type of the other VertexSet - * + * * @param other the other VertexSet with which to join. * @return a VertexSetRDD containing all the vertices in this * VertexSet with `None` attributes used for Vertices missing in the @@ -365,7 +365,7 @@ class VertexSetRDD[@specialized V: ClassManifest]( * VertexSet then a `None` attribute is generated * * @tparam W the attribute type of the other VertexSet - * + * * @param other the other VertexSet with which to join. * @param merge the function used combine duplicate vertex * attributes @@ -398,28 +398,28 @@ class VertexSetRDD[@specialized V: ClassManifest]( /** - * For each key k in `this` or `other`, return a resulting RDD that contains a + * For each key k in `this` or `other`, return a resulting RDD that contains a * tuple with the list of values for that key in `this` as well as `other`. */ /* - def cogroup[W: ClassManifest](other: RDD[(Vid, W)], partitioner: Partitioner): + def cogroup[W: ClassManifest](other: RDD[(Vid, W)], partitioner: Partitioner): VertexSetRDD[(Seq[V], Seq[W])] = { //RDD[(K, (Seq[V], Seq[W]))] = { other match { case other: VertexSetRDD[_] if index == other.index => { - // if both RDDs share exactly the same index and therefore the same - // super set of keys then we simply merge the value RDDs. - // However it is possible that both RDDs are missing a value for a given key in + // if both RDDs share exactly the same index and therefore the same + // super set of keys then we simply merge the value RDDs. + // However it is possible that both RDDs are missing a value for a given key in // which case the returned RDD should have a null value - val newValues: RDD[(IndexedSeq[(Seq[V], Seq[W])], BitSet)] = + val newValues: RDD[(IndexedSeq[(Seq[V], Seq[W])], BitSet)] = valuesRDD.zipPartitions(other.valuesRDD){ - (thisIter, otherIter) => + (thisIter, otherIter) => val (thisValues, thisBS) = thisIter.next() assert(!thisIter.hasNext) val (otherValues, otherBS) = otherIter.next() assert(!otherIter.hasNext) - /** - * @todo consider implementing this with a view as in leftJoin to + /** + * @todo consider implementing this with a view as in leftJoin to * reduce array allocations */ val newValues = new Array[(Seq[V], Seq[W])](thisValues.size) @@ -428,20 +428,20 @@ class VertexSetRDD[@specialized V: ClassManifest]( var ind = newBS.nextSetBit(0) while(ind >= 0) { val a = if (thisBS.get(ind)) Seq(thisValues(ind)) else Seq.empty[V] - val b = if (otherBS.get(ind)) Seq(otherValues(ind)) else Seq.empty[W] + val b = if (otherBS.get(ind)) Seq(otherValues(ind)) else Seq.empty[W] newValues(ind) = (a, b) ind = newBS.nextSetBit(ind+1) } Iterator((newValues.toIndexedSeq, newBS)) } - new VertexSetRDD(index, newValues) + new VertexSetRDD(index, newValues) } - case other: VertexSetRDD[_] + case other: VertexSetRDD[_] if index.rdd.partitioner == other.index.rdd.partitioner => { // If both RDDs are indexed using different indices but with the same partitioners // then we we need to first merge the indicies and then use the merged index to // merge the values. - val newIndex = + val newIndex = index.rdd.zipPartitions(other.index.rdd)( (thisIter, otherIter) => { val thisIndex = thisIter.next() @@ -463,7 +463,7 @@ class VertexSetRDD[@specialized V: ClassManifest]( List(newIndex).iterator }).cache() // Use the new index along with the this and the other indices to merge the values - val newValues: RDD[(IndexedSeq[(Seq[V], Seq[W])], BitSet)] = + val newValues: RDD[(IndexedSeq[(Seq[V], Seq[W])], BitSet)] = newIndex.zipPartitions(tuples, other.tuples)( (newIndexIter, thisTuplesIter, otherTuplesIter) => { // Get the new index for this partition @@ -507,7 +507,7 @@ class VertexSetRDD[@specialized V: ClassManifest]( case None => throw new SparkException("An index must have a partitioner.") } // Shuffle the other RDD using the partitioner for this index - val otherShuffled = + val otherShuffled = if (other.partitioner == Some(partitioner)) { other } else { @@ -527,7 +527,7 @@ class VertexSetRDD[@specialized V: ClassManifest]( // populate the newValues with the values in this VertexSetRDD for ((k,i) <- thisIndex) { if (thisBS.get(i)) { - newValues(i) = (Seq(thisValues(i)), ArrayBuffer.empty[W]) + newValues(i) = (Seq(thisValues(i)), ArrayBuffer.empty[W]) newBS.set(i) } } @@ -538,28 +538,28 @@ class VertexSetRDD[@specialized V: ClassManifest]( if(newBS.get(ind)) { newValues(ind)._2.asInstanceOf[ArrayBuffer[W]].append(w) } else { - // If the other key was in the index but not in the values - // of this indexed RDD then create a new values entry for it + // If the other key was in the index but not in the values + // of this indexed RDD then create a new values entry for it newBS.set(ind) newValues(ind) = (Seq.empty[V], ArrayBuffer(w)) - } + } } else { // update the index val ind = newIndex.size newIndex.put(k, ind) newBS.set(ind) // Update the values - newValues.append( (Seq.empty[V], ArrayBuffer(w) ) ) + newValues.append( (Seq.empty[V], ArrayBuffer(w) ) ) } } Iterator( (newIndex, (newValues.toIndexedSeq, newBS)) ) }).cache() - // Extract the index and values from the above RDD + // Extract the index and values from the above RDD val newIndex = groups.mapPartitions(_.map{ case (kMap,vAr) => kMap }, true) - val newValues: RDD[(IndexedSeq[(Seq[V], Seq[W])], BitSet)] = + val newValues: RDD[(IndexedSeq[(Seq[V], Seq[W])], BitSet)] = groups.mapPartitions(_.map{ case (kMap,vAr) => vAr }, true) - + new VertexSetRDD[(Seq[V], Seq[W])](new VertexSetIndex(newIndex), newValues) } } @@ -583,7 +583,7 @@ object VertexSetRDD { * * @param rdd the collection of vertex-attribute pairs */ - def apply[V: ClassManifest](rdd: RDD[(Vid,V)]): VertexSetRDD[V] = + def apply[V: ClassManifest](rdd: RDD[(Vid,V)]): VertexSetRDD[V] = apply(rdd, (a:V, b:V) => a ) /** @@ -591,7 +591,7 @@ object VertexSetRDD { * where duplicate entries are merged using the reduceFunc * * @tparam V the vertex attribute type - * + * * @param rdd the collection of vertex-attribute pairs * @param reduceFunc the function used to merge attributes of * duplicate vertices. @@ -602,12 +602,12 @@ object VertexSetRDD { // Preaggregate and shuffle if necessary val preAgg = rdd.partitioner match { case Some(p) => rdd - case None => + case None => val partitioner = new HashPartitioner(rdd.partitions.size) // Preaggregation. val aggregator = new Aggregator[Vid, V, V](v => v, cReduceFunc, cReduceFunc) rdd.mapPartitions(aggregator.combineValuesByKey, true).partitionBy(partitioner) - } + } val groups = preAgg.mapPartitions( iter => { val hashMap = new PrimitiveKeyOpenHashMap[Vid, V] @@ -629,8 +629,8 @@ object VertexSetRDD { /** * Construct a vertex set from an RDD using an existing index. - * - * @note duplicate vertices are discarded arbitrarily + * + * @note duplicate vertices are discarded arbitrarily * * @tparam V the vertex attribute type * @param rdd the rdd containing vertices @@ -638,13 +638,13 @@ object VertexSetRDD { * in RDD */ def apply[V: ClassManifest]( - rdd: RDD[(Vid,V)], index: VertexSetIndex): VertexSetRDD[V] = + rdd: RDD[(Vid,V)], index: VertexSetIndex): VertexSetRDD[V] = apply(rdd, index, (a:V,b:V) => a) /** * Construct a vertex set from an RDD using an existing index and a - * user defined `combiner` to merge duplicate vertices. + * user defined `combiner` to merge duplicate vertices. * * @tparam V the vertex attribute type * @param rdd the rdd containing vertices @@ -655,13 +655,13 @@ object VertexSetRDD { */ def apply[V: ClassManifest]( rdd: RDD[(Vid,V)], index: VertexSetIndex, - reduceFunc: (V, V) => V): VertexSetRDD[V] = + reduceFunc: (V, V) => V): VertexSetRDD[V] = apply(rdd,index, (v:V) => v, reduceFunc, reduceFunc) - + /** * Construct a vertex set from an RDD using an existing index and a - * user defined `combiner` to merge duplicate vertices. + * user defined `combiner` to merge duplicate vertices. * * @tparam V the vertex attribute type * @param rdd the rdd containing vertices @@ -675,7 +675,7 @@ object VertexSetRDD { * */ def apply[V: ClassManifest, C: ClassManifest]( - rdd: RDD[(Vid,V)], + rdd: RDD[(Vid,V)], index: VertexSetIndex, createCombiner: V => C, mergeValue: (C, V) => C, @@ -689,7 +689,7 @@ object VertexSetRDD { case None => throw new SparkException("An index must have a partitioner.") } // Preaggregate and shuffle if necessary - val partitioned = + val partitioned = if (rdd.partitioner != Some(partitioner)) { // Preaggregation. val aggregator = new Aggregator[Vid, V, C](cCreateCombiner, cMergeValue, @@ -732,23 +732,23 @@ object VertexSetRDD { /** * Construct an index of the unique vertices. The resulting index - * can be used to build VertexSets over subsets of the vertices in + * can be used to build VertexSets over subsets of the vertices in * the input. */ - def makeIndex(keys: RDD[Vid], + def makeIndex(keys: RDD[Vid], partitioner: Option[Partitioner] = None): VertexSetIndex = { // @todo: I don't need the boolean its only there to be the second type since I want to shuffle a single RDD - // Ugly hack :-(. In order to partition the keys they must have values. + // Ugly hack :-(. In order to partition the keys they must have values. val tbl = keys.mapPartitions(_.map(k => (k, false)), true) // Shuffle the table (if necessary) val shuffledTbl = partitioner match { case None => { if (tbl.partitioner.isEmpty) { - // @todo: I don't need the boolean its only there to be the second type of the shuffle. + // @todo: I don't need the boolean its only there to be the second type of the shuffle. new ShuffledRDD[Vid, Boolean, (Vid, Boolean)](tbl, Partitioner.defaultPartitioner(tbl)) } else { tbl } } - case Some(partitioner) => + case Some(partitioner) => tbl.partitionBy(partitioner) } diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index b178e3bb42bb6..9ce06eb9e8b2c 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -14,6 +14,7 @@ import org.apache.spark.graph._ import org.apache.spark.graph.impl.GraphImpl._ import org.apache.spark.graph.impl.MessageToPartitionRDDFunctions._ import org.apache.spark.rdd.RDD +import org.apache.spark.storage.StorageLevel import org.apache.spark.util.collection.{BitSet, OpenHashSet, PrimitiveKeyOpenHashMap} @@ -102,13 +103,16 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( makeTriplets(localVidMap, vTableReplicatedValues, eTable) - override def cache(): Graph[VD, ED] = { - eTable.cache() - vid2pid.cache() - vTable.cache() + override def persist(newLevel: StorageLevel): Graph[VD, ED] = { + eTable.persist(newLevel) + vid2pid.persist(newLevel) + vTable.persist(newLevel) + localVidMap.persist(newLevel) + // vTableReplicatedValues.persist(newLevel) this } + override def cache(): Graph[VD, ED] = persist(StorageLevel.MEMORY_ONLY) override def statistics: Map[String, Any] = { val numVertices = this.numVertices @@ -398,7 +402,7 @@ object GraphImpl { val vSet = new VertexSet edgePartition.foreach(e => {vSet.add(e.srcId); vSet.add(e.dstId)}) vSet.iterator.map { vid => (vid.toLong, pid) } - } + }.partitionBy(vTableIndex.rdd.partitioner.get) VertexSetRDD[Pid, ArrayBuffer[Pid]](preAgg, vTableIndex, (p: Pid) => ArrayBuffer(p), (ab: ArrayBuffer[Pid], p:Pid) => {ab.append(p); ab}, diff --git a/graph/src/main/scala/org/apache/spark/graph/package.scala b/graph/src/main/scala/org/apache/spark/graph/package.scala index ee28d1429e017..7b53e9cce82a3 100644 --- a/graph/src/main/scala/org/apache/spark/graph/package.scala +++ b/graph/src/main/scala/org/apache/spark/graph/package.scala @@ -8,10 +8,9 @@ package object graph { type Vid = Long type Pid = Int - type VertexHashMap[T] = it.unimi.dsi.fastutil.longs.Long2ObjectOpenHashMap[T] - type VertexSet = it.unimi.dsi.fastutil.longs.LongOpenHashSet + type VertexSet = OpenHashSet[Vid] type VertexArrayList = it.unimi.dsi.fastutil.longs.LongArrayList - + // type VertexIdToIndexMap = it.unimi.dsi.fastutil.longs.Long2IntOpenHashMap type VertexIdToIndexMap = OpenHashSet[Vid] From 161784d0e63336bfb12b0197fa1e376c60401c2e Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Thu, 7 Nov 2013 20:40:21 -0800 Subject: [PATCH 184/531] Fixing tests --- graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala b/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala index 3bedf89c42ee1..62608e506d85b 100644 --- a/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala @@ -23,6 +23,7 @@ import org.apache.spark.rdd._ import org.apache.spark.storage.StorageLevel import org.apache.spark.util.collection.{BitSet, OpenHashSet, PrimitiveKeyOpenHashMap} import org.apache.spark.graph.impl.AggregationMsg +import org.apache.spark.graph.impl.MsgRDDFunctions._ /** * The `VertexSetIndex` maintains the per-partition mapping from From 6083e4350f13123b258d4c3b73421e219c58c82b Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Fri, 8 Nov 2013 15:39:30 -0800 Subject: [PATCH 185/531] Adding unit tests to reproduce error. --- .../spark/graph/GraphKryoRegistrator.scala | 4 +- .../spark/graph/impl/MessageToPartition.scala | 4 + .../apache/spark/graph/impl/Serializers.scala | 57 ++++++- .../apache/spark/graph/SerializerSuite.scala | 139 ++++++++++++++++++ 4 files changed, 200 insertions(+), 4 deletions(-) create mode 100644 graph/src/test/scala/org/apache/spark/graph/SerializerSuite.scala diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphKryoRegistrator.scala b/graph/src/main/scala/org/apache/spark/graph/GraphKryoRegistrator.scala index f65f96ed0c1b2..82b9198e432c7 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphKryoRegistrator.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphKryoRegistrator.scala @@ -2,7 +2,7 @@ package org.apache.spark.graph import com.esotericsoftware.kryo.Kryo -import org.apache.spark.graph.impl.{EdgePartition, MessageToPartition} +import org.apache.spark.graph.impl._ import org.apache.spark.serializer.KryoRegistrator import org.apache.spark.util.collection.BitSet @@ -12,6 +12,8 @@ class GraphKryoRegistrator extends KryoRegistrator { kryo.register(classOf[Edge[Object]]) kryo.register(classOf[MutableTuple2[Object, Object]]) kryo.register(classOf[MessageToPartition[Object]]) + kryo.register(classOf[VertexBroadcastMsg[Object]]) + kryo.register(classOf[AggregationMsg[Object]]) kryo.register(classOf[(Vid, Object)]) kryo.register(classOf[EdgePartition[Object]]) kryo.register(classOf[BitSet]) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/MessageToPartition.scala b/graph/src/main/scala/org/apache/spark/graph/impl/MessageToPartition.scala index 3fc0b7c0f7588..d0a5adb85cd8a 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/MessageToPartition.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/MessageToPartition.scala @@ -55,6 +55,8 @@ class VertexBroadcastMsgRDDFunctions[T: ClassManifest](self: RDD[VertexBroadcast // Set a custom serializer if the data is of int or double type. if (classManifest[T] == ClassManifest.Int) { rdd.setSerializer(classOf[IntVertexBroadcastMsgSerializer].getName) + } else if (classManifest[T] == ClassManifest.Long) { + rdd.setSerializer(classOf[LongVertexBroadcastMsgSerializer].getName) } else if (classManifest[T] == ClassManifest.Double) { rdd.setSerializer(classOf[DoubleVertexBroadcastMsgSerializer].getName) } @@ -70,6 +72,8 @@ class AggregationMessageRDDFunctions[T: ClassManifest](self: RDD[AggregationMsg[ // Set a custom serializer if the data is of int or double type. if (classManifest[T] == ClassManifest.Int) { rdd.setSerializer(classOf[IntAggMsgSerializer].getName) + } else if (classManifest[T] == ClassManifest.Long) { + rdd.setSerializer(classOf[LongAggMsgSerializer].getName) } else if (classManifest[T] == ClassManifest.Double) { rdd.setSerializer(classOf[DoubleAggMsgSerializer].getName) } diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala b/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala index 8b4c0868b1a6d..54fd65e7381f2 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala @@ -27,6 +27,28 @@ class IntVertexBroadcastMsgSerializer extends Serializer { } } +/** A special shuffle serializer for VertexBroadcastMessage[Long]. */ +class LongVertexBroadcastMsgSerializer extends Serializer { + override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { + + override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { + def writeObject[T](t: T) = { + val msg = t.asInstanceOf[VertexBroadcastMsg[Long]] + writeLong(msg.vid) + writeLong(msg.data) + this + } + } + + override def deserializeStream(s: InputStream) = new ShuffleDeserializationStream(s) { + override def readObject[T](): T = { + val a = readLong() + val b = readLong() + new VertexBroadcastMsg[Long](0, a, b).asInstanceOf[T] + } + } + } +} /** A special shuffle serializer for VertexBroadcastMessage[Double]. */ class DoubleVertexBroadcastMsgSerializer extends Serializer { @@ -43,7 +65,9 @@ class DoubleVertexBroadcastMsgSerializer extends Serializer { override def deserializeStream(s: InputStream) = new ShuffleDeserializationStream(s) { def readObject[T](): T = { - new VertexBroadcastMsg[Double](0, readLong(), readDouble()).asInstanceOf[T] + val a = readLong() + val b = readDouble() + new VertexBroadcastMsg[Double](0, a, b).asInstanceOf[T] } } } @@ -65,7 +89,32 @@ class IntAggMsgSerializer extends Serializer { override def deserializeStream(s: InputStream) = new ShuffleDeserializationStream(s) { override def readObject[T](): T = { - new AggregationMsg[Int](readLong(), readInt()).asInstanceOf[T] + val a = readLong() + val b = readInt() + new AggregationMsg[Int](a, b).asInstanceOf[T] + } + } + } +} + +/** A special shuffle serializer for AggregationMessage[Long]. */ +class LongAggMsgSerializer extends Serializer { + override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { + + override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { + def writeObject[T](t: T) = { + val msg = t.asInstanceOf[AggregationMsg[Long]] + writeLong(msg.vid) + writeLong(msg.data) + this + } + } + + override def deserializeStream(s: InputStream) = new ShuffleDeserializationStream(s) { + override def readObject[T](): T = { + val a = readLong() + val b = readLong() + new AggregationMsg[Long](a, b).asInstanceOf[T] } } } @@ -87,7 +136,9 @@ class DoubleAggMsgSerializer extends Serializer { override def deserializeStream(s: InputStream) = new ShuffleDeserializationStream(s) { def readObject[T](): T = { - new AggregationMsg[Double](readLong(), readDouble()).asInstanceOf[T] + val a = readLong() + val b = readDouble() + new AggregationMsg[Double](a, b).asInstanceOf[T] } } } diff --git a/graph/src/test/scala/org/apache/spark/graph/SerializerSuite.scala b/graph/src/test/scala/org/apache/spark/graph/SerializerSuite.scala new file mode 100644 index 0000000000000..5a59fd912a519 --- /dev/null +++ b/graph/src/test/scala/org/apache/spark/graph/SerializerSuite.scala @@ -0,0 +1,139 @@ +package org.apache.spark.graph + +import org.scalatest.FunSuite + +import org.apache.spark.SparkContext +import org.apache.spark.graph.LocalSparkContext._ +import java.io.ByteArrayInputStream +import java.io.ByteArrayOutputStream +import org.apache.spark.graph.impl._ +import org.apache.spark.graph.impl.MsgRDDFunctions._ +import org.apache.spark._ + + +class SerializerSuite extends FunSuite with LocalSparkContext { + + System.setProperty("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + System.setProperty("spark.kryo.registrator", "org.apache.spark.graph.GraphKryoRegistrator") + + test("TestVertexBroadcastMessageInt") { + val outMsg = new VertexBroadcastMsg[Int](3,4,5) + val bout = new ByteArrayOutputStream + val outStrm = new IntVertexBroadcastMsgSerializer().newInstance().serializeStream(bout) + outStrm.writeObject(outMsg) + outStrm.writeObject(outMsg) + bout.flush + val bin = new ByteArrayInputStream(bout.toByteArray) + val inStrm = new IntVertexBroadcastMsgSerializer().newInstance().deserializeStream(bin) + val inMsg1: VertexBroadcastMsg[Int] = inStrm.readObject() + val inMsg2: VertexBroadcastMsg[Int] = inStrm.readObject() + assert(outMsg.vid === inMsg1.vid) + assert(outMsg.vid === inMsg2.vid) + assert(outMsg.data === inMsg1.data) + assert(outMsg.data === inMsg2.data) + } + + test("TestVertexBroadcastMessageLong") { + val outMsg = new VertexBroadcastMsg[Long](3,4,5) + val bout = new ByteArrayOutputStream + val outStrm = new LongVertexBroadcastMsgSerializer().newInstance().serializeStream(bout) + outStrm.writeObject(outMsg) + outStrm.writeObject(outMsg) + bout.flush + val bin = new ByteArrayInputStream(bout.toByteArray) + val inStrm = new LongVertexBroadcastMsgSerializer().newInstance().deserializeStream(bin) + val inMsg1: VertexBroadcastMsg[Long] = inStrm.readObject() + val inMsg2: VertexBroadcastMsg[Long] = inStrm.readObject() + assert(outMsg.vid === inMsg1.vid) + assert(outMsg.vid === inMsg2.vid) + assert(outMsg.data === inMsg1.data) + assert(outMsg.data === inMsg2.data) + } + + test("TestVertexBroadcastMessageDouble") { + val outMsg = new VertexBroadcastMsg[Double](3,4,5.0) + val bout = new ByteArrayOutputStream + val outStrm = new DoubleVertexBroadcastMsgSerializer().newInstance().serializeStream(bout) + outStrm.writeObject(outMsg) + outStrm.writeObject(outMsg) + bout.flush + val bin = new ByteArrayInputStream(bout.toByteArray) + val inStrm = new DoubleVertexBroadcastMsgSerializer().newInstance().deserializeStream(bin) + val inMsg1: VertexBroadcastMsg[Double] = inStrm.readObject() + val inMsg2: VertexBroadcastMsg[Double] = inStrm.readObject() + assert(outMsg.vid === inMsg1.vid) + assert(outMsg.vid === inMsg2.vid) + assert(outMsg.data === inMsg1.data) + assert(outMsg.data === inMsg2.data) + } + + test("TestAggregationMessageInt") { + val outMsg = new AggregationMsg[Int](4,5) + val bout = new ByteArrayOutputStream + val outStrm = new IntAggMsgSerializer().newInstance().serializeStream(bout) + outStrm.writeObject(outMsg) + outStrm.writeObject(outMsg) + bout.flush + val bin = new ByteArrayInputStream(bout.toByteArray) + val inStrm = new IntAggMsgSerializer().newInstance().deserializeStream(bin) + val inMsg1: AggregationMsg[Int] = inStrm.readObject() + val inMsg2: AggregationMsg[Int] = inStrm.readObject() + assert(outMsg.vid === inMsg1.vid) + assert(outMsg.vid === inMsg2.vid) + assert(outMsg.data === inMsg1.data) + assert(outMsg.data === inMsg2.data) + } + + test("TestAggregationMessageLong") { + val outMsg = new AggregationMsg[Long](4,5) + val bout = new ByteArrayOutputStream + val outStrm = new LongAggMsgSerializer().newInstance().serializeStream(bout) + outStrm.writeObject(outMsg) + outStrm.writeObject(outMsg) + bout.flush + val bin = new ByteArrayInputStream(bout.toByteArray) + val inStrm = new LongAggMsgSerializer().newInstance().deserializeStream(bin) + val inMsg1: AggregationMsg[Long] = inStrm.readObject() + val inMsg2: AggregationMsg[Long] = inStrm.readObject() + assert(outMsg.vid === inMsg1.vid) + assert(outMsg.vid === inMsg2.vid) + assert(outMsg.data === inMsg1.data) + assert(outMsg.data === inMsg2.data) + } + + test("TestAggregationMessageDouble") { + val outMsg = new AggregationMsg[Double](4,5.0) + val bout = new ByteArrayOutputStream + val outStrm = new DoubleAggMsgSerializer().newInstance().serializeStream(bout) + outStrm.writeObject(outMsg) + outStrm.writeObject(outMsg) + bout.flush + val bin = new ByteArrayInputStream(bout.toByteArray) + val inStrm = new DoubleAggMsgSerializer().newInstance().deserializeStream(bin) + val inMsg1: AggregationMsg[Double] = inStrm.readObject() + val inMsg2: AggregationMsg[Double] = inStrm.readObject() + assert(outMsg.vid === inMsg1.vid) + assert(outMsg.vid === inMsg2.vid) + assert(outMsg.data === inMsg1.data) + assert(outMsg.data === inMsg2.data) + } + + test("TestShuffleVertexBroadcastMsg") { + withSpark(new SparkContext("local[2]", "test")) { sc => + val bmsgs = sc.parallelize( + (0 until 100).map(pid => new VertexBroadcastMsg[Int](pid, pid, pid)), 10) + val partitioner = new HashPartitioner(3) + val bmsgsArray = bmsgs.partitionBy(partitioner).collect + } + } + + test("TestShuffleAggregationMsg") { + withSpark(new SparkContext("local[2]", "test")) { sc => + val bmsgs = sc.parallelize( + (0 until 100).map(pid => new AggregationMsg[Int](pid, pid)), 10) + val partitioner = new HashPartitioner(3) + val bmsgsArray = bmsgs.partitionBy(partitioner).collect + } + } + +} \ No newline at end of file From 0e813cd483eb4cc612404f8602e635b29295efc3 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 9 Nov 2013 23:29:37 -0800 Subject: [PATCH 186/531] Fix the hanging bug. --- .../apache/spark/graph/impl/Serializers.scala | 12 ++++-- .../apache/spark/graph/SerializerSuite.scala | 41 ++++++++++++++----- 2 files changed, 39 insertions(+), 14 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala b/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala index 54fd65e7381f2..c56bbc8aee977 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala @@ -1,9 +1,9 @@ package org.apache.spark.graph.impl -import java.io.{InputStream, OutputStream} +import java.io.{EOFException, InputStream, OutputStream} import java.nio.ByteBuffer -import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance, Serializer} +import org.apache.spark.serializer._ /** A special shuffle serializer for VertexBroadcastMessage[Int]. */ @@ -185,11 +185,15 @@ sealed abstract class ShuffleDeserializationStream(s: InputStream) extends Deser def readObject[T](): T def readInt(): Int = { - (s.read() & 0xFF) << 24 | (s.read() & 0xFF) << 16 | (s.read() & 0xFF) << 8 | (s.read() & 0xFF) + val first = s.read() + if (first < 0) throw new EOFException + (first & 0xFF) << 24 | (s.read() & 0xFF) << 16 | (s.read() & 0xFF) << 8 | (s.read() & 0xFF) } def readLong(): Long = { - (s.read().toLong << 56) | + val first = s.read() + if (first < 0) throw new EOFException() + (first.toLong << 56) | (s.read() & 0xFF).toLong << 48 | (s.read() & 0xFF).toLong << 40 | (s.read() & 0xFF).toLong << 32 | diff --git a/graph/src/test/scala/org/apache/spark/graph/SerializerSuite.scala b/graph/src/test/scala/org/apache/spark/graph/SerializerSuite.scala index 5a59fd912a519..0d55cc01896cd 100644 --- a/graph/src/test/scala/org/apache/spark/graph/SerializerSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/SerializerSuite.scala @@ -4,8 +4,7 @@ import org.scalatest.FunSuite import org.apache.spark.SparkContext import org.apache.spark.graph.LocalSparkContext._ -import java.io.ByteArrayInputStream -import java.io.ByteArrayOutputStream +import java.io.{EOFException, ByteArrayInputStream, ByteArrayOutputStream} import org.apache.spark.graph.impl._ import org.apache.spark.graph.impl.MsgRDDFunctions._ import org.apache.spark._ @@ -31,6 +30,10 @@ class SerializerSuite extends FunSuite with LocalSparkContext { assert(outMsg.vid === inMsg2.vid) assert(outMsg.data === inMsg1.data) assert(outMsg.data === inMsg2.data) + + intercept[EOFException] { + inStrm.readObject() + } } test("TestVertexBroadcastMessageLong") { @@ -48,6 +51,10 @@ class SerializerSuite extends FunSuite with LocalSparkContext { assert(outMsg.vid === inMsg2.vid) assert(outMsg.data === inMsg1.data) assert(outMsg.data === inMsg2.data) + + intercept[EOFException] { + inStrm.readObject() + } } test("TestVertexBroadcastMessageDouble") { @@ -65,6 +72,10 @@ class SerializerSuite extends FunSuite with LocalSparkContext { assert(outMsg.vid === inMsg2.vid) assert(outMsg.data === inMsg1.data) assert(outMsg.data === inMsg2.data) + + intercept[EOFException] { + inStrm.readObject() + } } test("TestAggregationMessageInt") { @@ -82,6 +93,10 @@ class SerializerSuite extends FunSuite with LocalSparkContext { assert(outMsg.vid === inMsg2.vid) assert(outMsg.data === inMsg1.data) assert(outMsg.data === inMsg2.data) + + intercept[EOFException] { + inStrm.readObject() + } } test("TestAggregationMessageLong") { @@ -99,6 +114,10 @@ class SerializerSuite extends FunSuite with LocalSparkContext { assert(outMsg.vid === inMsg2.vid) assert(outMsg.data === inMsg1.data) assert(outMsg.data === inMsg2.data) + + intercept[EOFException] { + inStrm.readObject() + } } test("TestAggregationMessageDouble") { @@ -116,23 +135,25 @@ class SerializerSuite extends FunSuite with LocalSparkContext { assert(outMsg.vid === inMsg2.vid) assert(outMsg.data === inMsg1.data) assert(outMsg.data === inMsg2.data) + + intercept[EOFException] { + inStrm.readObject() + } } test("TestShuffleVertexBroadcastMsg") { withSpark(new SparkContext("local[2]", "test")) { sc => - val bmsgs = sc.parallelize( - (0 until 100).map(pid => new VertexBroadcastMsg[Int](pid, pid, pid)), 10) - val partitioner = new HashPartitioner(3) - val bmsgsArray = bmsgs.partitionBy(partitioner).collect + val bmsgs = sc.parallelize(0 until 100, 10).map { pid => + new VertexBroadcastMsg[Int](pid, pid, pid) + } + bmsgs.partitionBy(new HashPartitioner(3)).collect() } } test("TestShuffleAggregationMsg") { withSpark(new SparkContext("local[2]", "test")) { sc => - val bmsgs = sc.parallelize( - (0 until 100).map(pid => new AggregationMsg[Int](pid, pid)), 10) - val partitioner = new HashPartitioner(3) - val bmsgsArray = bmsgs.partitionBy(partitioner).collect + val bmsgs = sc.parallelize(0 until 100, 10).map(pid => new AggregationMsg[Int](pid, pid)) + bmsgs.partitionBy(new HashPartitioner(3)).collect() } } From 80abc2807825d69b0f7a5e374eb6e6442332f400 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Wed, 6 Nov 2013 22:50:30 -0800 Subject: [PATCH 187/531] Optimize mrTriplets for source-attr-only mapF using bytecode inspection --- .../apache/spark/graph/impl/GraphImpl.scala | 41 +++++++++++++++++-- .../org/apache/spark/graph/GraphSuite.scala | 24 ++++++++++- 2 files changed, 60 insertions(+), 5 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index 0d7546b57594c..64fdb10831436 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -12,6 +12,7 @@ import org.apache.spark.util.ClosureCleaner import org.apache.spark.graph._ import org.apache.spark.graph.impl.GraphImpl._ import org.apache.spark.graph.impl.MsgRDDFunctions._ +import org.apache.spark.graph.util.BytecodeUtils import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel import org.apache.spark.util.collection.{BitSet, OpenHashSet, PrimitiveKeyOpenHashMap} @@ -384,6 +385,22 @@ object GraphImpl { .mapValues(a => a.toArray).cache() } + protected def createVid2PidSourceAttrOnly[ED: ClassManifest]( + eTable: RDD[(Pid, EdgePartition[ED])], + vTableIndex: VertexSetIndex): VertexSetRDD[Array[Pid]] = { + val preAgg = eTable.mapPartitions { iter => + val (pid, edgePartition) = iter.next() + val vSet = new VertexSet + edgePartition.foreach(e => {vSet.add(e.srcId)}) + vSet.iterator.map { vid => (vid.toLong, pid) } + } + VertexSetRDD[Pid, ArrayBuffer[Pid]](preAgg, vTableIndex, + (p: Pid) => ArrayBuffer(p), + (ab: ArrayBuffer[Pid], p:Pid) => {ab.append(p); ab}, + (a: ArrayBuffer[Pid], b: ArrayBuffer[Pid]) => a ++ b) + .mapValues(a => a.toArray).cache() + } + protected def createLocalVidMap[ED: ClassManifest](eTable: RDD[(Pid, EdgePartition[ED])]): RDD[(Pid, VertexIdToIndexMap)] = { eTable.mapPartitions( _.map{ case (pid, epart) => @@ -468,8 +485,22 @@ object GraphImpl { ClosureCleaner.clean(mapFunc) ClosureCleaner.clean(reduceFunc) + // For each vertex, replicate its attribute only to partitions where it is + // in the relevant position in an edge. + val mapFuncUsesSrcAttr = + BytecodeUtils.invokedMethod(mapFunc, classOf[EdgeTriplet[VD, ED]], "srcAttr") + val mapFuncUsesDstAttr = + BytecodeUtils.invokedMethod(mapFunc, classOf[EdgeTriplet[VD, ED]], "dstAttr") + val vTableReplicatedValues = + if (mapFuncUsesSrcAttr && !mapFuncUsesDstAttr) { + val vid2pidSourceAttrOnly = createVid2PidSourceAttrOnly(g.eTable, g.vTable.index) + createVTableReplicated(g.vTable, vid2pidSourceAttrOnly, g.localVidMap) + } else { + g.vTableReplicatedValues + } + // Map and preaggregate - val preAgg = g.eTable.zipPartitions(g.localVidMap, g.vTableReplicatedValues){ + val preAgg = g.eTable.zipPartitions(g.localVidMap, vTableReplicatedValues){ (edgePartitionIter, vidToIndexIter, vertexArrayIter) => val (_, edgePartition) = edgePartitionIter.next() val (_, vidToIndex) = vidToIndexIter.next() @@ -488,8 +519,12 @@ object GraphImpl { edgePartition.foreach { e => et.set(e) - et.srcAttr = vmap(e.srcId) - et.dstAttr = vmap(e.dstId) + if (mapFuncUsesSrcAttr) { + et.srcAttr = vmap(e.srcId) + } + if (mapFuncUsesDstAttr) { + et.dstAttr = vmap(e.dstId) + } // TODO(rxin): rewrite the foreach using a simple while loop to speed things up. // Also given we are only allowing zero, one, or two messages, we can completely unroll // the for loop. diff --git a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala index ec548bda16069..37fb60c4cc9c8 100644 --- a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala @@ -58,6 +58,26 @@ class GraphSuite extends FunSuite with LocalSparkContext { } } + test("aggregateNeighborsSourceAttrOnly") { + withSpark(new SparkContext("local", "test")) { sc => + val n = 3 + // Create a star graph where the degree of each vertex is its attribute + val star = Graph(sc.parallelize((1 to n).map(x => ((n + 1): Vid, x: Vid)))) + + val totalOfInNeighborDegrees = star.aggregateNeighbors( + (vid, edge) => { + // All edges have the center vertex as the source, which has degree n + if (edge.srcAttr != n) { + throw new Exception("edge.srcAttr is %d, expected %d".format(edge.srcAttr, n)) + } + Some(edge.srcAttr) + }, + (a: Int, b: Int) => a + b, + EdgeDirection.In) + assert(totalOfInNeighborDegrees.collect().toSet === (1 to n).map(x => (x, n)).toSet) + } + } + test("joinVertices") { withSpark(new SparkContext("local", "test")) { sc => val vertices = sc.parallelize(Seq[(Vid, String)]((1, "one"), (2, "two"), (3, "three")), 2) @@ -87,6 +107,6 @@ class GraphSuite extends FunSuite with LocalSparkContext { assert(b.zipJoin(c)((id, b, c) => b + c).map(x => x._2).reduce(_+_) === 0) } - } - + } + } From 27e4355d61aef55cd5755cbaf2fce75022eaa937 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Thu, 7 Nov 2013 19:28:07 -0800 Subject: [PATCH 188/531] Test no vertex attribute replication --- .../org/apache/spark/graph/GraphSuite.scala | 20 ++++++++++++++++++- 1 file changed, 19 insertions(+), 1 deletion(-) diff --git a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala index 37fb60c4cc9c8..f08efa7bb9545 100644 --- a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala @@ -58,7 +58,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { } } - test("aggregateNeighborsSourceAttrOnly") { + test("aggregateNeighbors - source attribute replication only") { withSpark(new SparkContext("local", "test")) { sc => val n = 3 // Create a star graph where the degree of each vertex is its attribute @@ -78,6 +78,24 @@ class GraphSuite extends FunSuite with LocalSparkContext { } } + test("aggregateNeighbors - no vertex attribute replication") { + withSpark(new SparkContext("local[2]", "test")) { sc => + val n = 3 + // Not serializable because it captures org.scalatest.Engine + class UnserializableAttribute {} + // Create a star graph where vertex attributes are not serializable + val star = Graph(sc.parallelize((1 to n).map(x => (0: Vid, x: Vid)))) + .mapVertices { (id, attr) => new UnserializableAttribute } + + // Should not serialize any vertex attributes + val ignoreAttributes = star.aggregateNeighbors( + (vid, edge) => Some(0), + (a: Int, b: Int) => a + b, + EdgeDirection.In) + assert(ignoreAttributes.collect().toSet === (1 to n).map(x => (x, 0)).toSet) + } + } + test("joinVertices") { withSpark(new SparkContext("local", "test")) { sc => val vertices = sc.parallelize(Seq[(Vid, String)]((1, "one"), (2, "two"), (3, "three")), 2) From cdbd19bbeed7c737a7826ee766475adc0c56de05 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 8 Nov 2013 19:04:14 -0800 Subject: [PATCH 189/531] Create all versions of vid2pid ahead of time --- .../apache/spark/graph/impl/GraphImpl.scala | 102 ++++++++---------- .../org/apache/spark/graph/impl/Vid2Pid.scala | 48 +++++++++ .../org/apache/spark/graph/GraphSuite.scala | 38 ------- 3 files changed, 91 insertions(+), 97 deletions(-) create mode 100644 graph/src/main/scala/org/apache/spark/graph/impl/Vid2Pid.scala diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index 64fdb10831436..7c0b9e23f28dc 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -22,9 +22,9 @@ import org.apache.spark.util.collection.{BitSet, OpenHashSet, PrimitiveKeyOpenHa * The Iterator type returned when constructing edge triplets */ class EdgeTripletIterator[VD: ClassManifest, ED: ClassManifest]( - val vidToIndex: VertexIdToIndexMap, - val vertexArray: Array[VD], - val edgePartition: EdgePartition[ED]) extends Iterator[EdgeTriplet[VD, ED]] { + val vidToIndex: VertexIdToIndexMap, + val vertexArray: Array[VD], + val edgePartition: EdgePartition[ED]) extends Iterator[EdgeTriplet[VD, ED]] { private var pos = 0 private val et = new EdgeTriplet[VD, ED] @@ -66,7 +66,7 @@ class EdgeTripletIterator[VD: ClassManifest, ED: ClassManifest]( */ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( @transient val vTable: VertexSetRDD[VD], - @transient val vid2pid: VertexSetRDD[Array[Pid]], + @transient val vid2pid: Vid2Pid, @transient val localVidMap: RDD[(Pid, VertexIdToIndexMap)], @transient val eTable: RDD[(Pid, EdgePartition[ED])] ) extends Graph[VD, ED] { @@ -82,8 +82,18 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( * (vTableReplicatedValues: VertexSetRDD[Pid, Array[VD]]) holds the vertex data * and is arranged as described above. */ - @transient val vTableReplicatedValues: RDD[(Pid, Array[VD])] = - createVTableReplicated(vTable, vid2pid, localVidMap) + @transient val vTableReplicatedValuesBothAttrs: RDD[(Pid, Array[VD])] = + createVTableReplicated(vTable, vid2pid.bothAttrs, localVidMap) + + @transient val vTableReplicatedValuesSrcAttrOnly: RDD[(Pid, Array[VD])] = + createVTableReplicated(vTable, vid2pid.srcAttrOnly, localVidMap) + + @transient val vTableReplicatedValuesDstAttrOnly: RDD[(Pid, Array[VD])] = + createVTableReplicated(vTable, vid2pid.dstAttrOnly, localVidMap) + + // TODO(ankurdave): create this more efficiently + @transient val vTableReplicatedValuesNoAttrs: RDD[(Pid, Array[VD])] = + createVTableReplicated(vTable, vid2pid.noAttrs, localVidMap) /** Return a RDD of vertices. */ @transient override val vertices = vTable @@ -95,7 +105,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( /** Return a RDD that brings edges with its source and destination vertices together. */ @transient override val triplets: RDD[EdgeTriplet[VD, ED]] = - makeTriplets(localVidMap, vTableReplicatedValues, eTable) + makeTriplets(localVidMap, vTableReplicatedValuesBothAttrs, eTable) override def persist(newLevel: StorageLevel): Graph[VD, ED] = { eTable.persist(newLevel) @@ -111,15 +121,22 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( override def statistics: Map[String, Any] = { val numVertices = this.numVertices val numEdges = this.numEdges - val replicationRatio = - vid2pid.map(kv => kv._2.size).sum / vTable.count + val replicationRatioBothAttrs = + vid2pid.bothAttrs.map(kv => kv._2.size).sum / numVertices + val replicationRatioSrcAttrOnly = + vid2pid.srcAttrOnly.map(kv => kv._2.size).sum / numVertices + val replicationRatioDstAttrOnly = + vid2pid.dstAttrOnly.map(kv => kv._2.size).sum / numVertices val loadArray = eTable.map{ case (pid, epart) => epart.data.size }.collect.map(x => x.toDouble / numEdges) val minLoad = loadArray.min val maxLoad = loadArray.max Map( "Num Vertices" -> numVertices, "Num Edges" -> numEdges, - "Replication" -> replicationRatio, "Load Array" -> loadArray, + "Replication (both)" -> replicationRatioBothAttrs, + "Replication (src only)" -> replicationRatioSrcAttrOnly, + "Replication (dest only)" -> replicationRatioDstAttrOnly, + "Load Array" -> loadArray, "Min Load" -> minLoad, "Max Load" -> maxLoad) } @@ -162,18 +179,18 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( traverseLineage(vTable, " ", visited) visited += (vTable.id -> "vTable") - println("\n\nvid2pid -----------------------------------------") - traverseLineage(vid2pid, " ", visited) - visited += (vid2pid.id -> "vid2pid") - visited += (vid2pid.valuesRDD.id -> "vid2pid.values") + println("\n\nvid2pid.bothAttrs -------------------------------") + traverseLineage(vid2pid.bothAttrs, " ", visited) + visited += (vid2pid.bothAttrs.id -> "vid2pid") + visited += (vid2pid.bothAttrs.valuesRDD.id -> "vid2pid.values") println("\n\nlocalVidMap -------------------------------------") traverseLineage(localVidMap, " ", visited) visited += (localVidMap.id -> "localVidMap") - println("\n\nvTableReplicatedValues --------------------------") - traverseLineage(vTableReplicatedValues, " ", visited) - visited += (vTableReplicatedValues.id -> "vTableReplicatedValues") + println("\n\nvTableReplicatedValuesBothAttrs -----------------") + traverseLineage(vTableReplicatedValuesBothAttrs, " ", visited) + visited += (vTableReplicatedValuesBothAttrs.id -> "vTableReplicatedValuesBothAttrs") println("\n\ntriplets ----------------------------------------") traverseLineage(triplets, " ", visited) @@ -233,7 +250,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( // Construct the Vid2Pid map. Here we assume that the filter operation // behaves deterministically. // @todo reindex the vertex and edge tables - val newVid2Pid = createVid2Pid(newETable, newVTable.index) + val newVid2Pid = new Vid2Pid(newETable, newVTable.index) val newVidMap = createLocalVidMap(newETable) new GraphImpl(newVTable, newVid2Pid, localVidMap, newETable) @@ -329,7 +346,7 @@ object GraphImpl { * */ val etable = createETable(edges) - val vid2pid = createVid2Pid(etable, vtable.index) + val vid2pid = new Vid2Pid(etable, vtable.index) val localVidMap = createLocalVidMap(etable) new GraphImpl(vtable, vid2pid, localVidMap, etable) } @@ -369,38 +386,6 @@ object GraphImpl { }, preservesPartitioning = true).cache() } - protected def createVid2Pid[ED: ClassManifest]( - eTable: RDD[(Pid, EdgePartition[ED])], - vTableIndex: VertexSetIndex): VertexSetRDD[Array[Pid]] = { - val preAgg = eTable.mapPartitions { iter => - val (pid, edgePartition) = iter.next() - val vSet = new VertexSet - edgePartition.foreach(e => {vSet.add(e.srcId); vSet.add(e.dstId)}) - vSet.iterator.map { vid => (vid.toLong, pid) } - }.partitionBy(vTableIndex.rdd.partitioner.get) - VertexSetRDD[Pid, ArrayBuffer[Pid]](preAgg, vTableIndex, - (p: Pid) => ArrayBuffer(p), - (ab: ArrayBuffer[Pid], p:Pid) => {ab.append(p); ab}, - (a: ArrayBuffer[Pid], b: ArrayBuffer[Pid]) => a ++ b) - .mapValues(a => a.toArray).cache() - } - - protected def createVid2PidSourceAttrOnly[ED: ClassManifest]( - eTable: RDD[(Pid, EdgePartition[ED])], - vTableIndex: VertexSetIndex): VertexSetRDD[Array[Pid]] = { - val preAgg = eTable.mapPartitions { iter => - val (pid, edgePartition) = iter.next() - val vSet = new VertexSet - edgePartition.foreach(e => {vSet.add(e.srcId)}) - vSet.iterator.map { vid => (vid.toLong, pid) } - } - VertexSetRDD[Pid, ArrayBuffer[Pid]](preAgg, vTableIndex, - (p: Pid) => ArrayBuffer(p), - (ab: ArrayBuffer[Pid], p:Pid) => {ab.append(p); ab}, - (a: ArrayBuffer[Pid], b: ArrayBuffer[Pid]) => a ++ b) - .mapValues(a => a.toArray).cache() - } - protected def createLocalVidMap[ED: ClassManifest](eTable: RDD[(Pid, EdgePartition[ED])]): RDD[(Pid, VertexIdToIndexMap)] = { eTable.mapPartitions( _.map{ case (pid, epart) => @@ -459,7 +444,7 @@ object GraphImpl { def mapTriplets[VD: ClassManifest, ED: ClassManifest, ED2: ClassManifest]( g: GraphImpl[VD, ED], f: EdgeTriplet[VD, ED] => ED2): Graph[VD, ED2] = { - val newETable = g.eTable.zipPartitions(g.localVidMap, g.vTableReplicatedValues){ + val newETable = g.eTable.zipPartitions(g.localVidMap, g.vTableReplicatedValuesBothAttrs){ (edgePartitionIter, vidToIndexIter, vertexArrayIter) => val (pid, edgePartition) = edgePartitionIter.next() val (_, vidToIndex) = vidToIndexIter.next() @@ -491,13 +476,12 @@ object GraphImpl { BytecodeUtils.invokedMethod(mapFunc, classOf[EdgeTriplet[VD, ED]], "srcAttr") val mapFuncUsesDstAttr = BytecodeUtils.invokedMethod(mapFunc, classOf[EdgeTriplet[VD, ED]], "dstAttr") - val vTableReplicatedValues = - if (mapFuncUsesSrcAttr && !mapFuncUsesDstAttr) { - val vid2pidSourceAttrOnly = createVid2PidSourceAttrOnly(g.eTable, g.vTable.index) - createVTableReplicated(g.vTable, vid2pidSourceAttrOnly, g.localVidMap) - } else { - g.vTableReplicatedValues - } + val vTableReplicatedValues = (mapFuncUsesSrcAttr, mapFuncUsesDstAttr) match { + case (true, true) => g.vTableReplicatedValuesBothAttrs + case (true, false) => g.vTableReplicatedValuesSrcAttrOnly + case (false, true) => g.vTableReplicatedValuesDstAttrOnly + case (false, false) => g.vTableReplicatedValuesNoAttrs + } // Map and preaggregate val preAgg = g.eTable.zipPartitions(g.localVidMap, vTableReplicatedValues){ diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/Vid2Pid.scala b/graph/src/main/scala/org/apache/spark/graph/impl/Vid2Pid.scala new file mode 100644 index 0000000000000..d8c8d35ee10c0 --- /dev/null +++ b/graph/src/main/scala/org/apache/spark/graph/impl/Vid2Pid.scala @@ -0,0 +1,48 @@ +package org.apache.spark.graph.impl + +import scala.collection.JavaConversions._ +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.graph._ +import org.apache.spark.rdd.RDD +import org.apache.spark.storage.StorageLevel + +/** + * Stores the layout of vertex attributes. + */ +class Vid2Pid( + eTable: RDD[(Pid, EdgePartition[ED])] forSome { type ED }, + vTableIndex: VertexSetIndex) { + + val bothAttrs: VertexSetRDD[Array[Pid]] = createVid2Pid(true, true) + val srcAttrOnly: VertexSetRDD[Array[Pid]] = createVid2Pid(true, false) + val dstAttrOnly: VertexSetRDD[Array[Pid]] = createVid2Pid(false, true) + // TODO(ankurdave): create this more efficiently + val noAttrs: VertexSetRDD[Array[Pid]] = createVid2Pid(false, false) + + def persist(newLevel: StorageLevel) { + bothAttrs.persist(newLevel) + srcAttrOnly.persist(newLevel) + dstAttrOnly.persist(newLevel) + noAttrs.persist(newLevel) + } + + private def createVid2Pid( + includeSrcAttr: Boolean, + includeDstAttr: Boolean): VertexSetRDD[Array[Pid]] = { + val preAgg = eTable.mapPartitions { iter => + val (pid, edgePartition) = iter.next() + val vSet = new VertexSet + edgePartition.foreach(e => { + if (includeSrcAttr) vSet.add(e.srcId) + if (includeDstAttr) vSet.add(e.dstId) + }) + vSet.iterator.map { vid => (vid.toLong, pid) } + } + VertexSetRDD[Pid, ArrayBuffer[Pid]](preAgg, vTableIndex, + (p: Pid) => ArrayBuffer(p), + (ab: ArrayBuffer[Pid], p:Pid) => {ab.append(p); ab}, + (a: ArrayBuffer[Pid], b: ArrayBuffer[Pid]) => a ++ b) + .mapValues(a => a.toArray).cache() + } +} diff --git a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala index f08efa7bb9545..782e337504867 100644 --- a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala @@ -58,44 +58,6 @@ class GraphSuite extends FunSuite with LocalSparkContext { } } - test("aggregateNeighbors - source attribute replication only") { - withSpark(new SparkContext("local", "test")) { sc => - val n = 3 - // Create a star graph where the degree of each vertex is its attribute - val star = Graph(sc.parallelize((1 to n).map(x => ((n + 1): Vid, x: Vid)))) - - val totalOfInNeighborDegrees = star.aggregateNeighbors( - (vid, edge) => { - // All edges have the center vertex as the source, which has degree n - if (edge.srcAttr != n) { - throw new Exception("edge.srcAttr is %d, expected %d".format(edge.srcAttr, n)) - } - Some(edge.srcAttr) - }, - (a: Int, b: Int) => a + b, - EdgeDirection.In) - assert(totalOfInNeighborDegrees.collect().toSet === (1 to n).map(x => (x, n)).toSet) - } - } - - test("aggregateNeighbors - no vertex attribute replication") { - withSpark(new SparkContext("local[2]", "test")) { sc => - val n = 3 - // Not serializable because it captures org.scalatest.Engine - class UnserializableAttribute {} - // Create a star graph where vertex attributes are not serializable - val star = Graph(sc.parallelize((1 to n).map(x => (0: Vid, x: Vid)))) - .mapVertices { (id, attr) => new UnserializableAttribute } - - // Should not serialize any vertex attributes - val ignoreAttributes = star.aggregateNeighbors( - (vid, edge) => Some(0), - (a: Int, b: Int) => a + b, - EdgeDirection.In) - assert(ignoreAttributes.collect().toSet === (1 to n).map(x => (x, 0)).toSet) - } - } - test("joinVertices") { withSpark(new SparkContext("local", "test")) { sc => val vertices = sc.parallelize(Seq[(Vid, String)]((1, "one"), (2, "two"), (3, "three")), 2) From bf4e45e6857c747f4aa6578b930b5c3bc5afb0f1 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Sat, 9 Nov 2013 04:20:30 -0800 Subject: [PATCH 190/531] Factor out VTableReplicatedValues --- .../apache/spark/graph/impl/GraphImpl.scala | 86 +++++-------------- .../graph/impl/VTableReplicatedValues.scala | 72 ++++++++++++++++ .../org/apache/spark/graph/impl/Vid2Pid.scala | 28 ++++-- 3 files changed, 112 insertions(+), 74 deletions(-) create mode 100644 graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicatedValues.scala diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index 7c0b9e23f28dc..ae1ea715e2bad 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -63,6 +63,13 @@ class EdgeTripletIterator[VD: ClassManifest, ED: ClassManifest]( /** * A Graph RDD that supports computation on graphs. + * + * @param localVidMap Stores the location of vertex attributes after they are + * replicated. Within each partition, localVidMap holds a map from vertex ID to + * the index where that vertex's attribute is stored. This index refers to the + * arrays in the same partition in the variants of + * [[VTableReplicatedValues]]. Therefore, localVidMap can be reused across + * changes to the vertex attributes. */ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( @transient val vTable: VertexSetRDD[VD], @@ -73,27 +80,8 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( def this() = this(null, null, null, null) - /** - * (localVidMap: VertexSetRDD[Pid, VertexIdToIndexMap]) is a version of the - * vertex data after it is replicated. Within each partition, it holds a map - * from vertex ID to the index where that vertex's attribute is stored. This - * index refers to an array in the same partition in vTableReplicatedValues. - * - * (vTableReplicatedValues: VertexSetRDD[Pid, Array[VD]]) holds the vertex data - * and is arranged as described above. - */ - @transient val vTableReplicatedValuesBothAttrs: RDD[(Pid, Array[VD])] = - createVTableReplicated(vTable, vid2pid.bothAttrs, localVidMap) - - @transient val vTableReplicatedValuesSrcAttrOnly: RDD[(Pid, Array[VD])] = - createVTableReplicated(vTable, vid2pid.srcAttrOnly, localVidMap) - - @transient val vTableReplicatedValuesDstAttrOnly: RDD[(Pid, Array[VD])] = - createVTableReplicated(vTable, vid2pid.dstAttrOnly, localVidMap) - - // TODO(ankurdave): create this more efficiently - @transient val vTableReplicatedValuesNoAttrs: RDD[(Pid, Array[VD])] = - createVTableReplicated(vTable, vid2pid.noAttrs, localVidMap) + @transient val vTableReplicatedValues: VTableReplicatedValues[VD] = + new VTableReplicatedValues(vTable, vid2pid, localVidMap) /** Return a RDD of vertices. */ @transient override val vertices = vTable @@ -105,7 +93,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( /** Return a RDD that brings edges with its source and destination vertices together. */ @transient override val triplets: RDD[EdgeTriplet[VD, ED]] = - makeTriplets(localVidMap, vTableReplicatedValuesBothAttrs, eTable) + makeTriplets(localVidMap, vTableReplicatedValues.bothAttrs, eTable) override def persist(newLevel: StorageLevel): Graph[VD, ED] = { eTable.persist(newLevel) @@ -188,9 +176,9 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( traverseLineage(localVidMap, " ", visited) visited += (localVidMap.id -> "localVidMap") - println("\n\nvTableReplicatedValuesBothAttrs -----------------") - traverseLineage(vTableReplicatedValuesBothAttrs, " ", visited) - visited += (vTableReplicatedValuesBothAttrs.id -> "vTableReplicatedValuesBothAttrs") + println("\n\nvTableReplicatedValues.bothAttrs ----------------") + traverseLineage(vTableReplicatedValues.bothAttrs, " ", visited) + visited += (vTableReplicatedValues.bothAttrs.id -> "vTableReplicatedValues.bothAttrs") println("\n\ntriplets ----------------------------------------") traverseLineage(triplets, " ", visited) @@ -386,8 +374,9 @@ object GraphImpl { }, preservesPartitioning = true).cache() } - protected def createLocalVidMap[ED: ClassManifest](eTable: RDD[(Pid, EdgePartition[ED])]): - RDD[(Pid, VertexIdToIndexMap)] = { + private def createLocalVidMap( + eTable: RDD[(Pid, EdgePartition[ED])] forSome { type ED } + ): RDD[(Pid, VertexIdToIndexMap)] = { eTable.mapPartitions( _.map{ case (pid, epart) => val vidToIndex = new VertexIdToIndexMap epart.foreach{ e => @@ -398,36 +387,6 @@ object GraphImpl { }, preservesPartitioning = true).cache() } - protected def createVTableReplicated[VD: ClassManifest]( - vTable: VertexSetRDD[VD], - vid2pid: VertexSetRDD[Array[Pid]], - replicationMap: RDD[(Pid, VertexIdToIndexMap)]): - RDD[(Pid, Array[VD])] = { - // Join vid2pid and vTable, generate a shuffle dependency on the joined - // result, and get the shuffle id so we can use it on the slave. - val msgsByPartition = vTable.zipJoinFlatMap(vid2pid) { (vid, vdata, pids) => - // TODO(rxin): reuse VertexBroadcastMessage - pids.iterator.map { pid => - new VertexBroadcastMsg[VD](pid, vid, vdata) - } - }.partitionBy(replicationMap.partitioner.get).cache() - - replicationMap.zipPartitions(msgsByPartition){ - (mapIter, msgsIter) => - val (pid, vidToIndex) = mapIter.next() - assert(!mapIter.hasNext) - // Populate the vertex array using the vidToIndex map - val vertexArray = new Array[VD](vidToIndex.capacity) - for (msg <- msgsIter) { - val ind = vidToIndex.getPos(msg.vid) & OpenHashSet.POSITION_MASK - vertexArray(ind) = msg.data - } - Iterator((pid, vertexArray)) - }.cache() - - // @todo assert edge table has partitioner - } - def makeTriplets[VD: ClassManifest, ED: ClassManifest]( localVidMap: RDD[(Pid, VertexIdToIndexMap)], vTableReplicatedValues: RDD[(Pid, Array[VD]) ], @@ -444,7 +403,7 @@ object GraphImpl { def mapTriplets[VD: ClassManifest, ED: ClassManifest, ED2: ClassManifest]( g: GraphImpl[VD, ED], f: EdgeTriplet[VD, ED] => ED2): Graph[VD, ED2] = { - val newETable = g.eTable.zipPartitions(g.localVidMap, g.vTableReplicatedValuesBothAttrs){ + val newETable = g.eTable.zipPartitions(g.localVidMap, g.vTableReplicatedValues.bothAttrs){ (edgePartitionIter, vidToIndexIter, vertexArrayIter) => val (pid, edgePartition) = edgePartitionIter.next() val (_, vidToIndex) = vidToIndexIter.next() @@ -476,15 +435,12 @@ object GraphImpl { BytecodeUtils.invokedMethod(mapFunc, classOf[EdgeTriplet[VD, ED]], "srcAttr") val mapFuncUsesDstAttr = BytecodeUtils.invokedMethod(mapFunc, classOf[EdgeTriplet[VD, ED]], "dstAttr") - val vTableReplicatedValues = (mapFuncUsesSrcAttr, mapFuncUsesDstAttr) match { - case (true, true) => g.vTableReplicatedValuesBothAttrs - case (true, false) => g.vTableReplicatedValuesSrcAttrOnly - case (false, true) => g.vTableReplicatedValuesDstAttrOnly - case (false, false) => g.vTableReplicatedValuesNoAttrs - } // Map and preaggregate - val preAgg = g.eTable.zipPartitions(g.localVidMap, vTableReplicatedValues){ + val preAgg = g.eTable.zipPartitions( + g.localVidMap, + g.vTableReplicatedValues.get(mapFuncUsesSrcAttr, mapFuncUsesDstAttr) + ){ (edgePartitionIter, vidToIndexIter, vertexArrayIter) => val (_, edgePartition) = edgePartitionIter.next() val (_, vidToIndex) = vidToIndexIter.next() diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicatedValues.scala b/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicatedValues.scala new file mode 100644 index 0000000000000..a9ab6255fa3c4 --- /dev/null +++ b/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicatedValues.scala @@ -0,0 +1,72 @@ +package org.apache.spark.graph.impl + +import org.apache.spark.rdd.RDD +import org.apache.spark.util.collection.OpenHashSet + +import org.apache.spark.graph._ +import org.apache.spark.graph.impl.MsgRDDFunctions._ + +/** + * Stores the vertex attribute values after they are replicated. See + * the description of localVidMap in [[GraphImpl]]. + */ +class VTableReplicatedValues[VD: ClassManifest]( + vTable: VertexSetRDD[VD], + vid2pid: Vid2Pid, + localVidMap: RDD[(Pid, VertexIdToIndexMap)]) { + + val bothAttrs: RDD[(Pid, Array[VD])] = + VTableReplicatedValues.createVTableReplicated(vTable, vid2pid, localVidMap, true, true) + val srcAttrOnly: RDD[(Pid, Array[VD])] = + VTableReplicatedValues.createVTableReplicated(vTable, vid2pid, localVidMap, true, false) + val dstAttrOnly: RDD[(Pid, Array[VD])] = + VTableReplicatedValues.createVTableReplicated(vTable, vid2pid, localVidMap, false, true) + val noAttrs: RDD[(Pid, Array[VD])] = + VTableReplicatedValues.createVTableReplicated(vTable, vid2pid, localVidMap, false, false) + + + def get(includeSrcAttr: Boolean, includeDstAttr: Boolean): RDD[(Pid, Array[VD])] = + (includeSrcAttr, includeDstAttr) match { + case (true, true) => bothAttrs + case (true, false) => srcAttrOnly + case (false, true) => dstAttrOnly + case (false, false) => noAttrs + } +} + + + +object VTableReplicatedValues { + protected def createVTableReplicated[VD: ClassManifest]( + vTable: VertexSetRDD[VD], + vid2pid: Vid2Pid, + localVidMap: RDD[(Pid, VertexIdToIndexMap)], + includeSrcAttr: Boolean, + includeDstAttr: Boolean): RDD[(Pid, Array[VD])] = { + + // Join vid2pid and vTable, generate a shuffle dependency on the joined + // result, and get the shuffle id so we can use it on the slave. + val msgsByPartition = vTable.zipJoinFlatMap(vid2pid.get(includeSrcAttr, includeDstAttr)) { + // TODO(rxin): reuse VertexBroadcastMessage + (vid, vdata, pids) => pids.iterator.map { pid => + new VertexBroadcastMsg[VD](pid, vid, vdata) + } + }.partitionBy(localVidMap.partitioner.get).cache() + + localVidMap.zipPartitions(msgsByPartition){ + (mapIter, msgsIter) => + val (pid, vidToIndex) = mapIter.next() + assert(!mapIter.hasNext) + // Populate the vertex array using the vidToIndex map + val vertexArray = new Array[VD](vidToIndex.capacity) + for (msg <- msgsIter) { + val ind = vidToIndex.getPos(msg.vid) & OpenHashSet.POSITION_MASK + vertexArray(ind) = msg.data + } + Iterator((pid, vertexArray)) + }.cache() + + // @todo assert edge table has partitioner + } + +} diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/Vid2Pid.scala b/graph/src/main/scala/org/apache/spark/graph/impl/Vid2Pid.scala index d8c8d35ee10c0..9bdca7f407630 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/Vid2Pid.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/Vid2Pid.scala @@ -3,12 +3,13 @@ package org.apache.spark.graph.impl import scala.collection.JavaConversions._ import scala.collection.mutable.ArrayBuffer -import org.apache.spark.graph._ import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel +import org.apache.spark.graph._ + /** - * Stores the layout of vertex attributes. + * Stores the layout of vertex attributes for GraphImpl. */ class Vid2Pid( eTable: RDD[(Pid, EdgePartition[ED])] forSome { type ED }, @@ -17,9 +18,16 @@ class Vid2Pid( val bothAttrs: VertexSetRDD[Array[Pid]] = createVid2Pid(true, true) val srcAttrOnly: VertexSetRDD[Array[Pid]] = createVid2Pid(true, false) val dstAttrOnly: VertexSetRDD[Array[Pid]] = createVid2Pid(false, true) - // TODO(ankurdave): create this more efficiently val noAttrs: VertexSetRDD[Array[Pid]] = createVid2Pid(false, false) + def get(includeSrcAttr: Boolean, includeDstAttr: Boolean): VertexSetRDD[Array[Pid]] = + (includeSrcAttr, includeDstAttr) match { + case (true, true) => bothAttrs + case (true, false) => srcAttrOnly + case (false, true) => dstAttrOnly + case (false, false) => noAttrs + } + def persist(newLevel: StorageLevel) { bothAttrs.persist(newLevel) srcAttrOnly.persist(newLevel) @@ -28,15 +36,17 @@ class Vid2Pid( } private def createVid2Pid( - includeSrcAttr: Boolean, - includeDstAttr: Boolean): VertexSetRDD[Array[Pid]] = { + includeSrcAttr: Boolean, + includeDstAttr: Boolean): VertexSetRDD[Array[Pid]] = { val preAgg = eTable.mapPartitions { iter => val (pid, edgePartition) = iter.next() val vSet = new VertexSet - edgePartition.foreach(e => { - if (includeSrcAttr) vSet.add(e.srcId) - if (includeDstAttr) vSet.add(e.dstId) - }) + if (includeSrcAttr || includeDstAttr) { + edgePartition.foreach(e => { + if (includeSrcAttr) vSet.add(e.srcId) + if (includeDstAttr) vSet.add(e.dstId) + }) + } vSet.iterator.map { vid => (vid.toLong, pid) } } VertexSetRDD[Pid, ArrayBuffer[Pid]](preAgg, vTableIndex, From aa24b0bbe8d0f15d8edae5c1caf5da14de4a5607 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Sat, 9 Nov 2013 04:23:38 -0800 Subject: [PATCH 191/531] Add test for mapReduceTriplets in GraphSuite --- .../scala/org/apache/spark/graph/GraphSuite.scala | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala index 782e337504867..9c22608554671 100644 --- a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala @@ -33,6 +33,18 @@ class GraphSuite extends FunSuite with LocalSparkContext { } } + test("mapReduceTriplets") { + withSpark(new SparkContext("local", "test")) { sc => + val n = 3 + val star = Graph(sc.parallelize((1 to n).map(x => (0: Vid, x: Vid)))) + + val neighborDegreeSums = star.mapReduceTriplets( + edge => Array((edge.srcId, edge.dstAttr), (edge.dstId, edge.srcAttr)), + (a: Int, b: Int) => a + b) + assert(neighborDegreeSums.collect().toSet === (0 to n).map(x => (x, n)).toSet) + } + } + test("aggregateNeighbors") { withSpark(new SparkContext("local", "test")) { sc => val n = 3 From 53d24a973e7bd71d1509a326f0f12376a252f1db Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Sat, 9 Nov 2013 04:48:51 -0800 Subject: [PATCH 192/531] Fix typo --- .../src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index ae1ea715e2bad..e723ad1cec39b 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -170,7 +170,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( println("\n\nvid2pid.bothAttrs -------------------------------") traverseLineage(vid2pid.bothAttrs, " ", visited) visited += (vid2pid.bothAttrs.id -> "vid2pid") - visited += (vid2pid.bothAttrs.valuesRDD.id -> "vid2pid.values") + visited += (vid2pid.bothAttrs.valuesRDD.id -> "vid2pid.bothAttrs") println("\n\nlocalVidMap -------------------------------------") traverseLineage(localVidMap, " ", visited) From 502c5117110574ac1daf3d8347fb2ad71da80e71 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Sat, 9 Nov 2013 22:10:29 -0800 Subject: [PATCH 193/531] Use pid2vid for creating VTableReplicatedValues --- .../spark/graph/GraphKryoRegistrator.scala | 1 + .../graph/impl/VTableReplicatedValues.scala | 48 ++++++++++++++----- 2 files changed, 36 insertions(+), 13 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphKryoRegistrator.scala b/graph/src/main/scala/org/apache/spark/graph/GraphKryoRegistrator.scala index 82b9198e432c7..baf8099556c06 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphKryoRegistrator.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphKryoRegistrator.scala @@ -18,6 +18,7 @@ class GraphKryoRegistrator extends KryoRegistrator { kryo.register(classOf[EdgePartition[Object]]) kryo.register(classOf[BitSet]) kryo.register(classOf[VertexIdToIndexMap]) + kryo.register(classOf[VertexAttributeBlock[Object]]) // This avoids a large number of hash table lookups. kryo.setReferences(false) } diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicatedValues.scala b/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicatedValues.scala index a9ab6255fa3c4..25cd1b8054ec9 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicatedValues.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicatedValues.scala @@ -1,7 +1,10 @@ package org.apache.spark.graph.impl +import scala.collection.mutable.ArrayBuilder + +import org.apache.spark.SparkContext._ import org.apache.spark.rdd.RDD -import org.apache.spark.util.collection.OpenHashSet +import org.apache.spark.util.collection.{OpenHashSet, PrimitiveKeyOpenHashMap} import org.apache.spark.graph._ import org.apache.spark.graph.impl.MsgRDDFunctions._ @@ -34,7 +37,7 @@ class VTableReplicatedValues[VD: ClassManifest]( } } - +class VertexAttributeBlock[VD: ClassManifest](val vids: Array[Vid], val attrs: Array[VD]) object VTableReplicatedValues { protected def createVTableReplicated[VD: ClassManifest]( @@ -44,13 +47,30 @@ object VTableReplicatedValues { includeSrcAttr: Boolean, includeDstAttr: Boolean): RDD[(Pid, Array[VD])] = { - // Join vid2pid and vTable, generate a shuffle dependency on the joined - // result, and get the shuffle id so we can use it on the slave. - val msgsByPartition = vTable.zipJoinFlatMap(vid2pid.get(includeSrcAttr, includeDstAttr)) { - // TODO(rxin): reuse VertexBroadcastMessage - (vid, vdata, pids) => pids.iterator.map { pid => - new VertexBroadcastMsg[VD](pid, vid, vdata) + // Within each partition of vid2pid, construct a pid2vid mapping + val numPartitions = vTable.partitions.size + val pid2vid = vid2pid.get(includeSrcAttr, includeDstAttr).mapPartitions { iter => + val pid2vidLocal = Array.fill[ArrayBuilder[Vid]](numPartitions)(ArrayBuilder.make[Vid]) + for ((vid, pids) <- iter) { + pids.foreach { pid => pid2vidLocal(pid) += vid } } + Iterator(pid2vidLocal.map(_.result)) + } + + val msgsByPartition = pid2vid.zipPartitions(vTable.index.rdd, vTable.valuesRDD) { + (pid2vidIter, indexIter, valuesIter) => + val pid2vid = pid2vidIter.next() + val index = indexIter.next() + val values = valuesIter.next() + val vmap = new PrimitiveKeyOpenHashMap(index, values._1) + + // Send each partition the vertex attributes it wants + val output = new Array[(Pid, VertexAttributeBlock[VD])](pid2vid.size) + for (pid <- 0 until pid2vid.size) { + val block = new VertexAttributeBlock(pid2vid(pid), pid2vid(pid).map(vid => vmap(vid))) + output(pid) = (pid, block) + } + output.iterator }.partitionBy(localVidMap.partitioner.get).cache() localVidMap.zipPartitions(msgsByPartition){ @@ -59,14 +79,16 @@ object VTableReplicatedValues { assert(!mapIter.hasNext) // Populate the vertex array using the vidToIndex map val vertexArray = new Array[VD](vidToIndex.capacity) - for (msg <- msgsIter) { - val ind = vidToIndex.getPos(msg.vid) & OpenHashSet.POSITION_MASK - vertexArray(ind) = msg.data + for ((_, block) <- msgsIter) { + for (i <- 0 until block.vids.size) { + val vid = block.vids(i) + val attr = block.attrs(i) + val ind = vidToIndex.getPos(vid) & OpenHashSet.POSITION_MASK + vertexArray(ind) = attr + } } Iterator((pid, vertexArray)) }.cache() - - // @todo assert edge table has partitioner } } From d1ff1b722274de8e03938452d8155f2a26c55f96 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Sun, 10 Nov 2013 01:51:42 -0800 Subject: [PATCH 194/531] Build pid2vid structures only once, in Vid2Pid --- .../graph/impl/VTableReplicatedValues.scala | 12 +------- .../org/apache/spark/graph/impl/Vid2Pid.scala | 29 +++++++++++++++++++ 2 files changed, 30 insertions(+), 11 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicatedValues.scala b/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicatedValues.scala index 25cd1b8054ec9..fee2d40ee4aa1 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicatedValues.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicatedValues.scala @@ -1,7 +1,5 @@ package org.apache.spark.graph.impl -import scala.collection.mutable.ArrayBuilder - import org.apache.spark.SparkContext._ import org.apache.spark.rdd.RDD import org.apache.spark.util.collection.{OpenHashSet, PrimitiveKeyOpenHashMap} @@ -47,15 +45,7 @@ object VTableReplicatedValues { includeSrcAttr: Boolean, includeDstAttr: Boolean): RDD[(Pid, Array[VD])] = { - // Within each partition of vid2pid, construct a pid2vid mapping - val numPartitions = vTable.partitions.size - val pid2vid = vid2pid.get(includeSrcAttr, includeDstAttr).mapPartitions { iter => - val pid2vidLocal = Array.fill[ArrayBuilder[Vid]](numPartitions)(ArrayBuilder.make[Vid]) - for ((vid, pids) <- iter) { - pids.foreach { pid => pid2vidLocal(pid) += vid } - } - Iterator(pid2vidLocal.map(_.result)) - } + val pid2vid = vid2pid.getPid2Vid(includeSrcAttr, includeDstAttr) val msgsByPartition = pid2vid.zipPartitions(vTable.index.rdd, vTable.valuesRDD) { (pid2vidIter, indexIter, valuesIter) => diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/Vid2Pid.scala b/graph/src/main/scala/org/apache/spark/graph/impl/Vid2Pid.scala index 9bdca7f407630..363adbbce949e 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/Vid2Pid.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/Vid2Pid.scala @@ -2,6 +2,7 @@ package org.apache.spark.graph.impl import scala.collection.JavaConversions._ import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.ArrayBuilder import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel @@ -20,6 +21,11 @@ class Vid2Pid( val dstAttrOnly: VertexSetRDD[Array[Pid]] = createVid2Pid(false, true) val noAttrs: VertexSetRDD[Array[Pid]] = createVid2Pid(false, false) + val pid2VidBothAttrs: RDD[Array[Array[Vid]]] = createPid2Vid(bothAttrs) + val pid2VidSrcAttrOnly: RDD[Array[Array[Vid]]] = createPid2Vid(srcAttrOnly) + val pid2VidDstAttrOnly: RDD[Array[Array[Vid]]] = createPid2Vid(dstAttrOnly) + val pid2VidNoAttrs: RDD[Array[Array[Vid]]] = createPid2Vid(noAttrs) + def get(includeSrcAttr: Boolean, includeDstAttr: Boolean): VertexSetRDD[Array[Pid]] = (includeSrcAttr, includeDstAttr) match { case (true, true) => bothAttrs @@ -28,6 +34,14 @@ class Vid2Pid( case (false, false) => noAttrs } + def getPid2Vid(includeSrcAttr: Boolean, includeDstAttr: Boolean): RDD[Array[Array[Vid]]] = + (includeSrcAttr, includeDstAttr) match { + case (true, true) => pid2VidBothAttrs + case (true, false) => pid2VidSrcAttrOnly + case (false, true) => pid2VidDstAttrOnly + case (false, false) => pid2VidNoAttrs + } + def persist(newLevel: StorageLevel) { bothAttrs.persist(newLevel) srcAttrOnly.persist(newLevel) @@ -55,4 +69,19 @@ class Vid2Pid( (a: ArrayBuffer[Pid], b: ArrayBuffer[Pid]) => a ++ b) .mapValues(a => a.toArray).cache() } + + /** + * Creates an intermediate pid2vid structure that tells each partition of the + * vertex data where it should go. + */ + private def createPid2Vid(vid2pid: VertexSetRDD[Array[Pid]]): RDD[Array[Array[Vid]]] = { + val numPartitions = vid2pid.partitions.size + vid2pid.mapPartitions { iter => + val pid2vidLocal = Array.fill[ArrayBuilder[Vid]](numPartitions)(ArrayBuilder.make[Vid]) + for ((vid, pids) <- iter) { + pids.foreach { pid => pid2vidLocal(pid) += vid } + } + Iterator(pid2vidLocal.map(_.result)) + } + } } From bee1015620de28806660f20d7c9f404dd3bdb2cb Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Sun, 10 Nov 2013 22:58:32 -0800 Subject: [PATCH 195/531] Handle ClassNotFoundException from ByteCodeUtils ByteCodeUtils.invokedMethod(), which we use in mapReduceTriplets, throws a ClassNotFoundException when called with a closure defined in the console. This commit catches the exception and conservatively assumes the closure references all edge attributes. --- .../org/apache/spark/graph/impl/GraphImpl.scala | 15 +++++++++++---- 1 file changed, 11 insertions(+), 4 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index e723ad1cec39b..d185f9f8cd7ba 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -431,10 +431,8 @@ object GraphImpl { // For each vertex, replicate its attribute only to partitions where it is // in the relevant position in an edge. - val mapFuncUsesSrcAttr = - BytecodeUtils.invokedMethod(mapFunc, classOf[EdgeTriplet[VD, ED]], "srcAttr") - val mapFuncUsesDstAttr = - BytecodeUtils.invokedMethod(mapFunc, classOf[EdgeTriplet[VD, ED]], "dstAttr") + val mapFuncUsesSrcAttr = accessesVertexAttr[VD, ED](mapFunc, "srcAttr") + val mapFuncUsesDstAttr = accessesVertexAttr[VD, ED](mapFunc, "dstAttr") // Map and preaggregate val preAgg = g.eTable.zipPartitions( @@ -571,4 +569,13 @@ object GraphImpl { (col * ceilSqrtNumParts + row) % numParts } + private def accessesVertexAttr[VD: ClassManifest, ED: ClassManifest]( + closure: AnyRef, attrName: String): Boolean = { + try { + BytecodeUtils.invokedMethod(closure, classOf[EdgeTriplet[VD, ED]], attrName) + } catch { + case _: ClassNotFoundException => true // if we don't know, be conservative + } + } + } // end of object GraphImpl From 577092080cdd78fe10a54dc5f4c128595568e0ac Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Mon, 11 Nov 2013 17:29:22 -0800 Subject: [PATCH 196/531] Cleanning up documentation of VertexSetRDD.scala --- .../org/apache/spark/graph/VertexSetRDD.scala | 230 +++--------------- 1 file changed, 28 insertions(+), 202 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala b/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala index 62608e506d85b..401d5f29bc134 100644 --- a/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala @@ -25,6 +25,7 @@ import org.apache.spark.util.collection.{BitSet, OpenHashSet, PrimitiveKeyOpenHa import org.apache.spark.graph.impl.AggregationMsg import org.apache.spark.graph.impl.MsgRDDFunctions._ + /** * The `VertexSetIndex` maintains the per-partition mapping from * vertex id to the corresponding location in the per-partition values @@ -88,7 +89,6 @@ class VertexSetRDD[@specialized V: ClassManifest]( extends RDD[(Vid, V)](index.rdd.context, List(new OneToOneDependency(index.rdd), new OneToOneDependency(valuesRDD)) ) { - /** * Construct a new VertexSetRDD that is indexed by only the keys in the RDD. * The resulting VertexSet will be based on a different index and can @@ -96,7 +96,6 @@ class VertexSetRDD[@specialized V: ClassManifest]( */ def reindex(): VertexSetRDD[V] = VertexSetRDD(this) - /** * An internal representation which joins the block indices with the values * This is used by the compute function to emulate RDD[(Vid, V)] @@ -104,19 +103,16 @@ class VertexSetRDD[@specialized V: ClassManifest]( protected[spark] val tuples = new ZippedRDD(index.rdd.context, index.rdd, valuesRDD) - /** * The partitioner is defined by the index. */ override val partitioner = index.rdd.partitioner - /** * The actual partitions are defined by the tuples. */ override def getPartitions: Array[Partition] = tuples.getPartitions - /** * The preferred locations are computed based on the preferred * locations of the tuples. @@ -124,7 +120,6 @@ class VertexSetRDD[@specialized V: ClassManifest]( override def getPreferredLocations(s: Partition): Seq[String] = tuples.getPreferredLocations(s) - /** * Caching an VertexSetRDD causes the index and values to be cached separately. */ @@ -134,15 +129,12 @@ class VertexSetRDD[@specialized V: ClassManifest]( return this } - /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */ override def persist(): VertexSetRDD[V] = persist(StorageLevel.MEMORY_ONLY) - /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */ override def cache(): VertexSetRDD[V] = persist() - /** * Provide the RDD[(K,V)] equivalent output. */ @@ -152,7 +144,6 @@ class VertexSetRDD[@specialized V: ClassManifest]( } } // end of compute - /** * Restrict the vertex set to the set of vertices satisfying the * given predicate. @@ -190,7 +181,6 @@ class VertexSetRDD[@specialized V: ClassManifest]( new VertexSetRDD[V](index, newValues) } // end of filter - /** * Pass each vertex attribute through a map function and retain the * original RDD's partitioning and index. @@ -214,7 +204,6 @@ class VertexSetRDD[@specialized V: ClassManifest]( new VertexSetRDD[U](index, newValuesRDD) } // end of mapValues - /** * Pass each vertex attribute along with the vertex id through a map * function and retain the original RDD's partitioning and index. @@ -247,8 +236,6 @@ class VertexSetRDD[@specialized V: ClassManifest]( /** - * @todo update docs to reflect function argument - * * Inner join this VertexSet with another VertexSet which has the * same Index. This function will fail if both VertexSets do not * share the same index. The resulting vertex set will only contain @@ -257,6 +244,8 @@ class VertexSetRDD[@specialized V: ClassManifest]( * @tparam W the attribute type of the other VertexSet * * @param other the other VertexSet with which to join. + * @param f the function mapping a vertex id and its attributes in + * this and the other vertex set to a new vertex attribute. * @return a VertexSetRDD containing only the vertices in both this * and the other VertexSet and with tuple attributes. * @@ -287,13 +276,16 @@ class VertexSetRDD[@specialized V: ClassManifest]( /** - * @todo document + * Inner join this VertexSet with another VertexSet which has the + * same Index. This function will fail if both VertexSets do not + * share the same index. * - * @param other - * @param f - * @tparam W - * @tparam Z - * @return + * @param other the vertex set to join with this vertex set + * @param f the function mapping a vertex id and its attributes in + * this and the other vertex set to a collection of tuples. + * @tparam W the type of the other vertex set attributes + * @tparam Z the type of the tuples emitted by `f` + * @return an RDD containing the tuples emitted by `f` */ def zipJoinFlatMap[W: ClassManifest, Z: ClassManifest](other: VertexSetRDD[W])(f: (Vid, V,W) => Iterator[Z]): RDD[Z] = { @@ -316,8 +308,6 @@ class VertexSetRDD[@specialized V: ClassManifest]( /** - * @todo update docs to reflect function argument - * Left join this VertexSet with another VertexSet which has the * same Index. This function will fail if both VertexSets do not * share the same index. The resulting vertex set contains an entry @@ -327,6 +317,8 @@ class VertexSetRDD[@specialized V: ClassManifest]( * @tparam W the attribute type of the other VertexSet * * @param other the other VertexSet with which to join. + * @param f the function mapping a vertex id and its attributes in + * this and the other vertex set to a new vertex attribute. * @return a VertexSetRDD containing all the vertices in this * VertexSet with `None` attributes used for Vertices missing in the * other VertexSet. @@ -368,11 +360,12 @@ class VertexSetRDD[@specialized V: ClassManifest]( * @tparam W the attribute type of the other VertexSet * * @param other the other VertexSet with which to join. + * @param f the function mapping a vertex id and its attributes in + * this and the other vertex set to a new vertex attribute. * @param merge the function used combine duplicate vertex * attributes * @return a VertexSetRDD containing all the vertices in this - * VertexSet with `None` attributes used for Vertices missing in the - * other VertexSet. + * VertexSet with the attribute emitted by f. * */ def leftJoin[W: ClassManifest, Z: ClassManifest](other: RDD[(Vid,W)]) @@ -396,181 +389,9 @@ class VertexSetRDD[@specialized V: ClassManifest]( } } // end of leftJoin - - - /** - * For each key k in `this` or `other`, return a resulting RDD that contains a - * tuple with the list of values for that key in `this` as well as `other`. - */ - /* - def cogroup[W: ClassManifest](other: RDD[(Vid, W)], partitioner: Partitioner): - VertexSetRDD[(Seq[V], Seq[W])] = { - //RDD[(K, (Seq[V], Seq[W]))] = { - other match { - case other: VertexSetRDD[_] if index == other.index => { - // if both RDDs share exactly the same index and therefore the same - // super set of keys then we simply merge the value RDDs. - // However it is possible that both RDDs are missing a value for a given key in - // which case the returned RDD should have a null value - val newValues: RDD[(IndexedSeq[(Seq[V], Seq[W])], BitSet)] = - valuesRDD.zipPartitions(other.valuesRDD){ - (thisIter, otherIter) => - val (thisValues, thisBS) = thisIter.next() - assert(!thisIter.hasNext) - val (otherValues, otherBS) = otherIter.next() - assert(!otherIter.hasNext) - /** - * @todo consider implementing this with a view as in leftJoin to - * reduce array allocations - */ - val newValues = new Array[(Seq[V], Seq[W])](thisValues.size) - val newBS = thisBS | otherBS - - var ind = newBS.nextSetBit(0) - while(ind >= 0) { - val a = if (thisBS.get(ind)) Seq(thisValues(ind)) else Seq.empty[V] - val b = if (otherBS.get(ind)) Seq(otherValues(ind)) else Seq.empty[W] - newValues(ind) = (a, b) - ind = newBS.nextSetBit(ind+1) - } - Iterator((newValues.toIndexedSeq, newBS)) - } - new VertexSetRDD(index, newValues) - } - case other: VertexSetRDD[_] - if index.rdd.partitioner == other.index.rdd.partitioner => { - // If both RDDs are indexed using different indices but with the same partitioners - // then we we need to first merge the indicies and then use the merged index to - // merge the values. - val newIndex = - index.rdd.zipPartitions(other.index.rdd)( - (thisIter, otherIter) => { - val thisIndex = thisIter.next() - assert(!thisIter.hasNext) - val otherIndex = otherIter.next() - assert(!otherIter.hasNext) - // Merge the keys - val newIndex = new VertexIdToIndexMap(thisIndex.capacity + otherIndex.capacity) - var ind = thisIndex.nextPos(0) - while(ind >= 0) { - newIndex.fastAdd(thisIndex.getValue(ind)) - ind = thisIndex.nextPos(ind+1) - } - var ind = otherIndex.nextPos(0) - while(ind >= 0) { - newIndex.fastAdd(otherIndex.getValue(ind)) - ind = otherIndex.nextPos(ind+1) - } - List(newIndex).iterator - }).cache() - // Use the new index along with the this and the other indices to merge the values - val newValues: RDD[(IndexedSeq[(Seq[V], Seq[W])], BitSet)] = - newIndex.zipPartitions(tuples, other.tuples)( - (newIndexIter, thisTuplesIter, otherTuplesIter) => { - // Get the new index for this partition - val newIndex = newIndexIter.next() - assert(!newIndexIter.hasNext) - // Get the corresponding indicies and values for this and the other VertexSetRDD - val (thisIndex, (thisValues, thisBS)) = thisTuplesIter.next() - assert(!thisTuplesIter.hasNext) - val (otherIndex, (otherValues, otherBS)) = otherTuplesIter.next() - assert(!otherTuplesIter.hasNext) - // Preallocate the new Values array - val newValues = new Array[(Seq[V], Seq[W])](newIndex.size) - val newBS = new BitSet(newIndex.size) - - // Lookup the sequences in both submaps - for ((k,ind) <- newIndex) { - // Get the left key - val a = if (thisIndex.contains(k)) { - val ind = thisIndex.get(k) - if(thisBS.get(ind)) Seq(thisValues(ind)) else Seq.empty[V] - } else Seq.empty[V] - // Get the right key - val b = if (otherIndex.contains(k)) { - val ind = otherIndex.get(k) - if (otherBS.get(ind)) Seq(otherValues(ind)) else Seq.empty[W] - } else Seq.empty[W] - // If at least one key was present then we generate a tuple. - if (!a.isEmpty || !b.isEmpty) { - newValues(ind) = (a, b) - newBS.set(ind) - } - } - Iterator((newValues.toIndexedSeq, newBS)) - }) - new VertexSetRDD(new VertexSetIndex(newIndex), newValues) - } - case _ => { - // Get the partitioner from the index - val partitioner = index.rdd.partitioner match { - case Some(p) => p - case None => throw new SparkException("An index must have a partitioner.") - } - // Shuffle the other RDD using the partitioner for this index - val otherShuffled = - if (other.partitioner == Some(partitioner)) { - other - } else { - other.partitionBy(partitioner) - } - // Join the other RDD with this RDD building a new valueset and new index on the fly - val groups = tuples.zipPartitions(otherShuffled)( - (thisTuplesIter, otherTuplesIter) => { - // Get the corresponding indicies and values for this VertexSetRDD - val (thisIndex, (thisValues, thisBS)) = thisTuplesIter.next() - assert(!thisTuplesIter.hasNext()) - // Construct a new index - val newIndex = thisIndex.clone().asInstanceOf[VertexIdToIndexMap] - // Construct a new array Buffer to store the values - val newValues = ArrayBuffer.fill[ (Seq[V], Seq[W]) ](thisValues.size)(null) - val newBS = new BitSet(thisValues.size) - // populate the newValues with the values in this VertexSetRDD - for ((k,i) <- thisIndex) { - if (thisBS.get(i)) { - newValues(i) = (Seq(thisValues(i)), ArrayBuffer.empty[W]) - newBS.set(i) - } - } - // Now iterate through the other tuples updating the map - for ((k,w) <- otherTuplesIter){ - if (newIndex.contains(k)) { - val ind = newIndex.get(k) - if(newBS.get(ind)) { - newValues(ind)._2.asInstanceOf[ArrayBuffer[W]].append(w) - } else { - // If the other key was in the index but not in the values - // of this indexed RDD then create a new values entry for it - newBS.set(ind) - newValues(ind) = (Seq.empty[V], ArrayBuffer(w)) - } - } else { - // update the index - val ind = newIndex.size - newIndex.put(k, ind) - newBS.set(ind) - // Update the values - newValues.append( (Seq.empty[V], ArrayBuffer(w) ) ) - } - } - Iterator( (newIndex, (newValues.toIndexedSeq, newBS)) ) - }).cache() - - // Extract the index and values from the above RDD - val newIndex = groups.mapPartitions(_.map{ case (kMap,vAr) => kMap }, true) - val newValues: RDD[(IndexedSeq[(Seq[V], Seq[W])], BitSet)] = - groups.mapPartitions(_.map{ case (kMap,vAr) => vAr }, true) - - new VertexSetRDD[(Seq[V], Seq[W])](new VertexSetIndex(newIndex), newValues) - } - } - } // end of cogroup - */ - } // End of VertexSetRDD - /** * The VertexSetRDD singleton is used to construct VertexSets */ @@ -627,7 +448,6 @@ object VertexSetRDD { new VertexSetRDD[V](new VertexSetIndex(index), values) } // end of apply - /** * Construct a vertex set from an RDD using an existing index. * @@ -642,7 +462,6 @@ object VertexSetRDD { rdd: RDD[(Vid,V)], index: VertexSetIndex): VertexSetRDD[V] = apply(rdd, index, (a:V,b:V) => a) - /** * Construct a vertex set from an RDD using an existing index and a * user defined `combiner` to merge duplicate vertices. @@ -659,8 +478,17 @@ object VertexSetRDD { reduceFunc: (V, V) => V): VertexSetRDD[V] = apply(rdd,index, (v:V) => v, reduceFunc, reduceFunc) - - def aggregate[V: ClassManifest]( + /** + * Construct a vertex set from an RDD of AggregationMsgs + * + * @tparam V the vertex attribute type + * @param rdd the rdd containing vertices + * @param index the index which must be a superset of the vertices + * in RDD + * @param reduceFunc the user defined reduce function used to merge + * duplicate vertex attributes. + */ + private[spark] def aggregate[V: ClassManifest]( rdd: RDD[AggregationMsg[V]], index: VertexSetIndex, reduceFunc: (V, V) => V): VertexSetRDD[V] = { @@ -696,7 +524,6 @@ object VertexSetRDD { new VertexSetRDD(index, values) } - /** * Construct a vertex set from an RDD using an existing index and a * user defined `combiner` to merge duplicate vertices. @@ -767,7 +594,6 @@ object VertexSetRDD { new VertexSetRDD(index, values) } // end of apply - /** * Construct an index of the unique vertices. The resulting index * can be used to build VertexSets over subsets of the vertices in From 143c01dbd65af73d5630be8ea818850e9b59a9d4 Mon Sep 17 00:00:00 2001 From: Joey Date: Mon, 11 Nov 2013 19:37:16 -0800 Subject: [PATCH 197/531] Update README.md Changing image references to master branch. --- README.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index 7790139c8ffd9..5b06d82225229 100644 --- a/README.md +++ b/README.md @@ -27,7 +27,7 @@ the challenges of graph construction and transformation and provide limited fault-tolerance and support for interactive analysis.

- +

@@ -47,7 +47,7 @@ Finally, by exploiting the Scala foundation of Spark, we enable users to interactively load, transform, and compute on massive graphs.

- +

## Examples From d19f2e8f3e094a38b9f2d62c6334ad231fe6d74b Mon Sep 17 00:00:00 2001 From: Dan Crankshaw Date: Tue, 12 Nov 2013 05:21:34 +0000 Subject: [PATCH 198/531] Removed slaves from git --- conf/slaves | 2 -- 1 file changed, 2 deletions(-) delete mode 100644 conf/slaves diff --git a/conf/slaves b/conf/slaves deleted file mode 100644 index da0a01343d20a..0000000000000 --- a/conf/slaves +++ /dev/null @@ -1,2 +0,0 @@ -# A Spark Worker will be started on each of the machines listed below. -localhost \ No newline at end of file From 8d8056da14d3a7eac39f4cf14970467729350018 Mon Sep 17 00:00:00 2001 From: Dan Crankshaw Date: Wed, 30 Oct 2013 15:03:21 -0700 Subject: [PATCH 199/531] Fixed issue with canonical edge partitioner. --- .../scala/org/apache/spark/graph/Graph.scala | 1 + .../apache/spark/graph/impl/GraphImpl.scala | 23 ++++++++----------- 2 files changed, 10 insertions(+), 14 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/Graph.scala b/graph/src/main/scala/org/apache/spark/graph/Graph.scala index f5b4c57f72902..ddc8cbf4fbde6 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Graph.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Graph.scala @@ -1,6 +1,7 @@ package org.apache.spark.graph import org.apache.spark.rdd.RDD +import org.apache.spark.util.ClosureCleaner import org.apache.spark.storage.StorageLevel /** diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index d185f9f8cd7ba..693bb888bca26 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -253,8 +253,8 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( .toList // groups all ETs in this partition that have the same src and dst // Because all ETs with the same src and dst will live on the same - // partition due to the EdgePartitioner, this guarantees that these - // ET groups will be complete. + // partition due to the canonicalRandomVertexCut partitioner, this + // guarantees that these ET groups will be complete. .groupBy { t: EdgeTriplet[VD, ED] => (t.srcId, t.dstId) } .mapValues { ts: List[EdgeTriplet[VD, ED]] => f(ts.toIterator) } .toList @@ -357,7 +357,6 @@ object GraphImpl { // val part: Pid = edgePartitionFunction1D(e.srcId, e.dstId, numPartitions) // val part: Pid = edgePartitionFunction2D(e.srcId, e.dstId, numPartitions, ceilSqrt) val part: Pid = randomVertexCut(e.srcId, e.dstId, numPartitions) - //val part: Pid = canonicalEdgePartitionFunction2D(e.srcId, e.dstId, numPartitions, ceilSqrt) // Should we be using 3-tuple or an optimized class new MessageToPartition(part, (e.srcId, e.dstId, e.attr)) @@ -555,18 +554,14 @@ object GraphImpl { } /** - * @todo This will only partition edges to the upper diagonal - * of the 2D processor space. + * Assign edges to an arbitrary machine corresponding to a random vertex cut. This + * function ensures that edges of opposite direction between the same two vertices + * will end up on the same partition. */ - protected def canonicalEdgePartitionFunction2D(srcOrig: Vid, dstOrig: Vid, - numParts: Pid, ceilSqrtNumParts: Pid): Pid = { - val mixingPrime: Vid = 1125899906842597L - // Partitions by canonical edge direction - val src = math.min(srcOrig, dstOrig) - val dst = math.max(srcOrig, dstOrig) - val col: Pid = ((math.abs(src) * mixingPrime) % ceilSqrtNumParts).toInt - val row: Pid = ((math.abs(dst) * mixingPrime) % ceilSqrtNumParts).toInt - (col * ceilSqrtNumParts + row) % numParts + protected def canonicalRandomVertexCut(src: Vid, dst: Vid, numParts: Pid): Pid = { + val lower = math.min(src, dst) + val higher = math.max(src, dst) + math.abs((lower, higher).hashCode()) % numParts } private def accessesVertexAttr[VD: ClassManifest, ED: ClassManifest]( From 7c573a8b43ff8e54225eb8cb6f64d30628f2ca56 Mon Sep 17 00:00:00 2001 From: Dan Crankshaw Date: Sun, 10 Nov 2013 22:32:52 -0800 Subject: [PATCH 200/531] Added PartitionStrategy option --- .../org/apache/spark/graph/Analytics.scala | 10 ++++ .../spark/graph/GraphKryoRegistrator.scala | 3 ++ .../org/apache/spark/graph/GraphLoader.scala | 12 +++-- .../spark/graph/PartitionStrategy.scala | 42 +++++++++++++++++ .../apache/spark/graph/impl/GraphImpl.scala | 46 +++++++++++++++---- 5 files changed, 99 insertions(+), 14 deletions(-) create mode 100644 graph/src/main/scala/org/apache/spark/graph/PartitionStrategy.scala diff --git a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala index 6beaea07fa060..a6e808cdbe9ab 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala @@ -241,6 +241,7 @@ object Analytics extends Logging { var outFname = "" var numVPart = 4 var numEPart = 4 + var partitionStrategy: PartitionStrategy = RandomVertexCut options.foreach{ case ("numIter", v) => numIter = v.toInt @@ -249,6 +250,15 @@ object Analytics extends Logging { case ("output", v) => outFname = v case ("numVPart", v) => numVPart = v.toInt case ("numEPart", v) => numEPart = v.toInt + case ("partStrategy", v) => { + v match { + case "RandomVertexCut" => partitionStrategy = RandomVertexCut + case "EdgePartition1D" => partitionStrategy = EdgePartition1D + case "EdgePartition2D" => partitionStrategy = EdgePartition2D + case "CanonicalRandomVertexCut" => partitionStrategy = CanonicalRandomVertexCut + case _ => throw new IllegalArgumentException("Invalid Partion Strategy: " + v) + } + } case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) } diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphKryoRegistrator.scala b/graph/src/main/scala/org/apache/spark/graph/GraphKryoRegistrator.scala index baf8099556c06..6f18e46ab2a54 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphKryoRegistrator.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphKryoRegistrator.scala @@ -5,6 +5,7 @@ import com.esotericsoftware.kryo.Kryo import org.apache.spark.graph.impl._ import org.apache.spark.serializer.KryoRegistrator import org.apache.spark.util.collection.BitSet +import org.apache.spark.graph._ class GraphKryoRegistrator extends KryoRegistrator { @@ -19,6 +20,8 @@ class GraphKryoRegistrator extends KryoRegistrator { kryo.register(classOf[BitSet]) kryo.register(classOf[VertexIdToIndexMap]) kryo.register(classOf[VertexAttributeBlock[Object]]) + kryo.register(classOf[PartitionStrategy]) + // This avoids a large number of hash table lookups. kryo.setReferences(false) } diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala b/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala index 313737fdbe1a9..f314083353afb 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala @@ -27,8 +27,8 @@ object GraphLoader { path: String, edgeParser: Array[String] => ED, minEdgePartitions: Int = 1, - minVertexPartitions: Int = 1) - : GraphImpl[Int, ED] = { + minVertexPartitions: Int = 1, + partitionStrategy: PartitionStrategy = RandomVertexCut): GraphImpl[Int, ED] = { // Parse the edge data table val edges = sc.textFile(path, minEdgePartitions).flatMap { line => @@ -48,13 +48,15 @@ object GraphLoader { } }.cache() - val graph = fromEdges(edges) + val graph = fromEdges(edges, partitionStrategy) graph } - private def fromEdges[ED: ClassManifest](edges: RDD[Edge[ED]]): GraphImpl[Int, ED] = { + private def fromEdges[ED: ClassManifest]( + edges: RDD[Edge[ED]], + partitionStrategy: PartitionStrategy): GraphImpl[Int, ED] = { val vertices = edges.flatMap { edge => List((edge.srcId, 1), (edge.dstId, 1)) } .reduceByKey(_ + _) - GraphImpl(vertices, edges, 0) + GraphImpl(vertices, edges, 0, (a: Int, b: Int) => a, partitionStrategy) } } diff --git a/graph/src/main/scala/org/apache/spark/graph/PartitionStrategy.scala b/graph/src/main/scala/org/apache/spark/graph/PartitionStrategy.scala new file mode 100644 index 0000000000000..caf96ad9ce4ad --- /dev/null +++ b/graph/src/main/scala/org/apache/spark/graph/PartitionStrategy.scala @@ -0,0 +1,42 @@ +package org.apache.spark.graph + +//import org.apache.spark.graph._ + + +sealed trait PartitionStrategy extends Serializable { def getPartition(src: Vid, dst: Vid, numParts: Pid): Pid} + +//case object EdgePartition2D extends PartitionStrategy { +object EdgePartition2D extends PartitionStrategy { + override def getPartition(src: Vid, dst: Vid, numParts: Pid): Pid = { + val ceilSqrtNumParts: Pid = math.ceil(math.sqrt(numParts)).toInt + val mixingPrime: Vid = 1125899906842597L + val col: Pid = ((math.abs(src) * mixingPrime) % ceilSqrtNumParts).toInt + val row: Pid = ((math.abs(dst) * mixingPrime) % ceilSqrtNumParts).toInt + (col * ceilSqrtNumParts + row) % numParts + } +} + + + +object EdgePartition1D extends PartitionStrategy { + override def getPartition(src: Vid, dst: Vid, numParts: Pid): Pid = { + val mixingPrime: Vid = 1125899906842597L + (math.abs(src) * mixingPrime).toInt % numParts + } +} + + +object RandomVertexCut extends PartitionStrategy { + override def getPartition(src: Vid, dst: Vid, numParts: Pid): Pid = { + math.abs((src, dst).hashCode()) % numParts + } +} + + +object CanonicalRandomVertexCut extends PartitionStrategy { + override def getPartition(src: Vid, dst: Vid, numParts: Pid): Pid = { + val lower = math.min(src, dst) + val higher = math.max(src, dst) + math.abs((lower, higher).hashCode()) % numParts + } +} diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index 693bb888bca26..b529a6964ef3f 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -308,10 +308,18 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( object GraphImpl { def apply[VD: ClassManifest, ED: ClassManifest]( - vertices: RDD[(Vid, VD)], edges: RDD[Edge[ED]], - defaultVertexAttr: VD): - GraphImpl[VD,ED] = { - apply(vertices, edges, defaultVertexAttr, (a:VD, b:VD) => a) + vertices: RDD[(Vid, VD)], + edges: RDD[Edge[ED]], + defaultVertexAttr: VD): GraphImpl[VD,ED] = { + apply(vertices, edges, defaultVertexAttr, (a:VD, b:VD) => a, RandomVertexCut) + } + + def apply[VD: ClassManifest, ED: ClassManifest]( + vertices: RDD[(Vid, VD)], + edges: RDD[Edge[ED]], + defaultVertexAttr: VD, + partitionStrategy: PartitionStrategy): GraphImpl[VD,ED] = { + apply(vertices, edges, defaultVertexAttr, (a:VD, b:VD) => a, partitionStrategy) } def apply[VD: ClassManifest, ED: ClassManifest]( @@ -319,6 +327,15 @@ object GraphImpl { edges: RDD[Edge[ED]], defaultVertexAttr: VD, mergeFunc: (VD, VD) => VD): GraphImpl[VD,ED] = { + apply(vertices, edges, defaultVertexAttr, mergeFunc, RandomVertexCut) + } + + def apply[VD: ClassManifest, ED: ClassManifest]( + vertices: RDD[(Vid, VD)], + edges: RDD[Edge[ED]], + defaultVertexAttr: VD, + mergeFunc: (VD, VD) => VD, + partitionStrategy: PartitionStrategy): GraphImpl[VD,ED] = { val vtable = VertexSetRDD(vertices, mergeFunc) /** @@ -339,6 +356,14 @@ object GraphImpl { new GraphImpl(vtable, vid2pid, localVidMap, etable) } + + + + protected def createETable[ED: ClassManifest](edges: RDD[Edge[ED]]) + : RDD[(Pid, EdgePartition[ED])] = { + createETable(edges, RandomVertexCut) + } + /** * Create the edge table RDD, which is much more efficient for Java heap storage than the * normal edges data structure (RDD[(Vid, Vid, ED)]). @@ -347,16 +372,18 @@ object GraphImpl { * key-value pair: the key is the partition id, and the value is an EdgePartition object * containing all the edges in a partition. */ - protected def createETable[ED: ClassManifest](edges: RDD[Edge[ED]]) - : RDD[(Pid, EdgePartition[ED])] = { + protected def createETable[ED: ClassManifest]( + edges: RDD[Edge[ED]], + partitionStrategy: PartitionStrategy): RDD[(Pid, EdgePartition[ED])] = { // Get the number of partitions val numPartitions = edges.partitions.size - val ceilSqrt: Pid = math.ceil(math.sqrt(numPartitions)).toInt + edges.map { e => // Random partitioning based on the source vertex id. // val part: Pid = edgePartitionFunction1D(e.srcId, e.dstId, numPartitions) // val part: Pid = edgePartitionFunction2D(e.srcId, e.dstId, numPartitions, ceilSqrt) - val part: Pid = randomVertexCut(e.srcId, e.dstId, numPartitions) + //val part: Pid = randomVertexCut(e.srcId, e.dstId, numPartitions) + val part: Pid = partitionStrategy.getPartition(e.srcId, e.dstId, numPartitions) // Should we be using 3-tuple or an optimized class new MessageToPartition(part, (e.srcId, e.dstId, e.attr)) @@ -538,7 +565,8 @@ object GraphImpl { * */ protected def edgePartitionFunction2D(src: Vid, dst: Vid, - numParts: Pid, ceilSqrtNumParts: Pid): Pid = { + numParts: Pid): Pid = { + val ceilSqrtNumParts: Pid = math.ceil(math.sqrt(numParts)).toInt val mixingPrime: Vid = 1125899906842597L val col: Pid = ((math.abs(src) * mixingPrime) % ceilSqrtNumParts).toInt val row: Pid = ((math.abs(dst) * mixingPrime) % ceilSqrtNumParts).toInt From a13460bb64fd48fddfabbfb548fd9497d2b7d01e Mon Sep 17 00:00:00 2001 From: Dan Crankshaw Date: Mon, 11 Nov 2013 02:19:07 -0800 Subject: [PATCH 201/531] Updated documentation --- .../spark/graph/PartitionStrategy.scala | 56 ++++++++++++++++++- 1 file changed, 54 insertions(+), 2 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/PartitionStrategy.scala b/graph/src/main/scala/org/apache/spark/graph/PartitionStrategy.scala index caf96ad9ce4ad..5494ab73f4630 100644 --- a/graph/src/main/scala/org/apache/spark/graph/PartitionStrategy.scala +++ b/graph/src/main/scala/org/apache/spark/graph/PartitionStrategy.scala @@ -5,7 +5,51 @@ package org.apache.spark.graph sealed trait PartitionStrategy extends Serializable { def getPartition(src: Vid, dst: Vid, numParts: Pid): Pid} -//case object EdgePartition2D extends PartitionStrategy { + +/** + * This function implements a classic 2D-Partitioning of a sparse matrix. + * Suppose we have a graph with 11 vertices that we want to partition + * over 9 machines. We can use the following sparse matrix representation: + * + * __________________________________ + * v0 | P0 * | P1 | P2 * | + * v1 | **** | * | | + * v2 | ******* | ** | **** | + * v3 | ***** | * * | * | + * ---------------------------------- + * v4 | P3 * | P4 *** | P5 ** * | + * v5 | * * | * | | + * v6 | * | ** | **** | + * v7 | * * * | * * | * | + * ---------------------------------- + * v8 | P6 * | P7 * | P8 * *| + * v9 | * | * * | | + * v10 | * | ** | * * | + * v11 | * <-E | *** | ** | + * ---------------------------------- + * + * 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(numProc) by sqrt(numProc) blocks. Notice that edges + * adjacent to v11 can only be in the first colum 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(numProc) 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 effectively 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. + * + * + */ object EdgePartition2D extends PartitionStrategy { override def getPartition(src: Vid, dst: Vid, numParts: Pid): Pid = { val ceilSqrtNumParts: Pid = math.ceil(math.sqrt(numParts)).toInt @@ -17,7 +61,6 @@ object EdgePartition2D extends PartitionStrategy { } - object EdgePartition1D extends PartitionStrategy { override def getPartition(src: Vid, dst: Vid, numParts: Pid): Pid = { val mixingPrime: Vid = 1125899906842597L @@ -26,6 +69,10 @@ object EdgePartition1D extends PartitionStrategy { } +/** + * Assign edges to an aribtrary machine corresponding to a + * random vertex cut. + */ object RandomVertexCut extends PartitionStrategy { override def getPartition(src: Vid, dst: Vid, numParts: Pid): Pid = { math.abs((src, dst).hashCode()) % numParts @@ -33,6 +80,11 @@ object RandomVertexCut extends PartitionStrategy { } +/** + * Assign edges to an arbitrary machine corresponding to a random vertex cut. This + * function ensures that edges of opposite direction between the same two vertices + * will end up on the same partition. + */ object CanonicalRandomVertexCut extends PartitionStrategy { override def getPartition(src: Vid, dst: Vid, numParts: Pid): Pid = { val lower = math.min(src, dst) From 1e5c17812de073dd7fe890b99f921489895dae59 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 12 Nov 2013 15:30:27 -0800 Subject: [PATCH 202/531] Use variable encoding for ints, longs, and doubles in the specialized serializers. --- .../org/apache/spark/graph/Analytics.scala | 3 + .../apache/spark/graph/impl/Serializers.scala | 66 +++++++++++++++++-- 2 files changed, 63 insertions(+), 6 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala index 6beaea07fa060..8320b663a559d 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala @@ -284,6 +284,9 @@ object Analytics extends Logging { pr.vertices.map{case (id, r) => id + "\t" + r}.saveAsTextFile(outFname) } logInfo("GRAPHX: Runtime: " + ((System.currentTimeMillis - startTime)/1000.0) + " seconds") + + + Thread.sleep(1000000) sc.stop() } diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala b/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala index c56bbc8aee977..8c366f5fe0b5c 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala @@ -128,7 +128,7 @@ class DoubleAggMsgSerializer extends Serializer { override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { def writeObject[T](t: T) = { val msg = t.asInstanceOf[AggregationMsg[Double]] - writeLong(msg.vid) + writeUnsignedVarLong(msg.vid) writeDouble(msg.data) this } @@ -136,7 +136,7 @@ class DoubleAggMsgSerializer extends Serializer { override def deserializeStream(s: InputStream) = new ShuffleDeserializationStream(s) { def readObject[T](): T = { - val a = readLong() + val a = readUnsignedVarLong() val b = readDouble() new AggregationMsg[Double](a, b).asInstanceOf[T] } @@ -159,6 +159,24 @@ sealed abstract class ShuffleSerializationStream(s: OutputStream) extends Serial s.write(v) } + def writeUnsignedVarInt(value: Int) { + var v = value + while ((v & 0xFFFFFF80) != 0L) { + s.write((v & 0x7F) | 0x80) + v = v >>> 7 + } + s.write(v & 0x7F) + } + + def writeUnsignedVarLong(value: Long) { + var v = value + while ((v & 0xFFFFFF80) != 0L) { + s.write(((v & 0x7F) | 0x80).toInt) + v = v >>> 7 + } + s.write((v & 0x7F).toInt) + } + def writeLong(v: Long) { s.write((v >>> 56).toInt) s.write((v >>> 48).toInt) @@ -170,9 +188,8 @@ sealed abstract class ShuffleSerializationStream(s: OutputStream) extends Serial s.write(v.toInt) } - def writeDouble(v: Double) { - writeLong(java.lang.Double.doubleToLongBits(v)) - } + def writeDouble(v: Double): Unit = writeUnsignedVarLong(java.lang.Double.doubleToLongBits(v)) + //def writeDouble(v: Double): Unit = writeLong(java.lang.Double.doubleToLongBits(v)) override def flush(): Unit = s.flush() @@ -190,6 +207,42 @@ sealed abstract class ShuffleDeserializationStream(s: InputStream) extends Deser (first & 0xFF) << 24 | (s.read() & 0xFF) << 16 | (s.read() & 0xFF) << 8 | (s.read() & 0xFF) } + def readUnsignedVarInt(): Int = { + var value: Int = 0 + var i: Int = 0 + def readOrThrow(): Int = { + val in = s.read() + if (in < 0) throw new java.io.EOFException + in & 0xFF + } + var b: Int = readOrThrow() + while ((b & 0x80) != 0) { + value |= (b & 0x7F) << i + i += 7 + if (i > 35) throw new IllegalArgumentException("Variable length quantity is too long") + b = readOrThrow() + } + value | (b << i) + } + + def readUnsignedVarLong(): Long = { + var value: Long = 0L + var i: Int = 0 + def readOrThrow(): Int = { + val in = s.read() + if (in < 0) throw new java.io.EOFException + in & 0xFF + } + var b: Int = readOrThrow() + while ((b & 0x80) != 0) { + value |= (b & 0x7F) << i + i += 7 + if (i > 63) throw new IllegalArgumentException("Variable length quantity is too long") + b = readOrThrow() + } + value | (b << i) + } + def readLong(): Long = { val first = s.read() if (first < 0) throw new EOFException() @@ -203,7 +256,8 @@ sealed abstract class ShuffleDeserializationStream(s: InputStream) extends Deser (s.read() & 0xFF) } - def readDouble(): Double = java.lang.Double.longBitsToDouble(readLong()) + def readDouble(): Double = java.lang.Double.longBitsToDouble(readUnsignedVarLong()) + //def readDouble(): Double = java.lang.Double.longBitsToDouble(readLong()) override def close(): Unit = s.close() } From 882d069189f26bef344c318db9ec6cdc1c7b66f7 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 12 Nov 2013 18:50:03 -0800 Subject: [PATCH 203/531] Fixed the bug in variable encoding for longs. --- .../apache/spark/graph/impl/Serializers.scala | 117 ++++++++++++++---- .../apache/spark/graph/SerializerSuite.scala | 36 +++--- 2 files changed, 110 insertions(+), 43 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala b/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala index 8c366f5fe0b5c..2e768e85cfb47 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala @@ -82,7 +82,7 @@ class IntAggMsgSerializer extends Serializer { def writeObject[T](t: T) = { val msg = t.asInstanceOf[AggregationMsg[Int]] writeLong(msg.vid) - writeInt(msg.data) + writeUnsignedVarInt(msg.data) this } } @@ -90,7 +90,7 @@ class IntAggMsgSerializer extends Serializer { override def deserializeStream(s: InputStream) = new ShuffleDeserializationStream(s) { override def readObject[T](): T = { val a = readLong() - val b = readInt() + val b = readUnsignedVarInt() new AggregationMsg[Int](a, b).asInstanceOf[T] } } @@ -104,16 +104,16 @@ class LongAggMsgSerializer extends Serializer { override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { def writeObject[T](t: T) = { val msg = t.asInstanceOf[AggregationMsg[Long]] - writeLong(msg.vid) - writeLong(msg.data) + writeVarLong(msg.vid, optimizePositive = false) + writeVarLong(msg.data, optimizePositive = true) this } } override def deserializeStream(s: InputStream) = new ShuffleDeserializationStream(s) { override def readObject[T](): T = { - val a = readLong() - val b = readLong() + val a = readVarLong(optimizePositive = false) + val b = readVarLong(optimizePositive = true) new AggregationMsg[Long](a, b).asInstanceOf[T] } } @@ -128,7 +128,7 @@ class DoubleAggMsgSerializer extends Serializer { override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { def writeObject[T](t: T) = { val msg = t.asInstanceOf[AggregationMsg[Double]] - writeUnsignedVarLong(msg.vid) + writeVarLong(msg.vid, optimizePositive = false) writeDouble(msg.data) this } @@ -136,7 +136,7 @@ class DoubleAggMsgSerializer extends Serializer { override def deserializeStream(s: InputStream) = new ShuffleDeserializationStream(s) { def readObject[T](): T = { - val a = readUnsignedVarLong() + val a = readVarLong(optimizePositive = false) val b = readDouble() new AggregationMsg[Double](a, b).asInstanceOf[T] } @@ -160,21 +160,86 @@ sealed abstract class ShuffleSerializationStream(s: OutputStream) extends Serial } def writeUnsignedVarInt(value: Int) { - var v = value - while ((v & 0xFFFFFF80) != 0L) { - s.write((v & 0x7F) | 0x80) - v = v >>> 7 + if ((value >>> 7) == 0) { + s.write(value.toInt) + } else if ((value >>> 14) == 0) { + s.write((value & 0x7F) | 0x80) + s.write(value >>> 7) + } else if ((value >>> 21) == 0) { + s.write((value & 0x7F) | 0x80) + s.write(value >>> 7 | 0x80) + s.write(value >>> 14) + } else if ((value >>> 28) == 0) { + s.write((value & 0x7F) | 0x80) + s.write(value >>> 7 | 0x80) + s.write(value >>> 14 | 0x80) + s.write(value >>> 21) + } else { + s.write((value & 0x7F) | 0x80) + s.write(value >>> 7 | 0x80) + s.write(value >>> 14 | 0x80) + s.write(value >>> 21 | 0x80) + s.write(value >>> 28) } - s.write(v & 0x7F) } - def writeUnsignedVarLong(value: Long) { - var v = value - while ((v & 0xFFFFFF80) != 0L) { + def writeVarLong(value: Long, optimizePositive: Boolean) { + val v = if (!optimizePositive) (value << 1) ^ (value >> 63) else value + if ((v >>> 7) == 0) { + s.write(v.toInt) + } else if ((v >>> 14) == 0) { + s.write(((v & 0x7F) | 0x80).toInt) + s.write((v >>> 7).toInt) + } else if ((v >>> 21) == 0) { + s.write(((v & 0x7F) | 0x80).toInt) + s.write((v >>> 7 | 0x80).toInt) + s.write((v >>> 14).toInt) + } else if ((v >>> 28) == 0) { s.write(((v & 0x7F) | 0x80).toInt) - v = v >>> 7 + s.write((v >>> 7 | 0x80).toInt) + s.write((v >>> 14 | 0x80).toInt) + s.write((v >>> 21).toInt) + } else if ((v >>> 35) == 0) { + s.write(((v & 0x7F) | 0x80).toInt) + s.write((v >>> 7 | 0x80).toInt) + s.write((v >>> 14 | 0x80).toInt) + s.write((v >>> 21 | 0x80).toInt) + s.write((v >>> 28).toInt) + } else if ((v >>> 42) == 0) { + s.write(((v & 0x7F) | 0x80).toInt) + s.write((v >>> 7 | 0x80).toInt) + s.write((v >>> 14 | 0x80).toInt) + s.write((v >>> 21 | 0x80).toInt) + s.write((v >>> 28 | 0x80).toInt) + s.write((v >>> 35).toInt) + } else if ((v >>> 49) == 0) { + s.write(((v & 0x7F) | 0x80).toInt) + s.write((v >>> 7 | 0x80).toInt) + s.write((v >>> 14 | 0x80).toInt) + s.write((v >>> 21 | 0x80).toInt) + s.write((v >>> 28 | 0x80).toInt) + s.write((v >>> 35 | 0x80).toInt) + s.write((v >>> 42).toInt) + } else if ((v >>> 56) == 0) { + s.write(((v & 0x7F) | 0x80).toInt) + s.write((v >>> 7 | 0x80).toInt) + s.write((v >>> 14 | 0x80).toInt) + s.write((v >>> 21 | 0x80).toInt) + s.write((v >>> 28 | 0x80).toInt) + s.write((v >>> 35 | 0x80).toInt) + s.write((v >>> 42 | 0x80).toInt) + s.write((v >>> 49).toInt) + } else { + s.write(((v & 0x7F) | 0x80).toInt) + s.write((v >>> 7 | 0x80).toInt) + s.write((v >>> 14 | 0x80).toInt) + s.write((v >>> 21 | 0x80).toInt) + s.write((v >>> 28 | 0x80).toInt) + s.write((v >>> 35 | 0x80).toInt) + s.write((v >>> 42 | 0x80).toInt) + s.write((v >>> 49 | 0x80).toInt) + s.write((v >>> 56).toInt) } - s.write((v & 0x7F).toInt) } def writeLong(v: Long) { @@ -188,8 +253,8 @@ sealed abstract class ShuffleSerializationStream(s: OutputStream) extends Serial s.write(v.toInt) } - def writeDouble(v: Double): Unit = writeUnsignedVarLong(java.lang.Double.doubleToLongBits(v)) - //def writeDouble(v: Double): Unit = writeLong(java.lang.Double.doubleToLongBits(v)) + //def writeDouble(v: Double): Unit = writeUnsignedVarLong(java.lang.Double.doubleToLongBits(v)) + def writeDouble(v: Double): Unit = writeLong(java.lang.Double.doubleToLongBits(v)) override def flush(): Unit = s.flush() @@ -225,7 +290,8 @@ sealed abstract class ShuffleDeserializationStream(s: InputStream) extends Deser value | (b << i) } - def readUnsignedVarLong(): Long = { + def readVarLong(optimizePositive: Boolean): Long = { + // TODO: unroll the while loop. var value: Long = 0L var i: Int = 0 def readOrThrow(): Int = { @@ -235,12 +301,13 @@ sealed abstract class ShuffleDeserializationStream(s: InputStream) extends Deser } var b: Int = readOrThrow() while ((b & 0x80) != 0) { - value |= (b & 0x7F) << i + value |= (b & 0x7F).toLong << i i += 7 if (i > 63) throw new IllegalArgumentException("Variable length quantity is too long") b = readOrThrow() } - value | (b << i) + val ret = value | (b.toLong << i) + if (!optimizePositive) (ret >>> 1) ^ -(ret & 1) else ret } def readLong(): Long = { @@ -256,8 +323,8 @@ sealed abstract class ShuffleDeserializationStream(s: InputStream) extends Deser (s.read() & 0xFF) } - def readDouble(): Double = java.lang.Double.longBitsToDouble(readUnsignedVarLong()) - //def readDouble(): Double = java.lang.Double.longBitsToDouble(readLong()) + //def readDouble(): Double = java.lang.Double.longBitsToDouble(readUnsignedVarLong()) + def readDouble(): Double = java.lang.Double.longBitsToDouble(readLong()) override def close(): Unit = s.close() } diff --git a/graph/src/test/scala/org/apache/spark/graph/SerializerSuite.scala b/graph/src/test/scala/org/apache/spark/graph/SerializerSuite.scala index 0d55cc01896cd..6295f866b8c48 100644 --- a/graph/src/test/scala/org/apache/spark/graph/SerializerSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/SerializerSuite.scala @@ -15,13 +15,13 @@ class SerializerSuite extends FunSuite with LocalSparkContext { System.setProperty("spark.serializer", "org.apache.spark.serializer.KryoSerializer") System.setProperty("spark.kryo.registrator", "org.apache.spark.graph.GraphKryoRegistrator") - test("TestVertexBroadcastMessageInt") { - val outMsg = new VertexBroadcastMsg[Int](3,4,5) + test("IntVertexBroadcastMsgSerializer") { + val outMsg = new VertexBroadcastMsg[Int](3, 4, 5) val bout = new ByteArrayOutputStream val outStrm = new IntVertexBroadcastMsgSerializer().newInstance().serializeStream(bout) outStrm.writeObject(outMsg) outStrm.writeObject(outMsg) - bout.flush + bout.flush() val bin = new ByteArrayInputStream(bout.toByteArray) val inStrm = new IntVertexBroadcastMsgSerializer().newInstance().deserializeStream(bin) val inMsg1: VertexBroadcastMsg[Int] = inStrm.readObject() @@ -36,13 +36,13 @@ class SerializerSuite extends FunSuite with LocalSparkContext { } } - test("TestVertexBroadcastMessageLong") { - val outMsg = new VertexBroadcastMsg[Long](3,4,5) + test("LongVertexBroadcastMsgSerializer") { + val outMsg = new VertexBroadcastMsg[Long](3, 4, 5) val bout = new ByteArrayOutputStream val outStrm = new LongVertexBroadcastMsgSerializer().newInstance().serializeStream(bout) outStrm.writeObject(outMsg) outStrm.writeObject(outMsg) - bout.flush + bout.flush() val bin = new ByteArrayInputStream(bout.toByteArray) val inStrm = new LongVertexBroadcastMsgSerializer().newInstance().deserializeStream(bin) val inMsg1: VertexBroadcastMsg[Long] = inStrm.readObject() @@ -57,13 +57,13 @@ class SerializerSuite extends FunSuite with LocalSparkContext { } } - test("TestVertexBroadcastMessageDouble") { - val outMsg = new VertexBroadcastMsg[Double](3,4,5.0) + test("DoubleVertexBroadcastMsgSerializer") { + val outMsg = new VertexBroadcastMsg[Double](3, 4, 5.0) val bout = new ByteArrayOutputStream val outStrm = new DoubleVertexBroadcastMsgSerializer().newInstance().serializeStream(bout) outStrm.writeObject(outMsg) outStrm.writeObject(outMsg) - bout.flush + bout.flush() val bin = new ByteArrayInputStream(bout.toByteArray) val inStrm = new DoubleVertexBroadcastMsgSerializer().newInstance().deserializeStream(bin) val inMsg1: VertexBroadcastMsg[Double] = inStrm.readObject() @@ -78,13 +78,13 @@ class SerializerSuite extends FunSuite with LocalSparkContext { } } - test("TestAggregationMessageInt") { - val outMsg = new AggregationMsg[Int](4,5) + test("IntAggMsgSerializer") { + val outMsg = new AggregationMsg[Int](4, 5) val bout = new ByteArrayOutputStream val outStrm = new IntAggMsgSerializer().newInstance().serializeStream(bout) outStrm.writeObject(outMsg) outStrm.writeObject(outMsg) - bout.flush + bout.flush() val bin = new ByteArrayInputStream(bout.toByteArray) val inStrm = new IntAggMsgSerializer().newInstance().deserializeStream(bin) val inMsg1: AggregationMsg[Int] = inStrm.readObject() @@ -99,13 +99,13 @@ class SerializerSuite extends FunSuite with LocalSparkContext { } } - test("TestAggregationMessageLong") { - val outMsg = new AggregationMsg[Long](4,5) + test("LongAggMsgSerializer") { + val outMsg = new AggregationMsg[Long](4, 1L << 32) val bout = new ByteArrayOutputStream val outStrm = new LongAggMsgSerializer().newInstance().serializeStream(bout) outStrm.writeObject(outMsg) outStrm.writeObject(outMsg) - bout.flush + bout.flush() val bin = new ByteArrayInputStream(bout.toByteArray) val inStrm = new LongAggMsgSerializer().newInstance().deserializeStream(bin) val inMsg1: AggregationMsg[Long] = inStrm.readObject() @@ -120,13 +120,13 @@ class SerializerSuite extends FunSuite with LocalSparkContext { } } - test("TestAggregationMessageDouble") { - val outMsg = new AggregationMsg[Double](4,5.0) + test("DoubleAggMsgSerializer") { + val outMsg = new AggregationMsg[Double](4, 5.0) val bout = new ByteArrayOutputStream val outStrm = new DoubleAggMsgSerializer().newInstance().serializeStream(bout) outStrm.writeObject(outMsg) outStrm.writeObject(outMsg) - bout.flush + bout.flush() val bin = new ByteArrayInputStream(bout.toByteArray) val inStrm = new DoubleAggMsgSerializer().newInstance().deserializeStream(bin) val inMsg1: AggregationMsg[Double] = inStrm.readObject() From f0ef75c7a41c417096398d55c08785d1884c4d85 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Wed, 13 Nov 2013 10:35:23 -0800 Subject: [PATCH 204/531] Addressing bug in BitSet.setUntil(ind) where if invoked with a multiple of 64 could lead to an index out of bounds error. --- .../scala/org/apache/spark/util/collection/BitSet.scala | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala b/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala index 5e264b48ddc22..1f794379f74f3 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala @@ -41,9 +41,11 @@ class BitSet(numBits: Int) { val wordIndex = bitIndex >> 6 // divide by 64 var i = 0 while(i < wordIndex) { words(i) = -1; i += 1 } - // Set the remaining bits - val mask = ~(-1L << (bitIndex & 0x3f)) - words(wordIndex) |= mask + if(wordIndex < words.size) { + // Set the remaining bits (note that the mask could still be zero) + val mask = ~(-1L << (bitIndex & 0x3f)) + words(wordIndex) |= mask + } } From 266eb01ce8a71a1d187575e3199546985aa7b967 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Wed, 13 Nov 2013 10:33:18 -0800 Subject: [PATCH 205/531] Addressing issue in Graph creation where a graph created with a vertex set that does not span all of the vertices in the edges will crash on triplet construction. --- .../scala/org/apache/spark/graph/Graph.scala | 24 ++++++++ .../org/apache/spark/graph/VertexSetRDD.scala | 56 ++++++++++++------- .../apache/spark/graph/impl/GraphImpl.scala | 10 +++- .../org/apache/spark/graph/GraphSuite.scala | 17 +++++- 4 files changed, 86 insertions(+), 21 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/Graph.scala b/graph/src/main/scala/org/apache/spark/graph/Graph.scala index f5b4c57f72902..87667f69586ef 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Graph.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Graph.scala @@ -404,6 +404,30 @@ object Graph { } + + /** + * Construct a graph from a collection attributed vertices and + * edges. Duplicate vertices are combined using the `mergeFunc` and + * vertices found in the edge collection but not in the input + * vertices are the default attribute `defautVertexAttr`. + * + * @tparam VD the vertex attribute type + * @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 mergeFunc the function used to merge duplicate vertices + * in the `vertices` collection. + * + */ + def apply[VD: ClassManifest, ED: ClassManifest]( + vertices: RDD[(Vid,VD)], + edges: RDD[Edge[ED]], + defaultVertexAttr: VD): Graph[VD, ED] = { + GraphImpl(vertices, edges, defaultVertexAttr, (a,b) => a) + } + /** * Construct a graph from a collection attributed vertices and * edges. Duplicate vertices are combined using the `mergeFunc` and diff --git a/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala b/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala index 401d5f29bc134..51c0e3a33e281 100644 --- a/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala @@ -204,6 +204,29 @@ class VertexSetRDD[@specialized V: ClassManifest]( new VertexSetRDD[U](index, newValuesRDD) } // end of mapValues + /** + * + * @param missingValue + * @return + */ + def fillMissing(missingValue: V): VertexSetRDD[V] = { + val newValuesRDD: RDD[ (Array[V], BitSet) ] = + valuesRDD.zipPartitions(index.rdd){ (valuesIter, indexIter) => + val index = indexIter.next + assert(!indexIter.hasNext) + val (values, bs: BitSet) = valuesIter.next + assert(!valuesIter.hasNext) + // Allocate a new values array with missing value as the default + val newValues = Array.fill(values.size)(missingValue) + // Copy over the old values + bs.iterator.foreach { ind => newValues(ind) = values(ind) } + // Create a new bitset matching the keyset + val newBS = index.getBitSet + Iterator((newValues, newBS)) + } + new VertexSetRDD[V](index, newValuesRDD) + } + /** * Pass each vertex attribute along with the vertex id through a map * function and retain the original RDD's partitioning and index. @@ -380,7 +403,6 @@ class VertexSetRDD[@specialized V: ClassManifest]( // this vertex set then we use the much more efficient leftZipJoin case other: VertexSetRDD[_] if index == other.index => { leftZipJoin(other)(cleanF) - // @todo handle case where other is a VertexSetRDD with a different index } case _ => { val indexedOther: VertexSetRDD[W] = VertexSetRDD(other, index, cleanMerge) @@ -599,28 +621,24 @@ object VertexSetRDD { * can be used to build VertexSets over subsets of the vertices in * the input. */ - def makeIndex(keys: RDD[Vid], - partitioner: Option[Partitioner] = None): VertexSetIndex = { - // @todo: I don't need the boolean its only there to be the second type since I want to shuffle a single RDD - // Ugly hack :-(. In order to partition the keys they must have values. - val tbl = keys.mapPartitions(_.map(k => (k, false)), true) - // Shuffle the table (if necessary) - val shuffledTbl = partitioner match { - case None => { - if (tbl.partitioner.isEmpty) { - // @todo: I don't need the boolean its only there to be the second type of the shuffle. - new ShuffledRDD[Vid, Boolean, (Vid, Boolean)](tbl, Partitioner.defaultPartitioner(tbl)) - } else { tbl } - } - case Some(partitioner) => - tbl.partitionBy(partitioner) + def makeIndex(keys: RDD[Vid], partitionerOpt: Option[Partitioner] = None): VertexSetIndex = { + val partitioner = partitionerOpt match { + case Some(p) => p + case None => Partitioner.defaultPartitioner(keys) } - val index = shuffledTbl.mapPartitions( iter => { + val preAgg: RDD[(Vid, Unit)] = keys.mapPartitions( iter => { + val keys = new VertexIdToIndexMap + while(iter.hasNext) { keys.add(iter.next) } + keys.iterator.map(k => (k, ())) + }, true).partitionBy(partitioner) + + val index = preAgg.mapPartitions( iter => { val index = new VertexIdToIndexMap - for ( (k,_) <- iter ){ index.add(k) } + while(iter.hasNext) { index.add(iter.next._1) } Iterator(index) - }, true).cache + }, true).cache + new VertexSetIndex(index) } diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index d185f9f8cd7ba..655e46ee63a1a 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -9,6 +9,7 @@ import org.apache.spark.SparkContext._ import org.apache.spark.HashPartitioner import org.apache.spark.util.ClosureCleaner +import org.apache.spark.Partitioner import org.apache.spark.graph._ import org.apache.spark.graph.impl.GraphImpl._ import org.apache.spark.graph.impl.MsgRDDFunctions._ @@ -320,7 +321,14 @@ object GraphImpl { defaultVertexAttr: VD, mergeFunc: (VD, VD) => VD): GraphImpl[VD,ED] = { - val vtable = VertexSetRDD(vertices, mergeFunc) + vertices.cache + edges.cache + // Get the set of all vids + val allVids = vertices.map(_._1).union(edges.flatMap(e => Seq(e.srcId, e.dstId))) + // Index the set of all vids + val index = VertexSetRDD.makeIndex(allVids, Some(Partitioner.defaultPartitioner(vertices))) + // Index the vertices and fill in missing attributes with the default + val vtable = VertexSetRDD(vertices, index, mergeFunc).fillMissing(defaultVertexAttr) /** * @todo Verify that there are no edges that contain vertices * that are not in vTable. This should probably be resolved: diff --git a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala index 9c22608554671..899048a17a6d2 100644 --- a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala @@ -4,7 +4,7 @@ import org.scalatest.FunSuite import org.apache.spark.SparkContext import org.apache.spark.graph.LocalSparkContext._ - +import org.apache.spark.rdd._ class GraphSuite extends FunSuite with LocalSparkContext { @@ -20,6 +20,21 @@ class GraphSuite extends FunSuite with LocalSparkContext { } } + test("Graph Creation with invalid vertices") { + withSpark(new SparkContext("local", "test")) { sc => + val rawEdges = (0L to 98L).zip((1L to 99L) :+ 0L) + val edges: RDD[Edge[Int]] = sc.parallelize(rawEdges).map { case (s, t) => Edge(s, t, 1) } + val vertices: RDD[(Vid, Boolean)] = sc.parallelize((0L until 10L).map(id => (id, true))) + val graph = Graph(vertices, edges, false) + assert( graph.edges.count() === rawEdges.size ) + assert( graph.vertices.count() === 100) + graph.triplets.map { et => + assert( (et.srcId < 10 && et.srcAttr) || (et.srcId >= 10 && !et.srcAttr) ) + assert( (et.dstId < 10 && et.dstAttr) || (et.dstId >= 10 && !et.dstAttr) ) + } + } + } + test("mapEdges") { withSpark(new SparkContext("local", "test")) { sc => val n = 3 From 5a9b07ead2b129e57d00c7b9b5aeefb98c05fcc8 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Wed, 13 Nov 2013 10:44:11 -0800 Subject: [PATCH 206/531] Fixing documentation --- .../scala/org/apache/spark/graph/VertexSetRDD.scala | 6 ++++-- .../org/apache/spark/graph/impl/GraphImpl.scala | 13 +------------ 2 files changed, 5 insertions(+), 14 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala b/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala index 51c0e3a33e281..507370539e31a 100644 --- a/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala @@ -205,9 +205,11 @@ class VertexSetRDD[@specialized V: ClassManifest]( } // end of mapValues /** + * Fill in missing values for all vertices in the index. * - * @param missingValue - * @return + * @param missingValue the value to be used for vertices in the + * index that don't currently have values. + * @return A VertexSetRDD with a value for all vertices. */ def fillMissing(missingValue: V): VertexSetRDD[V] = { val newValuesRDD: RDD[ (Array[V], BitSet) ] = diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index 655e46ee63a1a..ead072dcb8626 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -329,18 +329,7 @@ object GraphImpl { val index = VertexSetRDD.makeIndex(allVids, Some(Partitioner.defaultPartitioner(vertices))) // Index the vertices and fill in missing attributes with the default val vtable = VertexSetRDD(vertices, index, mergeFunc).fillMissing(defaultVertexAttr) - /** - * @todo Verify that there are no edges that contain vertices - * that are not in vTable. This should probably be resolved: - * - * edges.flatMap{ e => Array((e.srcId, null), (e.dstId, null)) } - * .cogroup(vertices).map{ - * case (vid, _, attr) => - * if (attr.isEmpty) (vid, defaultValue) - * else (vid, attr) - * } - * - */ + val etable = createETable(edges) val vid2pid = new Vid2Pid(etable, vtable.index) val localVidMap = createLocalVidMap(etable) From fa8a53619b91623c23b6939ca38aafbfc8ec14a0 Mon Sep 17 00:00:00 2001 From: Dan Crankshaw Date: Thu, 14 Nov 2013 00:17:00 +0000 Subject: [PATCH 207/531] Added conf file to gitignore. --- .gitignore | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.gitignore b/.gitignore index e1f64a113390a..57e26c4d37338 100644 --- a/.gitignore +++ b/.gitignore @@ -14,6 +14,8 @@ conf/java-opts conf/spark-env.sh conf/streaming-env.sh conf/log4j.properties +conf/slaves +conf/core-site.xml docs/_site docs/api target/ From 3558e8bda1e5f9d6a540058440028d1b396cccc4 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Wed, 13 Nov 2013 17:07:23 -0800 Subject: [PATCH 208/531] During graph creation, create eTable earlier --- .../org/apache/spark/graph/impl/GraphImpl.scala | 15 ++++++++++----- 1 file changed, 10 insertions(+), 5 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index ead072dcb8626..d16a81d203c69 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -322,15 +322,20 @@ object GraphImpl { mergeFunc: (VD, VD) => VD): GraphImpl[VD,ED] = { vertices.cache - edges.cache - // Get the set of all vids - val allVids = vertices.map(_._1).union(edges.flatMap(e => Seq(e.srcId, e.dstId))) + val etable = createETable(edges).cache + // Get the set of all vids, preserving partitions + val partitioner = Partitioner.defaultPartitioner(vertices) + val implicitVids = etable.flatMap { + case (pid, partition) => Array.concat(partition.srcIds, partition.dstIds) + }.map(vid => (vid, ())).partitionBy(partitioner) + val allVids = vertices.zipPartitions(implicitVids) { + (a, b) => a.map(_._1) ++ b.map(_._1) + } // Index the set of all vids - val index = VertexSetRDD.makeIndex(allVids, Some(Partitioner.defaultPartitioner(vertices))) + val index = VertexSetRDD.makeIndex(allVids, Some(partitioner)) // Index the vertices and fill in missing attributes with the default val vtable = VertexSetRDD(vertices, index, mergeFunc).fillMissing(defaultVertexAttr) - val etable = createETable(edges) val vid2pid = new Vid2Pid(etable, vtable.index) val localVidMap = createLocalVidMap(etable) new GraphImpl(vtable, vid2pid, localVidMap, etable) From 8a460e1811452f9f8cdbe579cd6ed85f6c0a18a1 Mon Sep 17 00:00:00 2001 From: Dan Crankshaw Date: Mon, 18 Nov 2013 19:32:03 -0800 Subject: [PATCH 209/531] Added partitioner to GraphImpl constructor args. --- .../apache/spark/graph/impl/GraphImpl.scala | 43 +++++++++---------- 1 file changed, 20 insertions(+), 23 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index a0e1ad598944a..485e49f95e0ca 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -77,10 +77,10 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( @transient val vid2pid: Vid2Pid, @transient val localVidMap: RDD[(Pid, VertexIdToIndexMap)], @transient val eTable: RDD[(Pid, EdgePartition[ED])], - @transient val partitionStrategy: PartitionStrategy = RandomVertexCut) + @transient val partitioner: PartitionStrategy) extends Graph[VD, ED] { - def this() = this(null, null, null, null) + def this() = this(null, null, null, null, null) @transient val vTableReplicatedValues: VTableReplicatedValues[VD] = new VTableReplicatedValues(vTable, vid2pid, localVidMap) @@ -97,7 +97,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( @transient override val triplets: RDD[EdgeTriplet[VD, ED]] = makeTriplets(localVidMap, vTableReplicatedValues.bothAttrs, eTable) - @transient private val partitioner: PartitionStrategy = partitionStrategy + //@transient private val partitioner: PartitionStrategy = partitionStrategy override def persist(newLevel: StorageLevel): Graph[VD, ED] = { eTable.persist(newLevel) @@ -192,18 +192,18 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( override def reverse: Graph[VD, ED] = { val newEtable = eTable.mapPartitions( _.map{ case (pid, epart) => (pid, epart.reverse) }, preservesPartitioning = true) - new GraphImpl(vTable, vid2pid, localVidMap, newEtable) + new GraphImpl(vTable, vid2pid, localVidMap, newEtable, partitioner) } override def mapVertices[VD2: ClassManifest](f: (Vid, VD) => VD2): Graph[VD2, ED] = { val newVTable = vTable.mapValuesWithKeys((vid, data) => f(vid, data)) - new GraphImpl(newVTable, vid2pid, localVidMap, eTable) + new GraphImpl(newVTable, vid2pid, localVidMap, eTable, partitioner) } override def mapEdges[ED2: ClassManifest](f: Edge[ED] => ED2): Graph[VD, ED2] = { val newETable = eTable.mapPartitions(_.map{ case (pid, epart) => (pid, epart.map(f)) }, preservesPartitioning = true) - new GraphImpl(vTable, vid2pid, localVidMap, newETable) + new GraphImpl(vTable, vid2pid, localVidMap, newETable, partitioner) } override def mapTriplets[ED2: ClassManifest](f: EdgeTriplet[VD, ED] => ED2): Graph[VD, ED2] = @@ -237,7 +237,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( triplets.filter( t => vpred( t.srcId, t.srcAttr ) && vpred( t.dstId, t.dstAttr ) && epred(t) ) - .map( t => Edge(t.srcId, t.dstId, t.attr) )) + .map( t => Edge(t.srcId, t.dstId, t.attr) ), partitioner) // Construct the Vid2Pid map. Here we assume that the filter operation // behaves deterministically. @@ -245,7 +245,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( val newVid2Pid = new Vid2Pid(newETable, newVTable.index) val newVidMap = createLocalVidMap(newETable) - new GraphImpl(newVTable, newVid2Pid, localVidMap, newETable) + new GraphImpl(newVTable, newVid2Pid, localVidMap, newETable, partitioner) } override def groupEdgeTriplets[ED2: ClassManifest]( @@ -268,8 +268,8 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( } //TODO(crankshaw) eliminate the need to call createETable - val newETable = createETable(newEdges) - new GraphImpl(vTable, vid2pid, localVidMap, newETable) + val newETable = createETable(newEdges, partitioner) + new GraphImpl(vTable, vid2pid, localVidMap, newETable, partitioner) } override def groupEdges[ED2: ClassManifest](f: Iterator[Edge[ED]] => ED2 ): @@ -284,9 +284,9 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( .map { case ((src, dst), data) => Edge(src, dst, data) } } // TODO(crankshaw) eliminate the need to call createETable - val newETable = createETable(newEdges) + val newETable = createETable(newEdges, partitioner) - new GraphImpl(vTable, vid2pid, localVidMap, newETable) + new GraphImpl(vTable, vid2pid, localVidMap, newETable, partitioner) } ////////////////////////////////////////////////////////////////////////////////////////////////// @@ -304,7 +304,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( : Graph[VD2, ED] = { ClosureCleaner.clean(updateF) val newVTable = vTable.leftJoin(updates)(updateF) - new GraphImpl(newVTable, vid2pid, localVidMap, eTable) + new GraphImpl(newVTable, vid2pid, localVidMap, eTable, partitioner) } } // end of class GraphImpl @@ -358,16 +358,17 @@ object GraphImpl { val vid2pid = new Vid2Pid(etable, vtable.index) val localVidMap = createLocalVidMap(etable) - new GraphImpl(vtable, vid2pid, localVidMap, etable) + new GraphImpl(vtable, vid2pid, localVidMap, etable, partitionStrategy) } - protected def createETable[ED: ClassManifest](edges: RDD[Edge[ED]]) - : RDD[(Pid, EdgePartition[ED])] = { - createETable(edges, RandomVertexCut) - } + // TODO(crankshaw) - can I remove this + //protected def createETable[ED: ClassManifest](edges: RDD[Edge[ED]]) + // : RDD[(Pid, EdgePartition[ED])] = { + // createETable(edges, RandomVertexCut) + //} /** * Create the edge table RDD, which is much more efficient for Java heap storage than the @@ -384,10 +385,6 @@ object GraphImpl { val numPartitions = edges.partitions.size edges.map { e => - // Random partitioning based on the source vertex id. - // val part: Pid = edgePartitionFunction1D(e.srcId, e.dstId, numPartitions) - // val part: Pid = edgePartitionFunction2D(e.srcId, e.dstId, numPartitions, ceilSqrt) - //val part: Pid = randomVertexCut(e.srcId, e.dstId, numPartitions) val part: Pid = partitionStrategy.getPartition(e.srcId, e.dstId, numPartitions) // Should we be using 3-tuple or an optimized class @@ -449,7 +446,7 @@ object GraphImpl { } Iterator((pid, newEdgePartition)) } - new GraphImpl(g.vTable, g.vid2pid, g.localVidMap, newETable) + new GraphImpl(g.vTable, g.vid2pid, g.localVidMap, newETable, g.partitioner) } def mapReduceTriplets[VD: ClassManifest, ED: ClassManifest, A: ClassManifest]( From 5f3ee537518e63997ffcdd1c95acc20a3b413b54 Mon Sep 17 00:00:00 2001 From: Dan Crankshaw Date: Mon, 18 Nov 2013 19:34:02 -0800 Subject: [PATCH 210/531] Added accessVertexAttr func which somehow got lost in a merge. --- .../scala/org/apache/spark/graph/impl/GraphImpl.scala | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index 485e49f95e0ca..7c3d401832482 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -517,4 +517,13 @@ object GraphImpl { VertexSetRDD.aggregate(preAgg, g.vTable.index, reduceFunc) } + private def accessesVertexAttr[VD: ClassManifest, ED: ClassManifest]( + closure: AnyRef, attrName: String): Boolean = { + try { + BytecodeUtils.invokedMethod(closure, classOf[EdgeTriplet[VD, ED]], attrName) + } catch { + case _: ClassNotFoundException => true // if we don't know, be conservative + } + } + } // end of object GraphImpl From 37a524d91c2eb03dd9e4a24d8af33769a89a78e3 Mon Sep 17 00:00:00 2001 From: Dan Crankshaw Date: Tue, 19 Nov 2013 16:39:39 -0800 Subject: [PATCH 211/531] Addressed code review comments. --- .../org/apache/spark/graph/Analytics.scala | 12 +-- .../scala/org/apache/spark/graph/Graph.scala | 3 +- .../org/apache/spark/graph/GraphLab.scala | 3 +- .../org/apache/spark/graph/GraphLoader.scala | 2 +- .../spark/graph/PartitionStrategy.scala | 8 +- .../scala/org/apache/spark/graph/Pregel.scala | 3 +- .../apache/spark/graph/impl/GraphImpl.scala | 87 ++++++++++--------- 7 files changed, 59 insertions(+), 59 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala index 8455a145ffffd..f542ec60695af 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala @@ -241,7 +241,7 @@ object Analytics extends Logging { var outFname = "" var numVPart = 4 var numEPart = 4 - var partitionStrategy: PartitionStrategy = RandomVertexCut + var partitionStrategy: PartitionStrategy = RandomVertexCut() options.foreach{ case ("numIter", v) => numIter = v.toInt @@ -251,11 +251,11 @@ object Analytics extends Logging { case ("numVPart", v) => numVPart = v.toInt case ("numEPart", v) => numEPart = v.toInt case ("partStrategy", v) => { - v match { - case "RandomVertexCut" => partitionStrategy = RandomVertexCut - case "EdgePartition1D" => partitionStrategy = EdgePartition1D - case "EdgePartition2D" => partitionStrategy = EdgePartition2D - case "CanonicalRandomVertexCut" => partitionStrategy = CanonicalRandomVertexCut + partitionStrategy = v match { + case "RandomVertexCut" => RandomVertexCut() + case "EdgePartition1D" => EdgePartition1D() + case "EdgePartition2D" => EdgePartition2D() + case "CanonicalRandomVertexCut" => CanonicalRandomVertexCut() case _ => throw new IllegalArgumentException("Invalid Partition Strategy: " + v) } } diff --git a/graph/src/main/scala/org/apache/spark/graph/Graph.scala b/graph/src/main/scala/org/apache/spark/graph/Graph.scala index 6ce3f5d2e75c0..87667f69586ef 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Graph.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Graph.scala @@ -1,7 +1,6 @@ package org.apache.spark.graph import org.apache.spark.rdd.RDD -import org.apache.spark.Logging import org.apache.spark.storage.StorageLevel /** @@ -22,7 +21,7 @@ import org.apache.spark.storage.StorageLevel * @tparam VD the vertex attribute type * @tparam ED the edge attribute type */ -abstract class Graph[VD: ClassManifest, ED: ClassManifest] extends Logging { +abstract class Graph[VD: ClassManifest, ED: ClassManifest] { /** * Get the vertices and their data. diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala b/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala index 39dc33acf0f6e..b8503ab7fdb6c 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala @@ -2,12 +2,11 @@ package org.apache.spark.graph import scala.collection.JavaConversions._ import org.apache.spark.rdd.RDD -import org.apache.spark.Logging /** * This object implements the GraphLab gather-apply-scatter api. */ -object GraphLab extends Logging { +object GraphLab { /** * Execute the GraphLab Gather-Apply-Scatter API diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala b/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala index 813f176313a80..4dc33a02ceacc 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala @@ -28,7 +28,7 @@ object GraphLoader { edgeParser: Array[String] => ED, minEdgePartitions: Int = 1, minVertexPartitions: Int = 1, - partitionStrategy: PartitionStrategy = RandomVertexCut): GraphImpl[Int, ED] = { + partitionStrategy: PartitionStrategy = RandomVertexCut()): GraphImpl[Int, ED] = { // Parse the edge data table val edges = sc.textFile(path, minEdgePartitions).flatMap { line => diff --git a/graph/src/main/scala/org/apache/spark/graph/PartitionStrategy.scala b/graph/src/main/scala/org/apache/spark/graph/PartitionStrategy.scala index f7db667e2ff05..cf65f5065786a 100644 --- a/graph/src/main/scala/org/apache/spark/graph/PartitionStrategy.scala +++ b/graph/src/main/scala/org/apache/spark/graph/PartitionStrategy.scala @@ -50,7 +50,7 @@ sealed trait PartitionStrategy extends Serializable { * * */ -object EdgePartition2D extends PartitionStrategy { +case class EdgePartition2D() extends PartitionStrategy { override def getPartition(src: Vid, dst: Vid, numParts: Pid): Pid = { val ceilSqrtNumParts: Pid = math.ceil(math.sqrt(numParts)).toInt val mixingPrime: Vid = 1125899906842597L @@ -61,7 +61,7 @@ object EdgePartition2D extends PartitionStrategy { } -object EdgePartition1D extends PartitionStrategy { +case class EdgePartition1D() extends PartitionStrategy { override def getPartition(src: Vid, dst: Vid, numParts: Pid): Pid = { val mixingPrime: Vid = 1125899906842597L (math.abs(src) * mixingPrime).toInt % numParts @@ -73,7 +73,7 @@ object EdgePartition1D extends PartitionStrategy { * Assign edges to an aribtrary machine corresponding to a * random vertex cut. */ -object RandomVertexCut extends PartitionStrategy { +case class RandomVertexCut() extends PartitionStrategy { override def getPartition(src: Vid, dst: Vid, numParts: Pid): Pid = { math.abs((src, dst).hashCode()) % numParts } @@ -85,7 +85,7 @@ object RandomVertexCut extends PartitionStrategy { * function ensures that edges of opposite direction between the same two vertices * will end up on the same partition. */ -object CanonicalRandomVertexCut extends PartitionStrategy { +case class CanonicalRandomVertexCut() extends PartitionStrategy { override def getPartition(src: Vid, dst: Vid, numParts: Pid): Pid = { val lower = math.min(src, dst) val higher = math.max(src, dst) diff --git a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala index f3016e6ad3ee6..3b4d3c0df2a51 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala @@ -1,7 +1,6 @@ package org.apache.spark.graph import org.apache.spark.rdd.RDD -import org.apache.spark.Logging /** @@ -42,7 +41,7 @@ import org.apache.spark.Logging * }}} * */ -object Pregel extends Logging { +object Pregel { /** diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index 7c3d401832482..6ad0ce60a7add 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -8,6 +8,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.SparkContext._ import org.apache.spark.HashPartitioner import org.apache.spark.util.ClosureCleaner +import org.apache.spark.SparkException import org.apache.spark.Partitioner import org.apache.spark.graph._ @@ -97,8 +98,6 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( @transient override val triplets: RDD[EdgeTriplet[VD, ED]] = makeTriplets(localVidMap, vTableReplicatedValues.bothAttrs, eTable) - //@transient private val partitioner: PartitionStrategy = partitionStrategy - override def persist(newLevel: StorageLevel): Graph[VD, ED] = { eTable.persist(newLevel) vid2pid.persist(newLevel) @@ -250,43 +249,55 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( override def groupEdgeTriplets[ED2: ClassManifest]( f: Iterator[EdgeTriplet[VD,ED]] => ED2 ): Graph[VD,ED2] = { - val newEdges: RDD[Edge[ED2]] = triplets.mapPartitions { partIter => - partIter - // TODO(crankshaw) toList requires that the entire edge partition - // can fit in memory right now. - .toList - // groups all ETs in this partition that have the same src and dst - // Because all ETs with the same src and dst will live on the same - // partition due to the canonicalRandomVertexCut partitioner, this - // guarantees that these ET groups will be complete. - .groupBy { t: EdgeTriplet[VD, ED] => (t.srcId, t.dstId) } - .mapValues { ts: List[EdgeTriplet[VD, ED]] => f(ts.toIterator) } - .toList - .toIterator - .map { case ((src, dst), data) => Edge(src, dst, data) } - .toIterator - } + partitioner match { + case _: CanonicalRandomVertexCut => { + val newEdges: RDD[Edge[ED2]] = triplets.mapPartitions { partIter => + partIter + // TODO(crankshaw) toList requires that the entire edge partition + // can fit in memory right now. + .toList + // groups all ETs in this partition that have the same src and dst + // Because all ETs with the same src and dst will live on the same + // partition due to the canonicalRandomVertexCut partitioner, this + // guarantees that these ET groups will be complete. + .groupBy { t: EdgeTriplet[VD, ED] => (t.srcId, t.dstId) } + .mapValues { ts: List[EdgeTriplet[VD, ED]] => f(ts.toIterator) } + .toList + .toIterator + .map { case ((src, dst), data) => Edge(src, dst, data) } + .toIterator + } + //TODO(crankshaw) eliminate the need to call createETable + val newETable = createETable(newEdges, partitioner) + new GraphImpl(vTable, vid2pid, localVidMap, newETable, partitioner) + } - //TODO(crankshaw) eliminate the need to call createETable - val newETable = createETable(newEdges, partitioner) - new GraphImpl(vTable, vid2pid, localVidMap, newETable, partitioner) + case _ => throw new SparkException(partitioner.getClass.getName + + " is incompatible with groupEdgeTriplets") + } } override def groupEdges[ED2: ClassManifest](f: Iterator[Edge[ED]] => ED2 ): Graph[VD,ED2] = { + partitioner match { + case _: CanonicalRandomVertexCut => { + val newEdges: RDD[Edge[ED2]] = edges.mapPartitions { partIter => + partIter.toList + .groupBy { e: Edge[ED] => (e.srcId, e.dstId) } + .mapValues { ts => f(ts.toIterator) } + .toList + .toIterator + .map { case ((src, dst), data) => Edge(src, dst, data) } + } + // TODO(crankshaw) eliminate the need to call createETable + val newETable = createETable(newEdges, partitioner) - val newEdges: RDD[Edge[ED2]] = edges.mapPartitions { partIter => - partIter.toList - .groupBy { e: Edge[ED] => (e.srcId, e.dstId) } - .mapValues { ts => f(ts.toIterator) } - .toList - .toIterator - .map { case ((src, dst), data) => Edge(src, dst, data) } - } - // TODO(crankshaw) eliminate the need to call createETable - val newETable = createETable(newEdges, partitioner) + new GraphImpl(vTable, vid2pid, localVidMap, newETable, partitioner) + } - new GraphImpl(vTable, vid2pid, localVidMap, newETable, partitioner) + case _ => throw new SparkException(partitioner.getClass.getName + + " is incompatible with groupEdges") + } } ////////////////////////////////////////////////////////////////////////////////////////////////// @@ -315,7 +326,7 @@ object GraphImpl { vertices: RDD[(Vid, VD)], edges: RDD[Edge[ED]], defaultVertexAttr: VD): GraphImpl[VD,ED] = { - apply(vertices, edges, defaultVertexAttr, (a:VD, b:VD) => a, RandomVertexCut) + apply(vertices, edges, defaultVertexAttr, (a:VD, b:VD) => a, RandomVertexCut()) } def apply[VD: ClassManifest, ED: ClassManifest]( @@ -331,7 +342,7 @@ object GraphImpl { edges: RDD[Edge[ED]], defaultVertexAttr: VD, mergeFunc: (VD, VD) => VD): GraphImpl[VD,ED] = { - apply(vertices, edges, defaultVertexAttr, mergeFunc, RandomVertexCut) + apply(vertices, edges, defaultVertexAttr, mergeFunc, RandomVertexCut()) } def apply[VD: ClassManifest, ED: ClassManifest]( @@ -362,14 +373,6 @@ object GraphImpl { } - - - // TODO(crankshaw) - can I remove this - //protected def createETable[ED: ClassManifest](edges: RDD[Edge[ED]]) - // : RDD[(Pid, EdgePartition[ED])] = { - // createETable(edges, RandomVertexCut) - //} - /** * Create the edge table RDD, which is much more efficient for Java heap storage than the * normal edges data structure (RDD[(Vid, Vid, ED)]). From 34bcf1b32b99c1fd353f4f4073c89fb2db395142 Mon Sep 17 00:00:00 2001 From: Dan Crankshaw Date: Tue, 19 Nov 2013 16:46:25 -0800 Subject: [PATCH 212/531] Re-added slaves file for compatibility with Spark --- .gitignore | 2 -- conf/slaves | 2 ++ 2 files changed, 2 insertions(+), 2 deletions(-) create mode 100644 conf/slaves diff --git a/.gitignore b/.gitignore index 57e26c4d37338..e1f64a113390a 100644 --- a/.gitignore +++ b/.gitignore @@ -14,8 +14,6 @@ conf/java-opts conf/spark-env.sh conf/streaming-env.sh conf/log4j.properties -conf/slaves -conf/core-site.xml docs/_site docs/api target/ diff --git a/conf/slaves b/conf/slaves new file mode 100644 index 0000000000000..da0a01343d20a --- /dev/null +++ b/conf/slaves @@ -0,0 +1,2 @@ +# A Spark Worker will be started on each of the machines listed below. +localhost \ No newline at end of file From 96fafdbd4b42d01f40906711145f364c908200f6 Mon Sep 17 00:00:00 2001 From: Dan Crankshaw Date: Tue, 19 Nov 2013 20:39:34 -0800 Subject: [PATCH 213/531] Removed sleep from pagerank in Analytics. --- graph/src/main/scala/org/apache/spark/graph/Analytics.scala | 2 -- 1 file changed, 2 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala index f542ec60695af..01bf6684cb6ec 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala @@ -295,8 +295,6 @@ object Analytics extends Logging { } logInfo("GRAPHX: Runtime: " + ((System.currentTimeMillis - startTime)/1000.0) + " seconds") - - Thread.sleep(1000000) sc.stop() } From b12b2ccde8683a091ed2ac15113c5609184c95e6 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Mon, 18 Nov 2013 12:22:41 -0800 Subject: [PATCH 214/531] Addressing bug in open hash set where getPos on a full open hash set could loop forever. --- .../scala/org/apache/spark/util/collection/OpenHashSet.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala b/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala index e86d7ef767798..84583bf20f9e7 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala @@ -139,7 +139,8 @@ class OpenHashSet[@specialized(Long, Int) T: ClassManifest]( def getPos(k: T): Int = { var pos = hashcode(hasher.hash(k)) & _mask var i = 1 - while (true) { + val maxProbe = _data.size + while (i < maxProbe) { if (!_bitset.get(pos)) { return INVALID_POS } else if (k == _data(pos)) { From 2fc6f5bd47fcd38692fb814e2c04a6055e584dd5 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Mon, 18 Nov 2013 12:23:42 -0800 Subject: [PATCH 215/531] Switching collectNeighborIds to use mapReduceTriplets directly --- .../org/apache/spark/graph/GraphOps.scala | 18 +++++++++++++----- 1 file changed, 13 insertions(+), 5 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala b/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala index 5fd8cd699106e..11659ce66d10a 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala @@ -3,7 +3,7 @@ package org.apache.spark.graph import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext._ import org.apache.spark.util.ClosureCleaner - +import org.apache.spark.SparkException /** @@ -156,10 +156,18 @@ class GraphOps[VD: ClassManifest, ED: ClassManifest](graph: Graph[VD, ED]) { */ def collectNeighborIds(edgeDirection: EdgeDirection) : VertexSetRDD[Array[Vid]] = { - val nbrs = graph.aggregateNeighbors[Array[Vid]]( - (vid, edge) => Some(Array(edge.otherVertexId(vid))), - (a, b) => a ++ b, - edgeDirection) + val nbrs = + if (edgeDirection == EdgeDirection.Both) { + graph.mapReduceTriplets[Array[Vid]] ( + et => Array( (et.srcId, Array(et.dstId)), (et.dstId, Array(et.srcId))), _ ++ _ + ) + } else if (edgeDirection == EdgeDirection.Out) { + graph.mapReduceTriplets[Array[Vid]](et => Array((et.srcId, Array(et.dstId))), _ ++ _) + } else if (edgeDirection == EdgeDirection.In) { + graph.mapReduceTriplets[Array[Vid]](et => Array((et.dstId, Array(et.srcId))), _ ++ _) + } else { + throw new SparkException("It doesn't make sense to collect neighbor ids without a direction.") + } graph.vertices.leftZipJoin(nbrs) { (vid, vdata, nbrsOpt) => nbrsOpt.getOrElse(Array.empty[Vid]) } } // end of collectNeighborIds From 288ae310e72e62b822a0e59f662da6a118f90ae7 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Mon, 18 Nov 2013 12:24:15 -0800 Subject: [PATCH 216/531] adding test for collectNeighborIds --- .../org/apache/spark/graph/GraphSuite.scala | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala index 899048a17a6d2..677927f958566 100644 --- a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala @@ -99,6 +99,22 @@ class GraphSuite extends FunSuite with LocalSparkContext { } } + test("collectNeighborIds") { + withSpark(new SparkContext("local", "test")) { sc => + 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(rawEdges) + val nbrs = graph.collectNeighborIds(EdgeDirection.Both) + assert(nbrs.count === chain.size) + assert(graph.numVertices === nbrs.count) + nbrs.collect.foreach { case (vid, nbrs) => assert(nbrs.size === 2) } + nbrs.collect.foreach { case (vid, nbrs) => + val s = nbrs.toSet + assert(s.contains((vid + 1) % 100)) + assert(s.contains(if (vid > 0) { vid - 1 } else { 99 })) + } + } + } test("VertexSetRDD") { withSpark(new SparkContext("local", "test")) { sc => From 8719ba83c800028a5cb3d57ea84b7b8b7ffb004f Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Mon, 18 Nov 2013 18:41:05 -0800 Subject: [PATCH 217/531] Modifying graph loaders to create initial vertex sets more efficiently and load undirected graphs. --- .../scala/org/apache/spark/graph/Graph.scala | 52 ++++--- .../org/apache/spark/graph/GraphLoader.scala | 84 +++++++--- .../org/apache/spark/graph/VertexSetRDD.scala | 17 ++ .../spark/graph/impl/EdgePartition.scala | 20 ++- .../apache/spark/graph/impl/GraphImpl.scala | 145 ++++++------------ .../spark/graph/util/GraphGenerators.scala | 2 +- .../org/apache/spark/graph/GraphSuite.scala | 14 +- 7 files changed, 177 insertions(+), 157 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/Graph.scala b/graph/src/main/scala/org/apache/spark/graph/Graph.scala index 87667f69586ef..848e580ebd0fc 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Graph.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Graph.scala @@ -241,7 +241,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * @return Graph[VD,ED2] The resulting graph with a single Edge for * each source, dest vertex pair. */ - def groupEdges[ED2: ClassManifest](f: Iterator[Edge[ED]] => ED2 ): Graph[VD,ED2] + def groupEdges(merge: (ED, ED) => ED): Graph[VD,ED] /** @@ -334,21 +334,31 @@ object Graph { import org.apache.spark.graph.impl._ import org.apache.spark.SparkContext._ + /** + * Construct a graph from a collection of edges. + * + * @param edges the RDD containing the set of edges in the graph + * @param defaultValue the default vertex attribute to use for each vertex + * + * @return a graph with edge attributes described by `edges` and vertices + * given by all vertices in `edges` with value `defaultValue` + */ + def apply[VD: ClassManifest, ED: ClassManifest]( + edges: RDD[Edge[ED]], defaultValue: VD): Graph[VD, ED] = { + GraphImpl(edges, defaultValue) + } + /** * Construct a graph from a collection of edges encoded as vertex id - * pairs. Duplicate directed edges are merged to a single edge with - * weight equal to the number of duplicate edges. The returned - * vertex attribute is the number of edges adjacent to that vertex - * (i.e., the undirected degree). + * pairs. * * @param rawEdges the RDD containing the set of edges in the graph * * @return a graph with edge attributes containing the count of - * duplicate edges and vertex attributes containing the total degree - * of each vertex. + * duplicate edges. */ - def apply(rawEdges: RDD[(Vid, Vid)]): Graph[Int, Int] = { Graph(rawEdges, true) } - + def apply[VD: ClassManifest](rawEdges: RDD[(Vid, Vid)], defaultValue: VD): + Graph[VD, Int] = { Graph(rawEdges, defaultValue, false) } /** * Construct a graph from a collection of edges encoded as vertex id @@ -364,24 +374,16 @@ object Graph { * attributes containing the total degree of each vertex. * */ - def apply(rawEdges: RDD[(Vid, Vid)], uniqueEdges: Boolean): Graph[Int, Int] = { - // Reduce to unique edges. - val edges: RDD[Edge[Int]] = - if (uniqueEdges) { - rawEdges.map((_, 1)).reduceByKey(_ + _).map { case ((s, t), cnt) => Edge(s, t, cnt) } - } else { - rawEdges.map { case (s, t) => Edge(s, t, 1) } - } - // Determine unique vertices - /** @todo Should this reduceByKey operation be indexed? */ - val vertices: RDD[(Vid, Int)] = - edges.flatMap{ case Edge(s, t, cnt) => Array((s, 1), (t, 1)) }.reduceByKey(_ + _) - - // Return graph - GraphImpl(vertices, edges, 0) + def apply[VD: ClassManifest](rawEdges: RDD[(Vid, Vid)], defaultValue: VD, uniqueEdges: Boolean): + Graph[VD, Int] = { + val graph = GraphImpl(rawEdges.map(p => Edge(p._1, p._2, 1)), defaultValue) + if(uniqueEdges) { + graph.groupEdges((a,b) => a+b) + } else { + graph + } } - /** * Construct a graph from a collection attributed vertices and * edges. diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala b/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala index 4dc33a02ceacc..fadc58284c318 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala @@ -20,43 +20,83 @@ object GraphLoader { * @param minEdgePartitions the number of partitions for the * the Edge RDD * - * @todo remove minVertexPartitions arg */ def textFile[ED: ClassManifest]( sc: SparkContext, path: String, edgeParser: Array[String] => ED, minEdgePartitions: Int = 1, - minVertexPartitions: Int = 1, partitionStrategy: PartitionStrategy = RandomVertexCut()): GraphImpl[Int, ED] = { - // Parse the edge data table - val edges = sc.textFile(path, minEdgePartitions).flatMap { line => - if (!line.isEmpty && line(0) != '#') { + val edges = sc.textFile(path, minEdgePartitions).mapPartitions( iter => + iter.filter(line => !line.isEmpty && line(0) != '#').map { line => val lineArray = line.split("\\s+") if(lineArray.length < 2) { println("Invalid line: " + line) assert(false) } - val source = lineArray(0) - val target = lineArray(1) + val source = lineArray(0).trim.toLong + val target = lineArray(1).trim.toLong val tail = lineArray.drop(2) val edata = edgeParser(tail) - Array(Edge(source.trim.toInt, target.trim.toInt, edata)) - } else { - Array.empty[Edge[ED]] - } - }.cache() - - val graph = fromEdges(edges, partitionStrategy) - graph + Edge(source, target, edata) + }) + val defaultVertexAttr = 1 + Graph(edges, defaultVertexAttr, partitionStrategy) } - private def fromEdges[ED: ClassManifest]( - edges: RDD[Edge[ED]], - partitionStrategy: PartitionStrategy): GraphImpl[Int, ED] = { - val vertices = edges.flatMap { edge => List((edge.srcId, 1), (edge.dstId, 1)) } - .reduceByKey(_ + _) - GraphImpl(vertices, edges, 0, (a: Int, b: Int) => a, partitionStrategy) - } + /** + * Load a graph from an edge list formatted file with each line containing + * two integers: a source Id and a target Id. + * + * @example A file in the following format: + * {{{ + * # Comment Line + * # Source Id <\t> Target Id + * 1 -5 + * 1 2 + * 2 7 + * 1 8 + * }}} + * + * If desired the edges can be automatically oriented in the positive + * direction (source Id < target Id) by setting `canonicalOrientation` to + * true + * + * @param sc + * @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 + * @return + */ + def edgeListFile[ED: ClassManifest]( + sc: SparkContext, + path: String, + canonicalOrientation: Boolean = false, + minEdgePartitions: Int = 1, + partitionStrategy: PartitionStrategy = RandomVertexCut()): + Graph[Int, Int] = { + // Parse the edge data table + val edges = sc.textFile(path, minEdgePartitions).mapPartitions( iter => + iter.filter(line => !line.isEmpty && line(0) != '#').map { line => + val lineArray = line.split("\\s+") + if(lineArray.length < 2) { + println("Invalid line: " + line) + assert(false) + } + val source = lineArray(0).trim.toLong + val target = lineArray(1).trim.toLong + if (canonicalOrientation && target > source) { + Edge(target, source, 1) + } else { + Edge(source, target, 1) + } + }) + val defaultVertexAttr = 1 + Graph(edges, defaultVertexAttr, partitionStrategy) + } // end of edgeListFile + } diff --git a/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala b/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala index 507370539e31a..4a056911fda6f 100644 --- a/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala @@ -644,6 +644,23 @@ object VertexSetRDD { new VertexSetIndex(index) } + /** + * Cosntruct an VertexSetRDD with all vertices initialized to the default value. + * + * @param index + * @param defaultValue + * @tparam V + * @return + */ + def apply[V: ClassManifest](index: VertexSetIndex, defaultValue: V): VertexSetRDD[V] = { + // Use the index to build the new values table + val values: RDD[ (Array[V], BitSet) ] = index.rdd.mapPartitions(_.map { index => + val values = Array.fill(index.capacity)(defaultValue) + val bs = index.getBitSet + (values, bs) + }, preservesPartitioning = true) + new VertexSetRDD(index, values) + } // end of apply } // end of object VertexSetRDD diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala b/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala index 750075533af4e..9ecf531d50506 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala @@ -1,6 +1,7 @@ package org.apache.spark.graph.impl import org.apache.spark.graph._ +import org.apache.spark.util.collection.OpenHashMap /** @@ -9,8 +10,7 @@ import org.apache.spark.graph._ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED: ClassManifest]( val srcIds: Array[Vid], val dstIds: Array[Vid], - val data: Array[ED]) -{ + val data: Array[ED]) { def reverse: EdgePartition[ED] = new EdgePartition(dstIds, srcIds, data) @@ -42,6 +42,22 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) } } + def groupEdges(merge: (ED, ED) => ED): EdgePartition[ED] = { + val agg = new OpenHashMap[(Vid,Vid), ED] //(math.ceil((data.size + 1)/0.7).toInt) + foreach { e => agg.setMerge((e.srcId, e.dstId), e.attr, merge) } + val newSrcIds = new Array[Vid](agg.size) + val newDstIds = new Array[Vid](agg.size) + val newData = new Array[ED](agg.size) + var i = 0 + agg.foreach { kv => + newSrcIds(i) = kv._1._1 + newDstIds(i) = kv._1._2 + newData(i) = kv._2 + i += 1 + } + new EdgePartition(newSrcIds, newDstIds, newData) + } + def size: Int = srcIds.size def iterator = new Iterator[Edge[ED]] { diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index 6ad0ce60a7add..24b60c7a9bf16 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -135,8 +135,10 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( * Display the lineage information for this graph. */ def printLineage() = { - - def traverseLineage(rdd: RDD[_], indent: String = "", visited: Map[Int, String] = Map.empty[Int, String]) { + 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) @@ -147,42 +149,35 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( 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 + 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("eTable ------------------------------------------") traverseLineage(eTable, " ") var visited = Map(eTable.id -> "eTable") - println("\n\nvTable.index ------------------------------------") traverseLineage(vTable.index.rdd, " ", visited) visited += (vTable.index.rdd.id -> "vTable.index") - println("\n\nvTable.values ------------------------------------") traverseLineage(vTable.valuesRDD, " ", visited) visited += (vTable.valuesRDD.id -> "vTable.values") - println("\n\nvTable ------------------------------------------") traverseLineage(vTable, " ", visited) visited += (vTable.id -> "vTable") - println("\n\nvid2pid.bothAttrs -------------------------------") traverseLineage(vid2pid.bothAttrs, " ", visited) visited += (vid2pid.bothAttrs.id -> "vid2pid") visited += (vid2pid.bothAttrs.valuesRDD.id -> "vid2pid.bothAttrs") - println("\n\nlocalVidMap -------------------------------------") traverseLineage(localVidMap, " ", visited) visited += (localVidMap.id -> "localVidMap") - println("\n\nvTableReplicatedValues.bothAttrs ----------------") traverseLineage(vTableReplicatedValues.bothAttrs, " ", visited) visited += (vTableReplicatedValues.bothAttrs.id -> "vTableReplicatedValues.bothAttrs") - println("\n\ntriplets ----------------------------------------") traverseLineage(triplets, " ", visited) println(visited) @@ -210,94 +205,27 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( override def subgraph(epred: EdgeTriplet[VD,ED] => Boolean = (x => true), vpred: (Vid, VD) => Boolean = ((a,b) => true) ): Graph[VD, ED] = { - - /** @todo The following code behaves deterministically on each - * vertex predicate but uses additional space. Should we swithc to - * this version - */ - // val predGraph = mapVertices(v => (v.data, vpred(v))) - // val newETable = predGraph.triplets.filter(t => - // if(v.src.data._2 && v.dst.data._2) { - // val src = Vertex(t.src.id, t.src.data._1) - // val dst = Vertex(t.dst.id, t.dst.data._1) - // epred(new EdgeTriplet[VD, ED](src, dst, t.data)) - // } else { false }) - - // val newVTable = predGraph.vertices.filter(v => v.data._1) - // .map(v => (v.id, v.data._1)).indexed() - // Reuse the partitioner (but not the index) from this graph val newVTable = VertexSetRDD(vertices.filter(v => vpred(v._1, v._2)).partitionBy(vTable.index.partitioner)) - - // Restrict the set of edges to those that satisfy the vertex and the edge predicate. val newETable = createETable( triplets.filter( t => vpred( t.srcId, t.srcAttr ) && vpred( t.dstId, t.dstAttr ) && epred(t) - ) - .map( t => Edge(t.srcId, t.dstId, t.attr) ), partitioner) - + ).map( t => Edge(t.srcId, t.dstId, t.attr) ), partitioner) // Construct the Vid2Pid map. Here we assume that the filter operation // behaves deterministically. // @todo reindex the vertex and edge tables val newVid2Pid = new Vid2Pid(newETable, newVTable.index) val newVidMap = createLocalVidMap(newETable) - new GraphImpl(newVTable, newVid2Pid, localVidMap, newETable, partitioner) - } + } // end of subgraph - override def groupEdgeTriplets[ED2: ClassManifest]( - f: Iterator[EdgeTriplet[VD,ED]] => ED2 ): Graph[VD,ED2] = { - partitioner match { - case _: CanonicalRandomVertexCut => { - val newEdges: RDD[Edge[ED2]] = triplets.mapPartitions { partIter => - partIter - // TODO(crankshaw) toList requires that the entire edge partition - // can fit in memory right now. - .toList - // groups all ETs in this partition that have the same src and dst - // Because all ETs with the same src and dst will live on the same - // partition due to the canonicalRandomVertexCut partitioner, this - // guarantees that these ET groups will be complete. - .groupBy { t: EdgeTriplet[VD, ED] => (t.srcId, t.dstId) } - .mapValues { ts: List[EdgeTriplet[VD, ED]] => f(ts.toIterator) } - .toList - .toIterator - .map { case ((src, dst), data) => Edge(src, dst, data) } - .toIterator - } - //TODO(crankshaw) eliminate the need to call createETable - val newETable = createETable(newEdges, partitioner) - new GraphImpl(vTable, vid2pid, localVidMap, newETable, partitioner) - } - - case _ => throw new SparkException(partitioner.getClass.getName - + " is incompatible with groupEdgeTriplets") - } - } - - override def groupEdges[ED2: ClassManifest](f: Iterator[Edge[ED]] => ED2 ): - Graph[VD,ED2] = { - partitioner match { - case _: CanonicalRandomVertexCut => { - val newEdges: RDD[Edge[ED2]] = edges.mapPartitions { partIter => - partIter.toList - .groupBy { e: Edge[ED] => (e.srcId, e.dstId) } - .mapValues { ts => f(ts.toIterator) } - .toList - .toIterator - .map { case ((src, dst), data) => Edge(src, dst, data) } - } - // TODO(crankshaw) eliminate the need to call createETable - val newETable = createETable(newEdges, partitioner) - - new GraphImpl(vTable, vid2pid, localVidMap, newETable, partitioner) - } - - case _ => throw new SparkException(partitioner.getClass.getName - + " is incompatible with groupEdges") - } + override def groupEdges(merge: (ED, ED) => ED ): Graph[VD,ED] = { + ClosureCleaner.clean(merge) + val newETable = + eTable.mapPartitions { _.map(p => (p._1, p._2.groupEdges(merge))) } + new GraphImpl(vTable, vid2pid, localVidMap, newETable, partitioner) } ////////////////////////////////////////////////////////////////////////////////////////////////// @@ -322,6 +250,27 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( object GraphImpl { + def apply[VD: ClassManifest, ED: ClassManifest]( + edges: RDD[Edge[ED]], + defaultValue: VD, + partitionStrategy: PartitionStrategy): + GraphImpl[VD, ED] = { + val etable = createETable(edges).cache + // Get the set of all vids + val vids = etable.mapPartitions( iter => { + val (pid, epart) = iter.next() + assert(!iter.hasNext) + epart.iterator.flatMap(e => Iterator(e.srcId, e.dstId)) + }, true) + // Index the set of all vids + val index = VertexSetRDD.makeIndex(vids) + // Index the vertices and fill in missing attributes with the default + val vtable = VertexSetRDD(index, defaultValue) + val vid2pid = new Vid2Pid(etable, vtable.index) + val localVidMap = createLocalVidMap(etable) + new GraphImpl(vtable, vid2pid, localVidMap, etable, partitionStrategy) + } + def apply[VD: ClassManifest, ED: ClassManifest]( vertices: RDD[(Vid, VD)], edges: RDD[Edge[ED]], @@ -372,7 +321,6 @@ object GraphImpl { new GraphImpl(vtable, vid2pid, localVidMap, etable, partitionStrategy) } - /** * Create the edge table RDD, which is much more efficient for Java heap storage than the * normal edges data structure (RDD[(Vid, Vid, ED)]). @@ -419,9 +367,9 @@ object GraphImpl { } def makeTriplets[VD: ClassManifest, ED: ClassManifest]( - localVidMap: RDD[(Pid, VertexIdToIndexMap)], - vTableReplicatedValues: RDD[(Pid, Array[VD]) ], - eTable: RDD[(Pid, EdgePartition[ED])]): RDD[EdgeTriplet[VD, ED]] = { + localVidMap: RDD[(Pid, VertexIdToIndexMap)], + vTableReplicatedValues: RDD[(Pid, Array[VD]) ], + eTable: RDD[(Pid, EdgePartition[ED])]): RDD[EdgeTriplet[VD, ED]] = { eTable.zipPartitions(localVidMap, vTableReplicatedValues) { (eTableIter, vidMapIter, replicatedValuesIter) => val (_, vidToIndex) = vidMapIter.next() @@ -432,16 +380,16 @@ object GraphImpl { } def mapTriplets[VD: ClassManifest, ED: ClassManifest, ED2: ClassManifest]( - g: GraphImpl[VD, ED], - f: EdgeTriplet[VD, ED] => ED2): Graph[VD, ED2] = { - val newETable = g.eTable.zipPartitions(g.localVidMap, g.vTableReplicatedValues.bothAttrs){ + g: GraphImpl[VD, ED], + f: EdgeTriplet[VD, ED] => ED2): Graph[VD, ED2] = { + val newETable = g.eTable.zipPartitions(g.localVidMap, g.vTableReplicatedValues.bothAttrs) { (edgePartitionIter, vidToIndexIter, vertexArrayIter) => val (pid, edgePartition) = edgePartitionIter.next() val (_, vidToIndex) = vidToIndexIter.next() val (_, vertexArray) = vertexArrayIter.next() val et = new EdgeTriplet[VD, ED] val vmap = new PrimitiveKeyOpenHashMap[Vid, VD](vidToIndex, vertexArray) - val newEdgePartition = edgePartition.map{e => + val newEdgePartition = edgePartition.map { e => et.set(e) et.srcAttr = vmap(e.srcId) et.dstAttr = vmap(e.dstId) @@ -453,23 +401,20 @@ object GraphImpl { } def mapReduceTriplets[VD: ClassManifest, ED: ClassManifest, A: ClassManifest]( - g: GraphImpl[VD, ED], - mapFunc: EdgeTriplet[VD, ED] => Array[(Vid, A)], - reduceFunc: (A, A) => A): VertexSetRDD[A] = { - + g: GraphImpl[VD, ED], + mapFunc: EdgeTriplet[VD, ED] => Array[(Vid, A)], + reduceFunc: (A, A) => A): VertexSetRDD[A] = { ClosureCleaner.clean(mapFunc) ClosureCleaner.clean(reduceFunc) - // For each vertex, replicate its attribute only to partitions where it is // in the relevant position in an edge. val mapFuncUsesSrcAttr = accessesVertexAttr[VD, ED](mapFunc, "srcAttr") val mapFuncUsesDstAttr = accessesVertexAttr[VD, ED](mapFunc, "dstAttr") - // Map and preaggregate val preAgg = g.eTable.zipPartitions( g.localVidMap, g.vTableReplicatedValues.get(mapFuncUsesSrcAttr, mapFuncUsesDstAttr) - ){ + ) { (edgePartitionIter, vidToIndexIter, vertexArrayIter) => val (_, edgePartition) = edgePartitionIter.next() val (_, vidToIndex) = vidToIndexIter.next() diff --git a/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala b/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala index 2c652cb172b4a..f0e423a57ba78 100644 --- a/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala +++ b/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala @@ -275,7 +275,7 @@ object GraphGenerators { */ def starGraph(sc: SparkContext, nverts: Int): Graph[Int, Int] = { val edges: RDD[(Vid, Vid)] = sc.parallelize(1 until nverts).map(vid => (vid, 0)) - Graph(edges, false) + Graph(edges, 1) } // end of starGraph diff --git a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala index 677927f958566..b0d248b6511d5 100644 --- a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala @@ -15,7 +15,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { withSpark(new SparkContext("local", "test")) { sc => val rawEdges = (0L to 100L).zip((1L to 99L) :+ 0L) val edges = sc.parallelize(rawEdges) - val graph = Graph(edges) + val graph = Graph(edges, 1.0F) assert( graph.edges.count() === rawEdges.size ) } } @@ -38,7 +38,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { test("mapEdges") { withSpark(new SparkContext("local", "test")) { sc => val n = 3 - val star = Graph(sc.parallelize((1 to n).map(x => (0: Vid, x: Vid)))) + val star = Graph(sc.parallelize((1 to n).map(x => (0: Vid, x: Vid))), "defaultValue") val starWithEdgeAttrs = star.mapEdges(e => e.dstId) // map(_.copy()) is a workaround for https://github.com/amplab/graphx/issues/25 @@ -51,9 +51,9 @@ class GraphSuite extends FunSuite with LocalSparkContext { test("mapReduceTriplets") { withSpark(new SparkContext("local", "test")) { sc => val n = 3 - val star = Graph(sc.parallelize((1 to n).map(x => (0: Vid, x: Vid)))) - - val neighborDegreeSums = star.mapReduceTriplets( + val star = Graph(sc.parallelize((1 to n).map(x => (0: Vid, x: Vid))), 0) + val starDeg = star.joinVertices(star.degrees){ (vid, oldV, deg) => deg } + val neighborDegreeSums = starDeg.mapReduceTriplets( edge => Array((edge.srcId, edge.dstAttr), (edge.dstId, edge.srcAttr)), (a: Int, b: Int) => a + b) assert(neighborDegreeSums.collect().toSet === (0 to n).map(x => (x, n)).toSet) @@ -63,7 +63,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { test("aggregateNeighbors") { withSpark(new SparkContext("local", "test")) { sc => val n = 3 - val star = Graph(sc.parallelize((1 to n).map(x => (0: Vid, x: Vid)))) + val star = Graph(sc.parallelize((1 to n).map(x => (0: Vid, x: Vid))), 1) val indegrees = star.aggregateNeighbors( (vid, edge) => Some(1), @@ -103,7 +103,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { withSpark(new SparkContext("local", "test")) { sc => 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(rawEdges) + val graph = Graph(rawEdges, 1.0) val nbrs = graph.collectNeighborIds(EdgeDirection.Both) assert(nbrs.count === chain.size) assert(graph.numVertices === nbrs.count) From 2093a17ff388828f73145b039b90d4aeb799917d Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Mon, 18 Nov 2013 18:41:30 -0800 Subject: [PATCH 218/531] Adding triangle count code --- .../org/apache/spark/graph/Analytics.scala | 90 ++++++++++++++++++- .../apache/spark/graph/AnalyticsSuite.scala | 56 +++++++++++- 2 files changed, 139 insertions(+), 7 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala index f542ec60695af..d7e8282a2d295 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala @@ -2,8 +2,6 @@ package org.apache.spark.graph import org.apache.spark._ - - /** * The Analytics object contains a collection of basic graph analytics * algorithms that operate largely on the graph structure. @@ -204,6 +202,65 @@ object Analytics extends Logging { } // end of connectedComponents + /** + * Compute the number of triangles passing through each vertex. + * + * @param graph + * @tparam VD + * @tparam ED + * @return + */ + def triangleCount[VD: ClassManifest, ED: ClassManifest](rawGraph: Graph[VD,ED]): + Graph[Int, ED] = { + // Remove redundant edges + val graph = rawGraph.groupEdges( (a,b) => a ).cache + + // Construct set representations of the neighborhoods + val nbrSets: VertexSetRDD[VertexSet] = + graph.collectNeighborIds(EdgeDirection.Both).mapValuesWithKeys { (vid, nbrs) => + val set = new VertexSet//(math.ceil(nbrs.size/0.7).toInt) + var i = 0 + while (i < nbrs.size) { + // prevent self cycle + if(nbrs(i) != vid) set.add(nbrs(i)) + i += 1 + } + set + } + // join the sets with the graph + val setGraph: Graph[VertexSet, ED] = graph.outerJoinVertices(nbrSets) { + (vid, _, optSet) => optSet.getOrElse(null) + } + // Edge function computes intersection of smaller vertex with larger vertex + def edgeFunc(et: EdgeTriplet[VertexSet, ED]): Array[(Vid, Int)] = { + assert(et.srcAttr != null) + assert(et.dstAttr != null) + val (smallSet, largeSet) = + if (et.srcAttr.size < et.dstAttr.size) { (et.srcAttr, et.dstAttr) } + else { (et.dstAttr, et.srcAttr) } + 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 } + } + Array((et.srcId, counter), (et.dstId, counter)) + } + // compute the intersection along edges + val counters: VertexSetRDD[Int] = setGraph.mapReduceTriplets(edgeFunc, _+_) + // Merge counters with the graph and divide by two since each triangle is counted twice + graph.outerJoinVertices(counters) { + (vid, _, optCounter: Option[Int]) => + val dblCount = optCounter.getOrElse(0) + // double count should be even (divisible by two) + assert((dblCount & 1) == 0 ) + dblCount/2 + } + + } // end of TriangleCount + + + def main(args: Array[String]) = { val host = args(0) @@ -277,7 +334,7 @@ object Analytics extends Logging { val sc = new SparkContext(host, "PageRank(" + fname + ")") - val graph = GraphLoader.textFile(sc, fname, a => 1.0F, + val graph = GraphLoader.edgeListFile(sc, fname, minEdgePartitions = numEPart, minVertexPartitions = numVPart).cache() val startTime = System.currentTimeMillis @@ -329,7 +386,7 @@ object Analytics extends Logging { val sc = new SparkContext(host, "ConnectedComponents(" + fname + ")") //val graph = GraphLoader.textFile(sc, fname, a => 1.0F) - val graph = GraphLoader.textFile(sc, fname, a => 1.0F, + val graph = GraphLoader.edgeListFile(sc, fname, minEdgePartitions = numEPart, minVertexPartitions = numVPart).cache() val cc = Analytics.connectedComponents(graph) //val cc = if(isDynamic) Analytics.dynamicConnectedComponents(graph, numIter) @@ -338,6 +395,31 @@ object Analytics extends Logging { sc.stop() } + + case "triangles" => { + var numVPart = 4 + var numEPart = 4 + + options.foreach{ + case ("numEPart", v) => numEPart = v.toInt + case ("numVPart", v) => numVPart = v.toInt + case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) + } + println("======================================") + println("| Triangle Count |") + println("--------------------------------------") + val sc = new SparkContext(host, "TriangleCount(" + fname + ")") + //val graph = GraphLoader.textFile(sc, fname, a => 1.0F) + val graph = GraphLoader.edgeListFileUndirected(sc, fname, + minEdgePartitions = numEPart, minVertexPartitions = numVPart).cache() + val triangles = Analytics.triangleCount(graph) + //val cc = if(isDynamic) Analytics.dynamicConnectedComponents(graph, numIter) + // else Analytics.connectedComponents(graph, numIter) + println("Triangles: " + triangles.vertices.map{ case (vid,data) => data.toLong }.reduce(_+_) /3) + + sc.stop() + } + // // case "shortestpath" => { // diff --git a/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala b/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala index 0fb101a08c55a..ac8415a000fe9 100644 --- a/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala @@ -132,7 +132,7 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { val chain1 = (0 until 9).map(x => (x, x+1) ) val chain2 = (10 until 20).map(x => (x, x+1) ) val rawEdges = sc.parallelize(chain1 ++ chain2, 3).map { case (s,d) => (s.toLong, d.toLong) } - val twoChains = Graph(rawEdges) + val twoChains = Graph(rawEdges, 1.0) val ccGraph = Analytics.connectedComponents(twoChains).cache() val vertices = ccGraph.vertices.collect for ( (id, cc) <- vertices ) { @@ -153,7 +153,7 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { val chain1 = (0 until 9).map(x => (x, x+1) ) val chain2 = (10 until 20).map(x => (x, x+1) ) val rawEdges = sc.parallelize(chain1 ++ chain2, 3).map { case (s,d) => (s.toLong, d.toLong) } - val twoChains = Graph(rawEdges).reverse + val twoChains = Graph(rawEdges, true).reverse val ccGraph = Analytics.connectedComponents(twoChains).cache() val vertices = ccGraph.vertices.collect for ( (id, cc) <- vertices ) { @@ -167,8 +167,58 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { else { assert(ccMap(id) === 10) } } } - } // end of chain connected components + } // end of reverse chain connected components + + test("Count a single triangle") { + withSpark(new SparkContext("local", "test")) { sc => + val rawEdges = sc.parallelize(Array( 0L->1L, 1L->2L, 2L->0L ), 2) + val graph = Graph(rawEdges, true).cache + val triangleCount = Analytics.triangleCount(graph) + val verts = triangleCount.vertices + verts.collect.foreach { case (vid, count) => assert(count === 1) } + } + } + test("Count two triangles") { + withSpark(new SparkContext("local", "test")) { sc => + val triangles = Array( 0L -> 1L, 1L -> 2L, 2L -> 0L ) ++ + Array( 0L -> -1L, -1L -> -2L, -2L -> 0L ) + val rawEdges = sc.parallelize(triangles, 2) + val graph = Graph(rawEdges, true).cache + val triangleCount = Analytics.triangleCount(graph) + val verts = triangleCount.vertices + verts.collect.foreach { case (vid, count) => + if(vid == 0) { assert(count === 2) } + else { assert(count === 1) } + } + } + } + test("Count two triangles with bi-directed edges") { + withSpark(new SparkContext("local", "test")) { sc => + val triangles = + Array( 0L -> 1L, 1L -> 2L, 2L -> 0L ) ++ + Array( 0L -> -1L, -1L -> -2L, -2L -> 0L ) + val revTriangles = triangles.map { case (a,b) => (b,a) } + + val rawEdges = sc.parallelize(triangles ++ revTriangles, 2) + val graph = Graph(rawEdges, true).cache + val triangleCount = Analytics.triangleCount(graph) + val verts = triangleCount.vertices + verts.collect.foreach { case (vid, count) => + if(vid == 0) { assert(count === 4) } + else { assert(count === 2) } + } + } + } + test("Count a single triangle with duplicate edges") { + withSpark(new SparkContext("local", "test")) { sc => + val rawEdges = sc.parallelize(Array( 0L->1L, 1L->2L, 2L->0L ) ++ Array( 0L->1L, 1L->2L, 2L->0L ), 2) + val graph = Graph(rawEdges, true).cache + val triangleCount = Analytics.triangleCount(graph) + val verts = triangleCount.vertices + verts.collect.foreach { case (vid, count) => assert(count === 1) } + } + } } // end of AnalyticsSuite From 983810ad6910b84e4ab0f70a3991a9f5768fef17 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Mon, 18 Nov 2013 22:44:23 -0800 Subject: [PATCH 219/531] Now with style. Addressing most of Reynolds comments. --- .../org/apache/spark/graph/Analytics.scala | 32 +++++--- .../scala/org/apache/spark/graph/Graph.scala | 17 ++--- .../org/apache/spark/graph/GraphOps.scala | 13 +++- .../org/apache/spark/graph/VertexSetRDD.scala | 8 +- .../apache/spark/graph/AnalyticsSuite.scala | 75 +++++++++++-------- .../org/apache/spark/graph/GraphSuite.scala | 8 +- 6 files changed, 93 insertions(+), 60 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala index d7e8282a2d295..a5e3b050e503a 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala @@ -205,7 +205,17 @@ object Analytics extends Logging { /** * Compute the number of triangles passing through each vertex. * - * @param graph + * 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` * @tparam VD * @tparam ED * @return @@ -218,11 +228,13 @@ object Analytics extends Logging { // Construct set representations of the neighborhoods val nbrSets: VertexSetRDD[VertexSet] = graph.collectNeighborIds(EdgeDirection.Both).mapValuesWithKeys { (vid, nbrs) => - val set = new VertexSet//(math.ceil(nbrs.size/0.7).toInt) + val set = new VertexSet var i = 0 while (i < nbrs.size) { // prevent self cycle - if(nbrs(i) != vid) set.add(nbrs(i)) + if(nbrs(i) != vid) { + set.add(nbrs(i)) + } i += 1 } set @@ -235,9 +247,11 @@ object Analytics extends Logging { def edgeFunc(et: EdgeTriplet[VertexSet, ED]): Array[(Vid, Int)] = { assert(et.srcAttr != null) assert(et.dstAttr != null) - val (smallSet, largeSet) = - if (et.srcAttr.size < et.dstAttr.size) { (et.srcAttr, et.dstAttr) } - else { (et.dstAttr, et.srcAttr) } + val (smallSet, largeSet) = if (et.srcAttr.size < et.dstAttr.size) { + (et.srcAttr, et.dstAttr) + } else { + (et.dstAttr, et.srcAttr) + } val iter = smallSet.iterator() var counter: Int = 0 while (iter.hasNext) { @@ -247,14 +261,14 @@ object Analytics extends Logging { Array((et.srcId, counter), (et.dstId, counter)) } // compute the intersection along edges - val counters: VertexSetRDD[Int] = setGraph.mapReduceTriplets(edgeFunc, _+_) + val counters: VertexSetRDD[Int] = setGraph.mapReduceTriplets(edgeFunc, _ + _) // Merge counters with the graph and divide by two since each triangle is counted twice graph.outerJoinVertices(counters) { (vid, _, optCounter: Option[Int]) => val dblCount = optCounter.getOrElse(0) // double count should be even (divisible by two) - assert((dblCount & 1) == 0 ) - dblCount/2 + assert((dblCount & 1) == 0) + dblCount / 2 } } // end of TriangleCount diff --git a/graph/src/main/scala/org/apache/spark/graph/Graph.scala b/graph/src/main/scala/org/apache/spark/graph/Graph.scala index 848e580ebd0fc..a953f2cb97802 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Graph.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Graph.scala @@ -61,7 +61,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * type Color = Int * val graph: Graph[Color, Int] = Graph.textFile("hdfs://file.tsv") * val numInvalid = graph.edgesWithVertices() - * .map(e => if(e.src.data == e.dst.data) 1 else 0).sum + * .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 @@ -110,7 +110,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * val rawGraph: Graph[(), ()] = Graph.textFile("hdfs://file") * val root = 42 * var bfsGraph = rawGraph - * .mapVertices[Int]((vid, data) => if(vid == root) 0 else Math.MaxValue) + * .mapVertices[Int]((vid, data) => if (vid == root) 0 else Math.MaxValue) * }}} * */ @@ -349,16 +349,15 @@ object Graph { } /** - * Construct a graph from a collection of edges encoded as vertex id - * pairs. + * Construct a graph from a collection of edges encoded as vertex id pairs. * * @param rawEdges the RDD containing the set of edges in the graph * - * @return a graph with edge attributes containing the count of - * duplicate edges. + * @return a graph with edge attributes containing the count of duplicate edges. */ - def apply[VD: ClassManifest](rawEdges: RDD[(Vid, Vid)], defaultValue: VD): - Graph[VD, Int] = { Graph(rawEdges, defaultValue, false) } + def apply[VD: ClassManifest](rawEdges: RDD[(Vid, Vid)], defaultValue: VD): Graph[VD, Int] = { + Graph(rawEdges, defaultValue, false) + } /** * Construct a graph from a collection of edges encoded as vertex id @@ -377,7 +376,7 @@ object Graph { def apply[VD: ClassManifest](rawEdges: RDD[(Vid, Vid)], defaultValue: VD, uniqueEdges: Boolean): Graph[VD, Int] = { val graph = GraphImpl(rawEdges.map(p => Edge(p._1, p._2, 1)), defaultValue) - if(uniqueEdges) { + if (uniqueEdges) { graph.groupEdges((a,b) => a+b) } else { graph diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala b/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala index 11659ce66d10a..b5047cad11d07 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala @@ -158,13 +158,18 @@ class GraphOps[VD: ClassManifest, ED: ClassManifest](graph: Graph[VD, ED]) { VertexSetRDD[Array[Vid]] = { val nbrs = if (edgeDirection == EdgeDirection.Both) { - graph.mapReduceTriplets[Array[Vid]] ( - et => Array( (et.srcId, Array(et.dstId)), (et.dstId, Array(et.srcId))), _ ++ _ + graph.mapReduceTriplets[Array[Vid]]( + mapFunc = et => Array((et.srcId, Array(et.dstId)), (et.dstId, Array(et.srcId))), + reduceFunc = _ ++ _ ) } else if (edgeDirection == EdgeDirection.Out) { - graph.mapReduceTriplets[Array[Vid]](et => Array((et.srcId, Array(et.dstId))), _ ++ _) + graph.mapReduceTriplets[Array[Vid]]( + mapFunc = et => Array((et.srcId, Array(et.dstId))), + reduceFunc = _ ++ _) } else if (edgeDirection == EdgeDirection.In) { - graph.mapReduceTriplets[Array[Vid]](et => Array((et.dstId, Array(et.srcId))), _ ++ _) + graph.mapReduceTriplets[Array[Vid]]( + mapFunc = et => Array((et.dstId, Array(et.srcId))), + reduceFunc = _ ++ _) } else { throw new SparkException("It doesn't make sense to collect neighbor ids without a direction.") } diff --git a/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala b/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala index 4a056911fda6f..7e7382b7b53a9 100644 --- a/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala @@ -171,7 +171,7 @@ class VertexSetRDD[@specialized V: ClassManifest]( var ind = bs.nextSetBit(0) while(ind >= 0) { val k = index.getValueSafe(ind) - if( cleanPred( (k, oldValues(ind)) ) ) { + if (cleanPred((k, oldValues(ind)))) { newBS.set(ind) } ind = bs.nextSetBit(ind+1) @@ -278,7 +278,7 @@ class VertexSetRDD[@specialized V: ClassManifest]( def zipJoin[W: ClassManifest, Z: ClassManifest](other: VertexSetRDD[W])(f: (Vid, V, W) => Z): VertexSetRDD[Z] = { val cleanF = index.rdd.context.clean(f) - if(index != other.index) { + if (index != other.index) { throw new SparkException("A zipJoin can only be applied to RDDs with the same index!") } val newValuesRDD: RDD[ (Array[Z], BitSet) ] = @@ -315,7 +315,7 @@ class VertexSetRDD[@specialized V: ClassManifest]( def zipJoinFlatMap[W: ClassManifest, Z: ClassManifest](other: VertexSetRDD[W])(f: (Vid, V,W) => Iterator[Z]): RDD[Z] = { val cleanF = index.rdd.context.clean(f) - if(index != other.index) { + if (index != other.index) { throw new SparkException("A zipJoin can only be applied to RDDs with the same index!") } index.rdd.zipPartitions(valuesRDD, other.valuesRDD) { (indexIter, thisIter, otherIter) => @@ -351,7 +351,7 @@ class VertexSetRDD[@specialized V: ClassManifest]( */ def leftZipJoin[W: ClassManifest, Z: ClassManifest](other: VertexSetRDD[W])(f: (Vid, V, Option[W]) => Z): VertexSetRDD[Z] = { - if(index != other.index) { + if (index != other.index) { throw new SparkException("A zipJoin can only be applied to RDDs with the same index!") } val cleanF = index.rdd.context.clean(f) diff --git a/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala b/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala index ac8415a000fe9..e1ff8df4eaae0 100644 --- a/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala @@ -12,30 +12,30 @@ import org.apache.spark.graph.util.GraphGenerators object GridPageRank { - def apply(nRows: Int, nCols: Int, nIter: Int, resetProb: Double) = { + def apply(nRows: Int, nCols: Int, nIter: Int, resetProb: Double) = { val inNbrs = Array.fill(nRows * nCols)(collection.mutable.MutableList.empty[Int]) val outDegree = Array.fill(nRows * nCols)(0) // Convert row column address into vertex ids (row major order) - def sub2ind(r: Int, c: Int): Int = r * nCols + c + def sub2ind(r: Int, c: Int): Int = r * nCols + c // Make the grid graph - for(r <- 0 until nRows; c <- 0 until nCols){ + for (r <- 0 until nRows; c <- 0 until nCols) { val ind = sub2ind(r,c) - if(r+1 < nRows) { + if (r+1 < nRows) { outDegree(ind) += 1 - inNbrs(sub2ind(r+1,c)) += ind + inNbrs(sub2ind(r+1,c)) += ind } - if(c+1 < nCols) { + if (c+1 < nCols) { outDegree(ind) += 1 - inNbrs(sub2ind(r,c+1)) += ind + inNbrs(sub2ind(r,c+1)) += ind } } // compute the pagerank var pr = Array.fill(nRows * nCols)(resetProb) - for(iter <- 0 until nIter) { + for (iter <- 0 until nIter) { val oldPr = pr pr = new Array[Double](nRows * nCols) - for(ind <- 0 until (nRows * nCols)) { - pr(ind) = resetProb + (1.0 - resetProb) * + for (ind <- 0 until (nRows * nCols)) { + pr(ind) = resetProb + (1.0 - resetProb) * inNbrs(ind).map( nbr => oldPr(nbr) / outDegree(nbr)).sum } } @@ -58,13 +58,13 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { val resetProb = 0.15 val prGraph1 = Analytics.pagerank(starGraph, 1, resetProb) val prGraph2 = Analytics.pagerank(starGraph, 2, resetProb) - + val notMatching = prGraph1.vertices.zipJoin(prGraph2.vertices) { (vid, pr1, pr2) => if (pr1 != pr2) { 1 } else { 0 } }.map { case (vid, test) => test }.sum assert(notMatching === 0) prGraph2.vertices.foreach(println(_)) - val errors = prGraph2.vertices.map{ case (vid, pr) => + val errors = prGraph2.vertices.map { case (vid, pr) => val correct = (vid > 0 && pr == resetProb) || (vid == 0 && math.abs(pr - (resetProb + (1.0 - resetProb) * (resetProb * (nVertices - 1)) )) < 1.0E-5) if ( !correct ) { 1 } else { 0 } @@ -141,9 +141,12 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { } val ccMap = vertices.toMap println(ccMap) - for( id <- 0 until 20 ) { - if(id < 10) { assert(ccMap(id) === 0) } - else { assert(ccMap(id) === 10) } + for (id <- 0 until 20) { + if (id < 10) { + assert(ccMap(id) === 0) + } else { + assert(ccMap(id) === 10) + } } } } // end of chain connected components @@ -157,14 +160,20 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { val ccGraph = Analytics.connectedComponents(twoChains).cache() val vertices = ccGraph.vertices.collect for ( (id, cc) <- vertices ) { - if(id < 10) { assert(cc === 0) } - else { assert(cc === 10) } + if (id < 10) { + assert(cc === 0) + } else { + assert(cc === 10) + } } val ccMap = vertices.toMap println(ccMap) - for( id <- 0 until 20 ) { - if(id < 10) { assert(ccMap(id) === 0) } - else { assert(ccMap(id) === 10) } + for ( id <- 0 until 20 ) { + if (id < 10) { + assert(ccMap(id) === 0) + } else { + assert(ccMap(id) === 10) + } } } } // end of reverse chain connected components @@ -181,15 +190,18 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { test("Count two triangles") { withSpark(new SparkContext("local", "test")) { sc => - val triangles = Array( 0L -> 1L, 1L -> 2L, 2L -> 0L ) ++ - Array( 0L -> -1L, -1L -> -2L, -2L -> 0L ) + val triangles = Array(0L -> 1L, 1L -> 2L, 2L -> 0L) ++ + Array(0L -> -1L, -1L -> -2L, -2L -> 0L) val rawEdges = sc.parallelize(triangles, 2) val graph = Graph(rawEdges, true).cache val triangleCount = Analytics.triangleCount(graph) val verts = triangleCount.vertices verts.collect.foreach { case (vid, count) => - if(vid == 0) { assert(count === 2) } - else { assert(count === 1) } + if (vid == 0) { + assert(count === 2) + } else { + assert(count === 1) + } } } } @@ -197,24 +209,27 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { test("Count two triangles with bi-directed edges") { withSpark(new SparkContext("local", "test")) { sc => val triangles = - Array( 0L -> 1L, 1L -> 2L, 2L -> 0L ) ++ - Array( 0L -> -1L, -1L -> -2L, -2L -> 0L ) + Array(0L -> 1L, 1L -> 2L, 2L -> 0L) ++ + Array(0L -> -1L, -1L -> -2L, -2L -> 0L) val revTriangles = triangles.map { case (a,b) => (b,a) } - val rawEdges = sc.parallelize(triangles ++ revTriangles, 2) val graph = Graph(rawEdges, true).cache val triangleCount = Analytics.triangleCount(graph) val verts = triangleCount.vertices verts.collect.foreach { case (vid, count) => - if(vid == 0) { assert(count === 4) } - else { assert(count === 2) } + if (vid == 0) { + assert(count === 4) + } else { + assert(count === 2) + } } } } test("Count a single triangle with duplicate edges") { withSpark(new SparkContext("local", "test")) { sc => - val rawEdges = sc.parallelize(Array( 0L->1L, 1L->2L, 2L->0L ) ++ Array( 0L->1L, 1L->2L, 2L->0L ), 2) + val rawEdges = sc.parallelize(Array(0L -> 1L, 1L -> 2L, 2L -> 0L) ++ + Array(0L -> 1L, 1L -> 2L, 2L -> 0L), 2) val graph = Graph(rawEdges, true).cache val triangleCount = Analytics.triangleCount(graph) val verts = triangleCount.vertices diff --git a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala index b0d248b6511d5..ff1cd565991b2 100644 --- a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala @@ -16,7 +16,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { val rawEdges = (0L to 100L).zip((1L to 99L) :+ 0L) val edges = sc.parallelize(rawEdges) val graph = Graph(edges, 1.0F) - assert( graph.edges.count() === rawEdges.size ) + assert(graph.edges.count() === rawEdges.size) } } @@ -29,8 +29,8 @@ class GraphSuite extends FunSuite with LocalSparkContext { assert( graph.edges.count() === rawEdges.size ) assert( graph.vertices.count() === 100) graph.triplets.map { et => - assert( (et.srcId < 10 && et.srcAttr) || (et.srcId >= 10 && !et.srcAttr) ) - assert( (et.dstId < 10 && et.dstAttr) || (et.dstId >= 10 && !et.dstAttr) ) + assert((et.srcId < 10 && et.srcAttr) || (et.srcId >= 10 && !et.srcAttr)) + assert((et.dstId < 10 && et.dstAttr) || (et.dstId >= 10 && !et.dstAttr)) } } } @@ -111,7 +111,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { nbrs.collect.foreach { case (vid, nbrs) => val s = nbrs.toSet assert(s.contains((vid + 1) % 100)) - assert(s.contains(if (vid > 0) { vid - 1 } else { 99 })) + assert(s.contains(if (vid > 0) vid - 1 else 99 )) } } } From 18700b6e741ea397a8c184d292f6ea400f896d49 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Mon, 18 Nov 2013 23:27:08 -0800 Subject: [PATCH 220/531] Switching mapReduceTriplets mapFunction to return iterator instead of array to allow optimizations of the returned object. --- .../org/apache/spark/graph/Analytics.scala | 18 +++++++++------- .../scala/org/apache/spark/graph/Graph.scala | 2 +- .../org/apache/spark/graph/GraphLab.scala | 10 ++++----- .../org/apache/spark/graph/GraphOps.scala | 21 +++++++++---------- .../scala/org/apache/spark/graph/Pregel.scala | 10 +++++---- .../apache/spark/graph/impl/GraphImpl.scala | 4 ++-- .../org/apache/spark/graph/GraphSuite.scala | 2 +- 7 files changed, 35 insertions(+), 32 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala index a5e3b050e503a..c465b4b2a715f 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala @@ -75,7 +75,7 @@ object Analytics extends Logging { def vertexProgram(id: Vid, attr: Double, msgSum: Double): Double = resetProb + (1.0 - resetProb) * msgSum def sendMessage(edge: EdgeTriplet[Double, Double]) = - Array((edge.dstId, edge.srcAttr * edge.attr)) + Iterator((edge.dstId, edge.srcAttr * edge.attr)) def messageCombiner(a: Double, b: Double): Double = a + b // The initial message received by all vertices in PageRank val initialMessage = 0.0 @@ -151,8 +151,10 @@ object Analytics extends Logging { } def sendMessage(edge: EdgeTriplet[(Double, Double), Double]) = { if (edge.srcAttr._2 > tol) { - Array((edge.dstId, edge.srcAttr._2 * edge.attr)) - } else { Array.empty[(Vid, Double)] } + Iterator((edge.dstId, edge.srcAttr._2 * edge.attr)) + } else { + Iterator.empty + } } def messageCombiner(a: Double, b: Double): Double = a + b // The initial message received by all vertices in PageRank @@ -186,11 +188,11 @@ object Analytics extends Logging { def sendMessage(edge: EdgeTriplet[Vid, ED]) = { if (edge.srcAttr < edge.dstAttr) { - Array((edge.dstId, edge.srcAttr)) + Iterator((edge.dstId, edge.srcAttr)) } else if (edge.srcAttr > edge.dstAttr) { - Array((edge.srcId, edge.dstAttr)) + Iterator((edge.srcId, edge.dstAttr)) } else { - Array.empty[(Vid, Vid)] + Iterator.empty } } val initialMessage = Long.MaxValue @@ -244,7 +246,7 @@ object Analytics extends Logging { (vid, _, optSet) => optSet.getOrElse(null) } // Edge function computes intersection of smaller vertex with larger vertex - def edgeFunc(et: EdgeTriplet[VertexSet, ED]): Array[(Vid, Int)] = { + def edgeFunc(et: EdgeTriplet[VertexSet, ED]): Iterator[(Vid, Int)] = { assert(et.srcAttr != null) assert(et.dstAttr != null) val (smallSet, largeSet) = if (et.srcAttr.size < et.dstAttr.size) { @@ -258,7 +260,7 @@ object Analytics extends Logging { val vid = iter.next if (vid != et.srcId && vid != et.dstId && largeSet.contains(vid)) { counter += 1 } } - Array((et.srcId, counter), (et.dstId, counter)) + Iterator((et.srcId, counter), (et.dstId, counter)) } // compute the intersection along edges val counters: VertexSetRDD[Int] = setGraph.mapReduceTriplets(edgeFunc, _ + _) diff --git a/graph/src/main/scala/org/apache/spark/graph/Graph.scala b/graph/src/main/scala/org/apache/spark/graph/Graph.scala index a953f2cb97802..62c4834a68471 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Graph.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Graph.scala @@ -277,7 +277,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * */ def mapReduceTriplets[A: ClassManifest]( - mapFunc: EdgeTriplet[VD, ED] => Array[(Vid, A)], + mapFunc: EdgeTriplet[VD, ED] => Iterator[(Vid, A)], reduceFunc: (A, A) => A) : VertexSetRDD[A] diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala b/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala index b8503ab7fdb6c..bf1f4168da54b 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala @@ -34,7 +34,7 @@ object GraphLab { * @return the resulting graph after the algorithm converges */ def apply[VD: ClassManifest, ED: ClassManifest, A: ClassManifest] - (graph: Graph[VD, ED], numIter: Int, + (graph: Graph[VD, ED], numIter: Int, gatherDirection: EdgeDirection = EdgeDirection.In, scatterDirection: EdgeDirection = EdgeDirection.Out) (gatherFunc: (Vid, EdgeTriplet[VD, ED]) => A, @@ -100,20 +100,20 @@ object GraphLab { val gathered: RDD[(Vid, A)] = activeGraph.aggregateNeighbors(gather, mergeFunc, gatherDirection) - // Apply + // Apply activeGraph = activeGraph.outerJoinVertices(gathered)(apply).cache() - + // Scatter is basically a gather in the opposite direction so we reverse the edge direction // activeGraph: Graph[(Boolean, VD), ED] - val scattered: RDD[(Vid, Boolean)] = + val scattered: RDD[(Vid, Boolean)] = activeGraph.aggregateNeighbors(scatter, _ || _, scatterDirection.reverse) activeGraph = activeGraph.joinVertices(scattered)(applyActive).cache() // Calculate the number of active vertices - numActive = activeGraph.vertices.map{ + numActive = activeGraph.vertices.map{ case (vid, data) => if (data._1) 1 else 0 }.reduce(_ + _) println("Number active vertices: " + numActive) diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala b/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala index b5047cad11d07..b7e28186c6527 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala @@ -61,11 +61,11 @@ class GraphOps[VD: ClassManifest, ED: ClassManifest](graph: Graph[VD, ED]) { */ private def degreesRDD(edgeDirection: EdgeDirection): VertexSetRDD[Int] = { if (edgeDirection == EdgeDirection.In) { - graph.mapReduceTriplets(et => Array((et.dstId,1)), _+_) + graph.mapReduceTriplets(et => Iterator((et.dstId,1)), _ + _) } else if (edgeDirection == EdgeDirection.Out) { - graph.mapReduceTriplets(et => Array((et.srcId,1)), _+_) + graph.mapReduceTriplets(et => Iterator((et.srcId,1)), _ + _) } else { // EdgeDirection.both - graph.mapReduceTriplets(et => Array((et.srcId,1), (et.dstId,1)), _+_) + graph.mapReduceTriplets(et => Iterator((et.srcId,1), (et.dstId,1)), _ + _) } } @@ -133,11 +133,10 @@ class GraphOps[VD: ClassManifest, ED: ClassManifest](graph: Graph[VD, ED]) { } else { Option.empty[A] } // construct the return array (src, dst) match { - case (None, None) => Array.empty[(Vid, A)] - case (Some(srcA),None) => Array((et.srcId, srcA)) - case (None, Some(dstA)) => Array((et.dstId, dstA)) - case (Some(srcA), Some(dstA)) => - Array((et.srcId, srcA), (et.dstId, dstA)) + 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)) } } @@ -159,16 +158,16 @@ class GraphOps[VD: ClassManifest, ED: ClassManifest](graph: Graph[VD, ED]) { val nbrs = if (edgeDirection == EdgeDirection.Both) { graph.mapReduceTriplets[Array[Vid]]( - mapFunc = et => Array((et.srcId, Array(et.dstId)), (et.dstId, Array(et.srcId))), + mapFunc = et => Iterator((et.srcId, Array(et.dstId)), (et.dstId, Array(et.srcId))), reduceFunc = _ ++ _ ) } else if (edgeDirection == EdgeDirection.Out) { graph.mapReduceTriplets[Array[Vid]]( - mapFunc = et => Array((et.srcId, Array(et.dstId))), + mapFunc = et => Iterator((et.srcId, Array(et.dstId))), reduceFunc = _ ++ _) } else if (edgeDirection == EdgeDirection.In) { graph.mapReduceTriplets[Array[Vid]]( - mapFunc = et => Array((et.dstId, Array(et.srcId))), + 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.") diff --git a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala index 3b4d3c0df2a51..29d6225f33838 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala @@ -93,7 +93,7 @@ object Pregel { def apply[VD: ClassManifest, ED: ClassManifest, A: ClassManifest] (graph: Graph[VD, ED], initialMsg: A, numIter: Int)( vprog: (Vid, VD, A) => VD, - sendMsg: EdgeTriplet[VD, ED] => Array[(Vid,A)], + sendMsg: EdgeTriplet[VD, ED] => Iterator[(Vid,A)], mergeMsg: (A, A) => A) : Graph[VD, ED] = { @@ -163,7 +163,7 @@ object Pregel { def apply[VD: ClassManifest, ED: ClassManifest, A: ClassManifest] (graph: Graph[VD, ED], initialMsg: A)( vprog: (Vid, VD, A) => VD, - sendMsg: EdgeTriplet[VD, ED] => Array[(Vid,A)], + sendMsg: EdgeTriplet[VD, ED] => Iterator[(Vid,A)], mergeMsg: (A, A) => A) : Graph[VD, ED] = { @@ -174,7 +174,7 @@ object Pregel { } } - def sendMsgFun(edge: EdgeTriplet[(VD,Boolean), ED]): Array[(Vid, A)] = { + def sendMsgFun(edge: EdgeTriplet[(VD,Boolean), ED]): Iterator[(Vid, A)] = { if(edge.srcAttr._2) { val et = new EdgeTriplet[VD, ED] et.srcId = edge.srcId @@ -183,7 +183,9 @@ object Pregel { et.dstAttr = edge.dstAttr._1 et.attr = edge.attr sendMsg(et) - } else { Array.empty[(Vid,A)] } + } else { + Iterator.empty + } } var g = graph.mapVertices( (vid, vdata) => (vprog(vid, vdata, initialMsg), true) ) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index 24b60c7a9bf16..f995878bff4a7 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -233,7 +233,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( ////////////////////////////////////////////////////////////////////////////////////////////////// override def mapReduceTriplets[A: ClassManifest]( - mapFunc: EdgeTriplet[VD, ED] => Array[(Vid, A)], + mapFunc: EdgeTriplet[VD, ED] => Iterator[(Vid, A)], reduceFunc: (A, A) => A) : VertexSetRDD[A] = GraphImpl.mapReduceTriplets(this, mapFunc, reduceFunc) @@ -402,7 +402,7 @@ object GraphImpl { def mapReduceTriplets[VD: ClassManifest, ED: ClassManifest, A: ClassManifest]( g: GraphImpl[VD, ED], - mapFunc: EdgeTriplet[VD, ED] => Array[(Vid, A)], + mapFunc: EdgeTriplet[VD, ED] => Iterator[(Vid, A)], reduceFunc: (A, A) => A): VertexSetRDD[A] = { ClosureCleaner.clean(mapFunc) ClosureCleaner.clean(reduceFunc) diff --git a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala index ff1cd565991b2..da7b2bdd997cd 100644 --- a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala @@ -54,7 +54,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { val star = Graph(sc.parallelize((1 to n).map(x => (0: Vid, x: Vid))), 0) val starDeg = star.joinVertices(star.degrees){ (vid, oldV, deg) => deg } val neighborDegreeSums = starDeg.mapReduceTriplets( - edge => Array((edge.srcId, edge.dstAttr), (edge.dstId, edge.srcAttr)), + edge => Iterator((edge.srcId, edge.dstAttr), (edge.dstId, edge.srcAttr)), (a: Int, b: Int) => a + b) assert(neighborDegreeSums.collect().toSet === (0 to n).map(x => (x, n)).toSet) } From ae4ffc319a0da2c194e70cbe634553b98a13fa07 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Mon, 18 Nov 2013 23:40:32 -0800 Subject: [PATCH 221/531] Setting the initial vertex set size to be small. --- graph/src/main/scala/org/apache/spark/graph/Analytics.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala index c465b4b2a715f..d3d65942d0343 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala @@ -230,7 +230,7 @@ object Analytics extends Logging { // Construct set representations of the neighborhoods val nbrSets: VertexSetRDD[VertexSet] = graph.collectNeighborIds(EdgeDirection.Both).mapValuesWithKeys { (vid, nbrs) => - val set = new VertexSet + val set = new VertexSet(4) var i = 0 while (i < nbrs.size) { // prevent self cycle From 12cb19b1c1618f82f56fcfa7c4662c42bcc4e934 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Tue, 19 Nov 2013 17:02:24 -0800 Subject: [PATCH 222/531] Adding comments and addressing comments. --- .../scala/org/apache/spark/graph/Graph.scala | 45 ++--------------- .../org/apache/spark/graph/VertexSetRDD.scala | 8 +-- .../spark/graph/impl/EdgePartition.scala | 50 +++++++++++++++++-- 3 files changed, 54 insertions(+), 49 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/Graph.scala b/graph/src/main/scala/org/apache/spark/graph/Graph.scala index 62c4834a68471..9c01d60b591a2 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Graph.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Graph.scala @@ -74,7 +74,6 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { def persist(newLevel: StorageLevel): Graph[VD, ED] - /** * Return a graph that is cached when first created. This is used to * pin a graph in memory enabling multiple queries to reuse the same @@ -84,14 +83,11 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { */ def cache(): Graph[VD, ED] - /** * Compute statistics describing the graph representation. */ def statistics: Map[String, Any] - - /** * Construct a new graph where each vertex value has been * transformed by the map function. @@ -160,9 +156,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * }}} * */ - def mapTriplets[ED2: ClassManifest]( - map: EdgeTriplet[VD, ED] => ED2): Graph[VD, ED2] - + def mapTriplets[ED2: ClassManifest](map: EdgeTriplet[VD, ED] => ED2): Graph[VD, ED2] /** * Construct a new graph with all the edges reversed. If this graph @@ -172,7 +166,6 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { */ def reverse: Graph[VD, ED] - /** * This function takes a vertex and edge predicate and constructs * the subgraph that consists of vertices and edges that satisfy the @@ -198,35 +191,6 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { def subgraph(epred: EdgeTriplet[VD,ED] => Boolean = (x => true), vpred: (Vid, VD) => Boolean = ((v,d) => true) ): Graph[VD, ED] - - - /** - * groupEdgeTriplets is used to merge multiple edges that have the - * same source and destination vertex into a single edge. The user - * supplied function is applied to each directed pair of vertices - * (u, v) and has access to all EdgeTriplets - * - * {e: for all e in E where e.src = u and e.dst = v} - * - * This function is identical to - * [[org.apache.spark.graph.Graph.groupEdges]] except that this - * function provides the user-supplied function with an iterator - * over EdgeTriplets, which contain the vertex data, whereas - * groupEdges provides the user-supplied function with an iterator - * over Edges, which only contain the vertex IDs. - * - * @tparam ED2 the type of the resulting edge data after grouping - * - * @param f the user supplied function to merge multiple EdgeTriplets - * into a single ED2 object - * - * @return Graph[VD,ED2] The resulting graph with a single Edge for each - * source, dest vertex pair. - * - */ - def groupEdgeTriplets[ED2: ClassManifest](f: Iterator[EdgeTriplet[VD,ED]] => ED2 ): Graph[VD,ED2] - - /** * This function merges multiple edges between two vertices into a * single Edge. See @@ -235,15 +199,14 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * * @tparam ED2 the type of the resulting edge data after grouping. * - * @param f the user supplied function to merge multiple Edges - * into a single ED2 object. + * @param f the user supplied commutative associative function to merge + * edge attributes for duplicate edges. * * @return Graph[VD,ED2] The resulting graph with a single Edge for * each source, dest vertex pair. */ def groupEdges(merge: (ED, ED) => ED): Graph[VD,ED] - /** * The mapReduceTriplets function is used to compute statistics * about the neighboring edges and vertices of each vertex. The @@ -281,7 +244,6 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { reduceFunc: (A, A) => A) : VertexSetRDD[A] - /** * Join the vertices with an RDD and then apply a function from the * the vertex and RDD entry to a new vertex value and type. The @@ -315,7 +277,6 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { (mapFunc: (Vid, VD, Option[U]) => VD2) : Graph[VD2, ED] - // 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) diff --git a/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala b/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala index 7e7382b7b53a9..0f2006a42d7c6 100644 --- a/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala @@ -645,11 +645,11 @@ object VertexSetRDD { } /** - * Cosntruct an VertexSetRDD with all vertices initialized to the default value. + * Create a VertexSetRDD with all vertices initialized to the default value. * - * @param index - * @param defaultValue - * @tparam V + * @param index an index over the set of vertices + * @param defaultValue the default value to use when initializing the vertices + * @tparam V the type of the vertex attribute * @return */ def apply[V: ClassManifest](index: VertexSetIndex, defaultValue: V): VertexSetRDD[V] = { diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala b/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala index 9ecf531d50506..eb3fd60d74f54 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala @@ -3,17 +3,37 @@ package org.apache.spark.graph.impl import org.apache.spark.graph._ import org.apache.spark.util.collection.OpenHashMap - /** - * A partition of edges in 3 large columnar arrays. + * A collection of edges stored in 3 large columnar arrays (src, dst, attribute). + * + * @param srcIds the source vertex id of each edge + * @param dstIds the destination vertex id of each edge + * @param data the attribute associated with each edge + * @tparam ED the edge attribute type. */ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED: ClassManifest]( val srcIds: Array[Vid], val dstIds: Array[Vid], val data: Array[ED]) { + /** + * Reverse all the edges in this partition. + * + * @note No new data structures are created. + * + * @return a new edge partition with all edges reversed. + */ def reverse: EdgePartition[ED] = new EdgePartition(dstIds, srcIds, data) + /** + * Construct a new edge partition by applying the function f to all + * edges in this partition. + * + * @param f a function from an edge to a new attribute + * @tparam ED2 the type of the new attribute + * @return a new edge partition with the result of the function `f` + * applied to each edge + */ def map[ED2: ClassManifest](f: Edge[ED] => ED2): EdgePartition[ED2] = { val newData = new Array[ED2](data.size) val edge = new Edge[ED]() @@ -29,6 +49,11 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) new EdgePartition(srcIds, dstIds, newData) } + /** + * Apply the function f to all edges in this partition. + * + * @param f an external state mutating user defined function. + */ def foreach(f: Edge[ED] => Unit) { val edge = new Edge[ED] val size = data.size @@ -42,9 +67,18 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) } } + /** + * Merge all the edges with the same src and dest id into a single + * edge using the `merge` function + * + * @param merge a commutative associative merge operation + * @return a new edge partition without duplicate edges + */ def groupEdges(merge: (ED, ED) => ED): EdgePartition[ED] = { - val agg = new OpenHashMap[(Vid,Vid), ED] //(math.ceil((data.size + 1)/0.7).toInt) + // Aggregate all matching edges in a hashmap + val agg = new OpenHashMap[(Vid,Vid), ED] foreach { e => agg.setMerge((e.srcId, e.dstId), e.attr, merge) } + // Populate new srcId, dstId, and data, arrays val newSrcIds = new Array[Vid](agg.size) val newDstIds = new Array[Vid](agg.size) val newData = new Array[ED](agg.size) @@ -58,8 +92,18 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) new EdgePartition(newSrcIds, newDstIds, newData) } + /** + * The number of edges in this partition + * + * @return size of the partition + */ def size: Int = srcIds.size + /** + * Get an iterator over the edges in this partition. + * + * @return an iterator over edges in the partition + */ def iterator = new Iterator[Edge[ED]] { private[this] val edge = new Edge[ED] private[this] var pos = 0 From de3d6ee5a7ee5ed2e0369f05ed045a0cdc6b1668 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Tue, 19 Nov 2013 22:03:49 -0800 Subject: [PATCH 223/531] Fixing build after merging upstream changes. --- .../org/apache/spark/graph/Analytics.scala | 20 ++--- .../scala/org/apache/spark/graph/Graph.scala | 75 ++++++++++++------- .../org/apache/spark/graph/GraphLoader.scala | 3 +- .../apache/spark/graph/impl/GraphImpl.scala | 46 ++++++------ .../spark/graph/util/GraphGenerators.scala | 34 ++++----- 5 files changed, 98 insertions(+), 80 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala index d3d65942d0343..b1feb5e887e58 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala @@ -351,7 +351,7 @@ object Analytics extends Logging { val sc = new SparkContext(host, "PageRank(" + fname + ")") val graph = GraphLoader.edgeListFile(sc, fname, - minEdgePartitions = numEPart, minVertexPartitions = numVPart).cache() + minEdgePartitions = numEPart).cache() val startTime = System.currentTimeMillis logInfo("GRAPHX: starting tasks") @@ -401,14 +401,10 @@ object Analytics extends Logging { println("======================================") val sc = new SparkContext(host, "ConnectedComponents(" + fname + ")") - //val graph = GraphLoader.textFile(sc, fname, a => 1.0F) val graph = GraphLoader.edgeListFile(sc, fname, - minEdgePartitions = numEPart, minVertexPartitions = numVPart).cache() + minEdgePartitions = numEPart).cache() val cc = Analytics.connectedComponents(graph) - //val cc = if(isDynamic) Analytics.dynamicConnectedComponents(graph, numIter) - // else Analytics.connectedComponents(graph, numIter) println("Components: " + cc.vertices.map{ case (vid,data) => data}.distinct()) - sc.stop() } @@ -425,14 +421,12 @@ object Analytics extends Logging { println("| Triangle Count |") println("--------------------------------------") val sc = new SparkContext(host, "TriangleCount(" + fname + ")") - //val graph = GraphLoader.textFile(sc, fname, a => 1.0F) - val graph = GraphLoader.edgeListFileUndirected(sc, fname, - minEdgePartitions = numEPart, minVertexPartitions = numVPart).cache() + val graph = GraphLoader.edgeListFile(sc, fname, canonicalOrientation = true, + minEdgePartitions = numEPart).cache() val triangles = Analytics.triangleCount(graph) - //val cc = if(isDynamic) Analytics.dynamicConnectedComponents(graph, numIter) - // else Analytics.connectedComponents(graph, numIter) - println("Triangles: " + triangles.vertices.map{ case (vid,data) => data.toLong }.reduce(_+_) /3) - + println("Triangles: " + triangles.vertices.map { + case (vid,data) => data.toLong + }.reduce(_+_) / 3) sc.stop() } diff --git a/graph/src/main/scala/org/apache/spark/graph/Graph.scala b/graph/src/main/scala/org/apache/spark/graph/Graph.scala index 9c01d60b591a2..42ede2d420a0b 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Graph.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Graph.scala @@ -295,20 +295,6 @@ object Graph { import org.apache.spark.graph.impl._ import org.apache.spark.SparkContext._ - /** - * Construct a graph from a collection of edges. - * - * @param edges the RDD containing the set of edges in the graph - * @param defaultValue the default vertex attribute to use for each vertex - * - * @return a graph with edge attributes described by `edges` and vertices - * given by all vertices in `edges` with value `defaultValue` - */ - def apply[VD: ClassManifest, ED: ClassManifest]( - edges: RDD[Edge[ED]], defaultValue: VD): Graph[VD, ED] = { - GraphImpl(edges, defaultValue) - } - /** * Construct a graph from a collection of edges encoded as vertex id pairs. * @@ -317,7 +303,7 @@ object Graph { * @return a graph with edge attributes containing the count of duplicate edges. */ def apply[VD: ClassManifest](rawEdges: RDD[(Vid, Vid)], defaultValue: VD): Graph[VD, Int] = { - Graph(rawEdges, defaultValue, false) + Graph(rawEdges, defaultValue, false, RandomVertexCut()) } /** @@ -334,9 +320,14 @@ object Graph { * attributes containing the total degree of each vertex. * */ - def apply[VD: ClassManifest](rawEdges: RDD[(Vid, Vid)], defaultValue: VD, uniqueEdges: Boolean): + def apply[VD: ClassManifest]( + rawEdges: RDD[(Vid, Vid)], + defaultValue: VD, + uniqueEdges: Boolean, + partitionStrategy: PartitionStrategy): Graph[VD, Int] = { - val graph = GraphImpl(rawEdges.map(p => Edge(p._1, p._2, 1)), defaultValue) + val edges = rawEdges.map(p => Edge(p._1, p._2, 1)) + val graph = GraphImpl(edges, defaultValue, partitionStrategy) if (uniqueEdges) { graph.groupEdges((a,b) => a+b) } else { @@ -344,6 +335,37 @@ object Graph { } } + /** + * Construct a graph from a collection of edges. + * + * @param edges the RDD containing the set of edges in the graph + * @param defaultValue the default vertex attribute to use for each vertex + * + * @return a graph with edge attributes described by `edges` and vertices + * given by all vertices in `edges` with value `defaultValue` + */ + def apply[VD: ClassManifest, ED: ClassManifest]( + edges: RDD[Edge[ED]], + defaultValue: VD): Graph[VD, ED] = { + Graph(edges, defaultValue, RandomVertexCut()) + } + + /** + * Construct a graph from a collection of edges. + * + * @param edges the RDD containing the set of edges in the graph + * @param defaultValue the default vertex attribute to use for each vertex + * + * @return a graph with edge attributes described by `edges` and vertices + * given by all vertices in `edges` with value `defaultValue` + */ + def apply[VD: ClassManifest, ED: ClassManifest]( + edges: RDD[Edge[ED]], + defaultValue: VD, + partitionStrategy: PartitionStrategy): Graph[VD, ED] = { + GraphImpl(edges, defaultValue, partitionStrategy) + } + /** * Construct a graph from a collection attributed vertices and * edges. @@ -362,32 +384,30 @@ object Graph { vertices: RDD[(Vid,VD)], edges: RDD[Edge[ED]]): Graph[VD, ED] = { val defaultAttr: VD = null.asInstanceOf[VD] - Graph(vertices, edges, defaultAttr, (a:VD,b:VD) => a) + Graph(vertices, edges, defaultAttr, (a:VD,b:VD) => a, RandomVertexCut()) } - - /** * Construct a graph from a collection attributed vertices and * edges. Duplicate vertices are combined using the `mergeFunc` and * vertices found in the edge collection but not in the input * vertices are the default attribute `defautVertexAttr`. * + * @note Duplicate vertices are removed arbitrarily . + * * @tparam VD the vertex attribute type * @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 mergeFunc the function used to merge duplicate vertices - * in the `vertices` collection. + * vertices that are mentioned in `edges` but not in `vertices` * */ def apply[VD: ClassManifest, ED: ClassManifest]( vertices: RDD[(Vid,VD)], edges: RDD[Edge[ED]], defaultVertexAttr: VD): Graph[VD, ED] = { - GraphImpl(vertices, edges, defaultVertexAttr, (a,b) => a) + Graph(vertices, edges, defaultVertexAttr, (a,b) => a, RandomVertexCut()) } /** @@ -404,14 +424,17 @@ object Graph { * vertices that are mentioned in `edges` but not in `vertices * @param mergeFunc the function used to merge duplicate vertices * in the `vertices` collection. + * @param partitionStrategy the partition strategy to use when + * partitioning the edges. * */ def apply[VD: ClassManifest, ED: ClassManifest]( vertices: RDD[(Vid,VD)], edges: RDD[Edge[ED]], defaultVertexAttr: VD, - mergeFunc: (VD, VD) => VD): Graph[VD, ED] = { - GraphImpl(vertices, edges, defaultVertexAttr, mergeFunc) + mergeFunc: (VD, VD) => VD, + partitionStrategy: PartitionStrategy): Graph[VD, ED] = { + GraphImpl(vertices, edges, defaultVertexAttr, mergeFunc, partitionStrategy) } /** diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala b/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala index fadc58284c318..d97c028faae35 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala @@ -26,7 +26,8 @@ object GraphLoader { path: String, edgeParser: Array[String] => ED, minEdgePartitions: Int = 1, - partitionStrategy: PartitionStrategy = RandomVertexCut()): GraphImpl[Int, ED] = { + partitionStrategy: PartitionStrategy = RandomVertexCut()): + Graph[Int, ED] = { // Parse the edge data table val edges = sc.textFile(path, minEdgePartitions).mapPartitions( iter => iter.filter(line => !line.isEmpty && line(0) != '#').map { line => diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index f995878bff4a7..650bb515b5abf 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -255,7 +255,7 @@ object GraphImpl { defaultValue: VD, partitionStrategy: PartitionStrategy): GraphImpl[VD, ED] = { - val etable = createETable(edges).cache + val etable = createETable(edges, partitionStrategy).cache // Get the set of all vids val vids = etable.mapPartitions( iter => { val (pid, epart) = iter.next() @@ -271,28 +271,28 @@ object GraphImpl { new GraphImpl(vtable, vid2pid, localVidMap, etable, partitionStrategy) } - def apply[VD: ClassManifest, ED: ClassManifest]( - vertices: RDD[(Vid, VD)], - edges: RDD[Edge[ED]], - defaultVertexAttr: VD): GraphImpl[VD,ED] = { - apply(vertices, edges, defaultVertexAttr, (a:VD, b:VD) => a, RandomVertexCut()) - } - - def apply[VD: ClassManifest, ED: ClassManifest]( - vertices: RDD[(Vid, VD)], - edges: RDD[Edge[ED]], - defaultVertexAttr: VD, - partitionStrategy: PartitionStrategy): GraphImpl[VD,ED] = { - apply(vertices, edges, defaultVertexAttr, (a:VD, b:VD) => a, partitionStrategy) - } - - def apply[VD: ClassManifest, ED: ClassManifest]( - vertices: RDD[(Vid, VD)], - edges: RDD[Edge[ED]], - defaultVertexAttr: VD, - mergeFunc: (VD, VD) => VD): GraphImpl[VD,ED] = { - apply(vertices, edges, defaultVertexAttr, mergeFunc, RandomVertexCut()) - } + // def apply[VD: ClassManifest, ED: ClassManifest]( + // vertices: RDD[(Vid, VD)], + // edges: RDD[Edge[ED]], + // defaultVertexAttr: VD): GraphImpl[VD,ED] = { + // apply(vertices, edges, defaultVertexAttr, (a:VD, b:VD) => a, RandomVertexCut()) + // } + + // def apply[VD: ClassManifest, ED: ClassManifest]( + // vertices: RDD[(Vid, VD)], + // edges: RDD[Edge[ED]], + // defaultVertexAttr: VD, + // partitionStrategy: PartitionStrategy): GraphImpl[VD,ED] = { + // apply(vertices, edges, defaultVertexAttr, (a:VD, b:VD) => a, partitionStrategy) + // } + + // def apply[VD: ClassManifest, ED: ClassManifest]( + // vertices: RDD[(Vid, VD)], + // edges: RDD[Edge[ED]], + // defaultVertexAttr: VD, + // mergeFunc: (VD, VD) => VD): GraphImpl[VD,ED] = { + // apply(vertices, edges, defaultVertexAttr, mergeFunc, RandomVertexCut()) + // } def apply[VD: ClassManifest, ED: ClassManifest]( vertices: RDD[(Vid, VD)], diff --git a/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala b/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala index f0e423a57ba78..4c17bab0c47ab 100644 --- a/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala +++ b/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala @@ -65,7 +65,7 @@ object GraphGenerators { // Right now it just generates a bunch of edges where // the edge data is the weight (default 1) - def logNormalGraph(sc: SparkContext, numVertices: Int): GraphImpl[Int, Int] = { + def logNormalGraph(sc: SparkContext, numVertices: Int): Graph[Int, Int] = { // based on Pregel settings val mu = 4 val sigma = 1.3 @@ -75,11 +75,11 @@ object GraphGenerators { 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) } - - GraphImpl(vertices, edges, 0) + + Graph(vertices, edges, 0) //println("Vertices:") //for (v <- vertices) { // println(v.id) @@ -137,7 +137,7 @@ object GraphGenerators { - def rmatGraph(sc: SparkContext, requestedNumVertices: Int, numEdges: Int): GraphImpl[Int, Int] = { + 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 @@ -155,12 +155,12 @@ object GraphGenerators { } - def outDegreeFromEdges[ED: ClassManifest](edges: RDD[Edge[ED]]): GraphImpl[Int, ED] = { - + def outDegreeFromEdges[ED: ClassManifest](edges: RDD[Edge[ED]]): Graph[Int, ED] = { + val vertices = edges.flatMap { edge => List((edge.srcId, 1)) } .reduceByKey(_ + _) .map{ case (vid, degree) => (vid, degree) } - GraphImpl(vertices, edges, 0) + Graph(vertices, edges, 0) } /** @@ -192,7 +192,7 @@ object GraphGenerators { * | c | d | | * | | | | * *************** - - * + * * where this represents the subquadrant of the adj matrix currently being * subdivided. (x,y) represent the upper left hand corner of the subquadrant, * and T represents the side length (guaranteed to be a power of 2). @@ -204,7 +204,7 @@ object GraphGenerators { * quad = c, x'=x, y'=y+T/2, T'=T/2 * quad = d, x'=x+T/2, y'=y+T/2, T'=T/2 * - * @param src is the + * @param src is the */ @tailrec def chooseCell(x: Int, y: Int, t: Int): (Int, Int) = { @@ -242,7 +242,7 @@ object GraphGenerators { * 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 * order. - * + * * @param sc the spark context in which to construct the graph * @param rows the number of rows * @param cols the number of columns @@ -252,12 +252,12 @@ object GraphGenerators { */ def gridGraph(sc: SparkContext, rows: Int, cols: Int): Graph[(Int,Int), Double] = { // Convert row column address into vertex ids (row major order) - def sub2ind(r: Int, c: Int): Vid = r * cols + c + def sub2ind(r: Int, c: Int): Vid = r * cols + c - val vertices: RDD[(Vid, (Int,Int))] = + val vertices: RDD[(Vid, (Int,Int))] = sc.parallelize(0 until rows).flatMap( r => (0 until cols).map( c => (sub2ind(r,c), (r,c)) ) ) - val edges: RDD[Edge[Double]] = - vertices.flatMap{ case (vid, (r,c)) => + val edges: RDD[Edge[Double]] = + vertices.flatMap{ case (vid, (r,c)) => (if (r+1 < rows) { Seq( (sub2ind(r, c), sub2ind(r+1, c))) } else { Seq.empty }) ++ (if (c+1 < cols) { Seq( (sub2ind(r, c), sub2ind(r, c+1))) } else { Seq.empty }) }.map{ case (src, dst) => Edge(src, dst, 1.0) } @@ -266,7 +266,7 @@ object GraphGenerators { /** * Create a star graph with vertex 0 being the center. - * + * * @param sc the spark context in which to construct the graph * @param the number of vertices in the star * From ad56ae7bfd120406c64175d7e6923d28031a3f5d Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Sat, 23 Nov 2013 02:32:37 -0800 Subject: [PATCH 224/531] Support preservesPartitioning in RDD.zipPartitions --- .../main/scala/org/apache/spark/rdd/RDD.scala | 21 ++++++++++++++++--- .../spark/rdd/ZippedPartitionsRDD.scala | 21 ++++++++++++------- 2 files changed, 32 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 6e88be6f6ac64..7623c44d8856c 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -545,20 +545,35 @@ abstract class RDD[T: ClassManifest]( * *same number of partitions*, but does *not* require them to have the same number * of elements in each partition. */ + def zipPartitions[B: ClassManifest, V: ClassManifest] + (rdd2: RDD[B], preservesPartitioning: Boolean) + (f: (Iterator[T], Iterator[B]) => Iterator[V]): RDD[V] = + new ZippedPartitionsRDD2(sc, sc.clean(f), this, rdd2, preservesPartitioning) + def zipPartitions[B: ClassManifest, V: ClassManifest] (rdd2: RDD[B]) (f: (Iterator[T], Iterator[B]) => Iterator[V]): RDD[V] = - new ZippedPartitionsRDD2(sc, sc.clean(f), this, rdd2) + new ZippedPartitionsRDD2(sc, sc.clean(f), this, rdd2, false) + + def zipPartitions[B: ClassManifest, C: ClassManifest, V: ClassManifest] + (rdd2: RDD[B], rdd3: RDD[C], preservesPartitioning: Boolean) + (f: (Iterator[T], Iterator[B], Iterator[C]) => Iterator[V]): RDD[V] = + new ZippedPartitionsRDD3(sc, sc.clean(f), this, rdd2, rdd3, preservesPartitioning) def zipPartitions[B: ClassManifest, C: ClassManifest, V: ClassManifest] (rdd2: RDD[B], rdd3: RDD[C]) (f: (Iterator[T], Iterator[B], Iterator[C]) => Iterator[V]): RDD[V] = - new ZippedPartitionsRDD3(sc, sc.clean(f), this, rdd2, rdd3) + new ZippedPartitionsRDD3(sc, sc.clean(f), this, rdd2, rdd3, false) + + def zipPartitions[B: ClassManifest, C: ClassManifest, D: ClassManifest, V: ClassManifest] + (rdd2: RDD[B], rdd3: RDD[C], rdd4: RDD[D], preservesPartitioning: Boolean) + (f: (Iterator[T], Iterator[B], Iterator[C], Iterator[D]) => Iterator[V]): RDD[V] = + new ZippedPartitionsRDD4(sc, sc.clean(f), this, rdd2, rdd3, rdd4, preservesPartitioning) def zipPartitions[B: ClassManifest, C: ClassManifest, D: ClassManifest, V: ClassManifest] (rdd2: RDD[B], rdd3: RDD[C], rdd4: RDD[D]) (f: (Iterator[T], Iterator[B], Iterator[C], Iterator[D]) => Iterator[V]): RDD[V] = - new ZippedPartitionsRDD4(sc, sc.clean(f), this, rdd2, rdd3, rdd4) + new ZippedPartitionsRDD4(sc, sc.clean(f), this, rdd2, rdd3, rdd4, false) // Actions (launch a job to return a value to the user program) diff --git a/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala index 31e6fd519d0dd..faeb316664b00 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala @@ -39,9 +39,13 @@ private[spark] class ZippedPartitionsPartition( abstract class ZippedPartitionsBaseRDD[V: ClassManifest]( sc: SparkContext, - var rdds: Seq[RDD[_]]) + var rdds: Seq[RDD[_]], + preservesPartitioning: Boolean = false) extends RDD[V](sc, rdds.map(x => new OneToOneDependency(x))) { + override val partitioner = + if (preservesPartitioning) firstParent[Any].partitioner else None + override def getPartitions: Array[Partition] = { val sizes = rdds.map(x => x.partitions.size) if (!sizes.forall(x => x == sizes(0))) { @@ -76,8 +80,9 @@ class ZippedPartitionsRDD2[A: ClassManifest, B: ClassManifest, V: ClassManifest] sc: SparkContext, f: (Iterator[A], Iterator[B]) => Iterator[V], var rdd1: RDD[A], - var rdd2: RDD[B]) - extends ZippedPartitionsBaseRDD[V](sc, List(rdd1, rdd2)) { + var rdd2: RDD[B], + preservesPartitioning: Boolean = false) + extends ZippedPartitionsBaseRDD[V](sc, List(rdd1, rdd2), preservesPartitioning) { override def compute(s: Partition, context: TaskContext): Iterator[V] = { val partitions = s.asInstanceOf[ZippedPartitionsPartition].partitions @@ -97,8 +102,9 @@ class ZippedPartitionsRDD3 f: (Iterator[A], Iterator[B], Iterator[C]) => Iterator[V], var rdd1: RDD[A], var rdd2: RDD[B], - var rdd3: RDD[C]) - extends ZippedPartitionsBaseRDD[V](sc, List(rdd1, rdd2, rdd3)) { + var rdd3: RDD[C], + preservesPartitioning: Boolean = false) + extends ZippedPartitionsBaseRDD[V](sc, List(rdd1, rdd2, rdd3), preservesPartitioning) { override def compute(s: Partition, context: TaskContext): Iterator[V] = { val partitions = s.asInstanceOf[ZippedPartitionsPartition].partitions @@ -122,8 +128,9 @@ class ZippedPartitionsRDD4 var rdd1: RDD[A], var rdd2: RDD[B], var rdd3: RDD[C], - var rdd4: RDD[D]) - extends ZippedPartitionsBaseRDD[V](sc, List(rdd1, rdd2, rdd3, rdd4)) { + var rdd4: RDD[D], + preservesPartitioning: Boolean = false) + extends ZippedPartitionsBaseRDD[V](sc, List(rdd1, rdd2, rdd3, rdd4), preservesPartitioning) { override def compute(s: Partition, context: TaskContext): Iterator[V] = { val partitions = s.asInstanceOf[ZippedPartitionsPartition].partitions From fad6e70add83c2a32ada1be9b3a8bbd4a69412aa Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Sat, 23 Nov 2013 02:33:52 -0800 Subject: [PATCH 225/531] Simplify GraphImpl internals --- .../org/apache/spark/graph/VertexSetRDD.scala | 538 ++++++++---------- .../graph/impl/EdgePartitionBuilder.scala | 8 +- .../apache/spark/graph/impl/GraphImpl.scala | 211 +++---- .../spark/graph/impl/VTableReplicated.scala | 100 ++++ .../graph/impl/VTableReplicatedValues.scala | 84 --- .../spark/graph/impl/VertexPartition.scala | 75 +++ .../spark/graph/impl/VertexPlacement.scala | 72 +++ .../org/apache/spark/graph/impl/Vid2Pid.scala | 87 --- 8 files changed, 567 insertions(+), 608 deletions(-) create mode 100644 graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala delete mode 100644 graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicatedValues.scala create mode 100644 graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala create mode 100644 graph/src/main/scala/org/apache/spark/graph/impl/VertexPlacement.scala delete mode 100644 graph/src/main/scala/org/apache/spark/graph/impl/Vid2Pid.scala diff --git a/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala b/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala index 0f2006a42d7c6..ed70402a6f6f5 100644 --- a/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala @@ -22,14 +22,15 @@ import org.apache.spark.SparkContext._ import org.apache.spark.rdd._ import org.apache.spark.storage.StorageLevel import org.apache.spark.util.collection.{BitSet, OpenHashSet, PrimitiveKeyOpenHashMap} + import org.apache.spark.graph.impl.AggregationMsg import org.apache.spark.graph.impl.MsgRDDFunctions._ - +import org.apache.spark.graph.impl.VertexPartition /** - * The `VertexSetIndex` maintains the per-partition mapping from - * vertex id to the corresponding location in the per-partition values - * array. This class is meant to be an opaque type. + * Maintains the per-partition mapping from vertex id to the corresponding + * location in the per-partition values array. This class is meant to be an + * opaque type. * */ class VertexSetIndex(private[spark] val rdd: RDD[VertexIdToIndexMap]) { @@ -48,23 +49,12 @@ class VertexSetIndex(private[spark] val rdd: RDD[VertexIdToIndexMap]) { def partitioner: Partitioner = rdd.partitioner.get } // end of VertexSetIndex - /** - * An VertexSetRDD[V] extends the RDD[(Vid,V)] by ensuring that there - * is only one entry for each vertex and by pre-indexing the entries - * for fast, efficient joins. - * - * In addition to providing the basic RDD[(Vid,V)] functionality the - * VertexSetRDD exposes an index member which can be used to "key" - * other VertexSetRDDs - * - * @tparam V the vertex attribute associated with each vertex in the - * set. + * A `VertexSetRDD[VD]` extends the `RDD[(Vid, VD)]` by ensuring that there is + * only one entry for each vertex and by pre-indexing the entries for fast, + * efficient joins. * - * @param index the index which contains the vertex id information and - * is used to organize the values in the RDD. - * @param valuesRDD the values RDD contains the actual vertex - * attributes organized as an array within each partition. + * @tparam VD the vertex attribute associated with each vertex in the set. * * To construct a `VertexSetRDD` use the singleton object: * @@ -83,126 +73,124 @@ class VertexSetIndex(private[spark] val rdd: RDD[VertexIdToIndexMap]) { * }}} * */ -class VertexSetRDD[@specialized V: ClassManifest]( - @transient val index: VertexSetIndex, - @transient val valuesRDD: RDD[ ( Array[V], BitSet) ]) - extends RDD[(Vid, V)](index.rdd.context, - List(new OneToOneDependency(index.rdd), new OneToOneDependency(valuesRDD)) ) { +class VertexSetRDD[@specialized VD: ClassManifest]( + @transient val partitionsRDD: RDD[VertexPartition[VD]]) + extends RDD[(Vid, VD)](partitionsRDD.context, List(new OneToOneDependency(partitionsRDD))) { + + /** + * The `VertexSetIndex` representing the layout of this `VertexSetRDD`. + */ + def index = new VertexSetIndex(partitionsRDD.mapPartitions(_.map(_.index), + preservesPartitioning = true)) /** * Construct a new VertexSetRDD that is indexed by only the keys in the RDD. * The resulting VertexSet will be based on a different index and can * no longer be quickly joined with this RDD. */ - def reindex(): VertexSetRDD[V] = VertexSetRDD(this) + def reindex(): VertexSetRDD[VD] = VertexSetRDD(this) /** * An internal representation which joins the block indices with the values - * This is used by the compute function to emulate RDD[(Vid, V)] + * This is used by the compute function to emulate `RDD[(Vid, VD)]` */ - protected[spark] val tuples = - new ZippedRDD(index.rdd.context, index.rdd, valuesRDD) + protected[spark] val tuples = partitionsRDD.flatMap(_.iterator) /** * The partitioner is defined by the index. */ - override val partitioner = index.rdd.partitioner + override val partitioner = partitionsRDD.partitioner /** * The actual partitions are defined by the tuples. */ - override def getPartitions: Array[Partition] = tuples.getPartitions + override def getPartitions: Array[Partition] = tuples.partitions /** * The preferred locations are computed based on the preferred * locations of the tuples. */ override def getPreferredLocations(s: Partition): Seq[String] = - tuples.getPreferredLocations(s) + tuples.preferredLocations(s) /** - * Caching an VertexSetRDD causes the index and values to be cached separately. + * Caching a VertexSetRDD causes the index and values to be cached separately. */ - override def persist(newLevel: StorageLevel): VertexSetRDD[V] = { - index.persist(newLevel) - valuesRDD.persist(newLevel) - return this + override def persist(newLevel: StorageLevel): VertexSetRDD[VD] = { + partitionsRDD.persist(newLevel) + this } /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */ - override def persist(): VertexSetRDD[V] = persist(StorageLevel.MEMORY_ONLY) + override def persist(): VertexSetRDD[VD] = persist(StorageLevel.MEMORY_ONLY) /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */ - override def cache(): VertexSetRDD[V] = persist() + override def cache(): VertexSetRDD[VD] = persist() /** - * Provide the RDD[(K,V)] equivalent output. + * Provide the `RDD[(Vid, VD)]` equivalent output. */ - override def compute(part: Partition, context: TaskContext): Iterator[(Vid, V)] = { - tuples.compute(part, context).flatMap { case (indexMap, (values, bs) ) => - bs.iterator.map(ind => (indexMap.getValueSafe(ind), values(ind))) + override def compute(part: Partition, context: TaskContext): Iterator[(Vid, VD)] = + tuples.compute(part, context) + + /** + * Return a new VertexSetRDD by applying a function to each VertexPartition of + * this RDD. + */ + def mapVertexPartitions[VD2: ClassManifest]( + f: VertexPartition[VD] => VertexPartition[VD2]): VertexSetRDD[VD2] = { + val cleanF = sparkContext.clean(f) + val newPartitionsRDD = partitionsRDD.mapPartitions(_.map(f), preservesPartitioning = true) + new VertexSetRDD(newPartitionsRDD) + } + + /** + * Return a new VertexSetRDD by applying a function to corresponding + * VertexPartitions of this VertexSetRDD and another one. + */ + def zipVertexPartitions[VD2: ClassManifest, VD3: ClassManifest] + (other: VertexSetRDD[VD2]) + (f: (VertexPartition[VD], VertexPartition[VD2]) => VertexPartition[VD3]): VertexSetRDD[VD3] = { + val cleanF = sparkContext.clean(f) + val newPartitionsRDD = partitionsRDD.zipPartitions( + other.partitionsRDD, preservesPartitioning = true + ) { + (thisIter, otherIter) => + val thisPart = thisIter.next() + val otherPart = otherIter.next() + Iterator(cleanF(thisPart, otherPart)) } - } // end of compute + new VertexSetRDD(newPartitionsRDD) + } /** * Restrict the vertex set to the set of vertices satisfying the * given predicate. * - * @param pred the user defined predicate + * @param pred the user defined predicate, which takes a tuple to conform to + * the RDD[(Vid, VD)] interface * * @note The vertex set preserves the original index structure * which means that the returned RDD can be easily joined with * the original vertex-set. Furthermore, the filter only * modifies the bitmap index and so no new values are allocated. */ - override def filter(pred: Tuple2[Vid,V] => Boolean): VertexSetRDD[V] = { - val cleanPred = index.rdd.context.clean(pred) - val newValues = index.rdd.zipPartitions(valuesRDD){ - (keysIter: Iterator[VertexIdToIndexMap], - valuesIter: Iterator[(Array[V], BitSet)]) => - val index = keysIter.next() - assert(keysIter.hasNext == false) - val (oldValues, bs) = valuesIter.next() - assert(valuesIter.hasNext == false) - // Allocate the array to store the results into - val newBS = new BitSet(index.capacity) - // Iterate over the active bits in the old bitset and - // evaluate the predicate - var ind = bs.nextSetBit(0) - while(ind >= 0) { - val k = index.getValueSafe(ind) - if (cleanPred((k, oldValues(ind)))) { - newBS.set(ind) - } - ind = bs.nextSetBit(ind+1) - } - Array((oldValues, newBS)).iterator - } - new VertexSetRDD[V](index, newValues) - } // end of filter + override def filter(pred: Tuple2[Vid, VD] => Boolean): VertexSetRDD[VD] = + this.mapVertexPartitions(_.filter(Function.untupled(pred))) /** * Pass each vertex attribute through a map function and retain the * original RDD's partitioning and index. * - * @tparam U the type returned by the map function + * @tparam VD2 the type returned by the map function * * @param f the function applied to each value in the RDD - * @return a new VertexSet with values obtaind by applying `f` to + * @return a new VertexSetRDD with values obtained by applying `f` to * each of the entries in the original VertexSet. The resulting * VertexSetRDD retains the same index. */ - def mapValues[U: ClassManifest](f: V => U): VertexSetRDD[U] = { - val cleanF = index.rdd.context.clean(f) - val newValuesRDD: RDD[ (Array[U], BitSet) ] = - valuesRDD.mapPartitions(iter => iter.map{ - case (values, bs: BitSet) => - val newValues = new Array[U](values.size) - bs.iterator.foreach { ind => newValues(ind) = cleanF(values(ind)) } - (newValues, bs) - }, preservesPartitioning = true) - new VertexSetRDD[U](index, newValuesRDD) - } // end of mapValues + def mapValues[VD2: ClassManifest](f: VD => VD2): VertexSetRDD[VD2] = + this.mapVertexPartitions(_.map { case (vid, attr) => f(attr) }) /** * Fill in missing values for all vertices in the index. @@ -211,62 +199,51 @@ class VertexSetRDD[@specialized V: ClassManifest]( * index that don't currently have values. * @return A VertexSetRDD with a value for all vertices. */ - def fillMissing(missingValue: V): VertexSetRDD[V] = { - val newValuesRDD: RDD[ (Array[V], BitSet) ] = - valuesRDD.zipPartitions(index.rdd){ (valuesIter, indexIter) => - val index = indexIter.next - assert(!indexIter.hasNext) - val (values, bs: BitSet) = valuesIter.next - assert(!valuesIter.hasNext) - // Allocate a new values array with missing value as the default - val newValues = Array.fill(values.size)(missingValue) - // Copy over the old values - bs.iterator.foreach { ind => newValues(ind) = values(ind) } - // Create a new bitset matching the keyset - val newBS = index.getBitSet - Iterator((newValues, newBS)) + def fillMissing(missingValue: VD): VertexSetRDD[VD] = { + this.mapVertexPartitions { part => + // Allocate a new values array with missing value as the default + val newValues = Array.fill(part.values.size)(missingValue) + // Copy over the old values + part.mask.iterator.foreach { ind => + newValues(ind) = part.values(ind) } - new VertexSetRDD[V](index, newValuesRDD) + // Create a new mask with all vertices in the index + val newMask = part.index.getBitSet + new VertexPartition(part.index, newValues, newMask) + } } /** * Pass each vertex attribute along with the vertex id through a map * function and retain the original RDD's partitioning and index. * - * @tparam U the type returned by the map function + * @tparam VD2 the type returned by the map function * * @param f the function applied to each vertex id and vertex * attribute in the RDD - * @return a new VertexSet with values obtaind by applying `f` to + * @return a new VertexSet with values obtained by applying `f` to * each of the entries in the original VertexSet. The resulting * VertexSetRDD retains the same index. */ - def mapValuesWithKeys[U: ClassManifest](f: (Vid, V) => U): VertexSetRDD[U] = { - val cleanF = index.rdd.context.clean(f) - val newValues: RDD[ (Array[U], BitSet) ] = - index.rdd.zipPartitions(valuesRDD){ - (keysIter: Iterator[VertexIdToIndexMap], - valuesIter: Iterator[(Array[V], BitSet)]) => - val index = keysIter.next() - assert(keysIter.hasNext == false) - val (values, bs: BitSet) = valuesIter.next() - assert(valuesIter.hasNext == false) - // Cosntruct a view of the map transformation - val newValues = new Array[U](index.capacity) - bs.iterator.foreach { ind => newValues(ind) = cleanF(index.getValueSafe(ind), values(ind)) } - Iterator((newValues, bs)) + def mapValuesWithKeys[VD2: ClassManifest](f: (Vid, VD) => VD2): VertexSetRDD[VD2] = { + this.mapVertexPartitions { part => + // Construct a view of the map transformation + val newValues = new Array[VD2](part.index.capacity) + part.mask.iterator.foreach { ind => + newValues(ind) = f(part.index.getValueSafe(ind), part.values(ind)) } - new VertexSetRDD[U](index, newValues) + new VertexPartition(part.index, newValues, part.mask) + } } // end of mapValuesWithKeys - /** * Inner join this VertexSet with another VertexSet which has the * same Index. This function will fail if both VertexSets do not * share the same index. The resulting vertex set will only contain * vertices that are in both this and the other vertex set. * - * @tparam W the attribute type of the other VertexSet + * @tparam VD2 the attribute type of the other VertexSet + * @tparam VD3 the attribute type of the resulting VertexSet * * @param other the other VertexSet with which to join. * @param f the function mapping a vertex id and its attributes in @@ -275,31 +252,23 @@ class VertexSetRDD[@specialized V: ClassManifest]( * and the other VertexSet and with tuple attributes. * */ - def zipJoin[W: ClassManifest, Z: ClassManifest](other: VertexSetRDD[W])(f: (Vid, V, W) => Z): - VertexSetRDD[Z] = { - val cleanF = index.rdd.context.clean(f) - if (index != other.index) { - throw new SparkException("A zipJoin can only be applied to RDDs with the same index!") - } - val newValuesRDD: RDD[ (Array[Z], BitSet) ] = - index.rdd.zipPartitions(valuesRDD, other.valuesRDD) { (indexIter, thisIter, otherIter) => - val index = indexIter.next() - assert(!indexIter.hasNext) - val (thisValues, thisBS: BitSet) = thisIter.next() - assert(!thisIter.hasNext) - val (otherValues, otherBS: BitSet) = otherIter.next() - assert(!otherIter.hasNext) - val newBS: BitSet = thisBS & otherBS - val newValues = new Array[Z](index.capacity) - newBS.iterator.foreach { ind => - newValues(ind) = cleanF(index.getValueSafe(ind), thisValues(ind), otherValues(ind)) - } - Iterator((newValues, newBS)) + def zipJoin[VD2: ClassManifest, VD3: ClassManifest] + (other: VertexSetRDD[VD2])(f: (Vid, VD, VD2) => VD3): VertexSetRDD[VD3] = { + this.zipVertexPartitions(other) { + (thisPart, otherPart) => + if (thisPart.index != otherPart.index) { + throw new SparkException("can't zip join VertexSetRDDs with different indexes") } - new VertexSetRDD(index, newValuesRDD) + val newValues = new Array[VD3](thisPart.index.capacity) + val newMask = thisPart.mask & otherPart.mask + newMask.iterator.foreach { ind => + newValues(ind) = + f(thisPart.index.getValueSafe(ind), thisPart.values(ind), otherPart.values(ind)) + } + new VertexPartition(thisPart.index, newValues, newMask) + } } - /** * Inner join this VertexSet with another VertexSet which has the * same Index. This function will fail if both VertexSets do not @@ -308,30 +277,27 @@ class VertexSetRDD[@specialized V: ClassManifest]( * @param other the vertex set to join with this vertex set * @param f the function mapping a vertex id and its attributes in * this and the other vertex set to a collection of tuples. - * @tparam W the type of the other vertex set attributes - * @tparam Z the type of the tuples emitted by `f` + * @tparam VD2 the type of the other vertex set attributes + * @tparam VD3 the type of the tuples emitted by `f` * @return an RDD containing the tuples emitted by `f` */ - def zipJoinFlatMap[W: ClassManifest, Z: ClassManifest](other: VertexSetRDD[W])(f: (Vid, V,W) => Iterator[Z]): - RDD[Z] = { - val cleanF = index.rdd.context.clean(f) - if (index != other.index) { - throw new SparkException("A zipJoin can only be applied to RDDs with the same index!") - } - index.rdd.zipPartitions(valuesRDD, other.valuesRDD) { (indexIter, thisIter, otherIter) => - val index = indexIter.next() - assert(!indexIter.hasNext) - val (thisValues, thisBS: BitSet) = thisIter.next() - assert(!thisIter.hasNext) - val (otherValues, otherBS: BitSet) = otherIter.next() - assert(!otherIter.hasNext) - val newBS: BitSet = thisBS & otherBS - val newValues = new Array[Z](index.capacity) - newBS.iterator.flatMap { ind => cleanF(index.getValueSafe(ind), thisValues(ind), otherValues(ind)) } + def zipJoinFlatMap[VD2: ClassManifest, VD3: ClassManifest] + (other: VertexSetRDD[VD2]) + (f: (Vid, VD, VD2) => Iterator[VD3]): RDD[VD3] = { + val cleanF = sparkContext.clean(f) + partitionsRDD.zipPartitions(other.partitionsRDD) { + (thisPartIter, otherPartIter) => + val thisPart = thisPartIter.next() + val otherPart = otherPartIter.next() + if (thisPart.index != otherPart.index) { + throw new SparkException("can't zip join VertexSetRDDs with different indexes") + } + (thisPart.mask & otherPart.mask).iterator.flatMap { ind => + cleanF(thisPart.index.getValueSafe(ind), thisPart.values(ind), otherPart.values(ind)) + } } } - /** * Left join this VertexSet with another VertexSet which has the * same Index. This function will fail if both VertexSets do not @@ -339,7 +305,8 @@ class VertexSetRDD[@specialized V: ClassManifest]( * for each vertex in this set. If the other VertexSet is missing * any vertex in this VertexSet then a `None` attribute is generated * - * @tparam W the attribute type of the other VertexSet + * @tparam VD2 the attribute type of the other VertexSet + * @tparam VD3 the attribute type of the resulting VertexSet * * @param other the other VertexSet with which to join. * @param f the function mapping a vertex id and its attributes in @@ -349,28 +316,21 @@ class VertexSetRDD[@specialized V: ClassManifest]( * other VertexSet. * */ - def leftZipJoin[W: ClassManifest, Z: ClassManifest](other: VertexSetRDD[W])(f: (Vid, V, Option[W]) => Z): - VertexSetRDD[Z] = { - if (index != other.index) { - throw new SparkException("A zipJoin can only be applied to RDDs with the same index!") - } - val cleanF = index.rdd.context.clean(f) - val newValuesRDD: RDD[(Array[Z], BitSet)] = - index.rdd.zipPartitions(valuesRDD, other.valuesRDD) { (indexIter, thisIter, otherIter) => - val index = indexIter.next() - assert(!indexIter.hasNext) - val (thisValues, thisBS: BitSet) = thisIter.next() - assert(!thisIter.hasNext) - val (otherValues, otherBS: BitSet) = otherIter.next() - assert(!otherIter.hasNext) - val newValues = new Array[Z](index.capacity) - thisBS.iterator.foreach { ind => - val otherV = if (otherBS.get(ind)) Option(otherValues(ind)) else None - newValues(ind) = cleanF(index.getValueSafe(ind), thisValues(ind), otherV) + def leftZipJoin[VD2: ClassManifest, VD3: ClassManifest] + (other: VertexSetRDD[VD2])(f: (Vid, VD, Option[VD2]) => VD3): VertexSetRDD[VD3] = { + this.zipVertexPartitions(other) { + (thisPart, otherPart) => + if (thisPart.index != otherPart.index) { + throw new SparkException("can't zip join VertexSetRDDs with different indexes") } - Iterator((newValues, thisBS)) + val newValues = new Array[VD3](thisPart.index.capacity) + thisPart.mask.iterator.foreach { ind => + val otherV = if (otherPart.mask.get(ind)) Option(otherPart.values(ind)) else None + newValues(ind) = f( + thisPart.index.getValueSafe(ind), thisPart.values(ind), otherV) + } + new VertexPartition(thisPart.index, newValues, thisPart.mask) } - new VertexSetRDD(index, newValuesRDD) } // end of leftZipJoin @@ -382,7 +342,8 @@ class VertexSetRDD[@specialized V: ClassManifest]( * set. If the other VertexSet is missing any vertex in this * VertexSet then a `None` attribute is generated * - * @tparam W the attribute type of the other VertexSet + * @tparam VD2 the attribute type of the other VertexSet + * @tparam VD2 the attribute type of the resulting VertexSet * * @param other the other VertexSet with which to join. * @param f the function mapping a vertex id and its attributes in @@ -393,27 +354,26 @@ class VertexSetRDD[@specialized V: ClassManifest]( * VertexSet with the attribute emitted by f. * */ - def leftJoin[W: ClassManifest, Z: ClassManifest](other: RDD[(Vid,W)]) - (f: (Vid, V, Option[W]) => Z, merge: (W,W) => W = (a:W, b:W) => a ): - VertexSetRDD[Z] = { - val cleanF = index.rdd.context.clean(f) - val cleanMerge = index.rdd.context.clean(merge) + def leftJoin[VD2: ClassManifest, VD3: ClassManifest] + (other: RDD[(Vid, VD2)]) + (f: (Vid, VD, Option[VD2]) => VD3, merge: (VD2, VD2) => VD2 = (a: VD2, b: VD2) => a) + : VertexSetRDD[VD3] = { // Test if the other vertex is a VertexSetRDD to choose the optimal // join strategy other match { - // If the other set is a VertexSetRDD and shares the same index as - // this vertex set then we use the much more efficient leftZipJoin - case other: VertexSetRDD[_] if index == other.index => { - leftZipJoin(other)(cleanF) + // If the other set is a VertexSetRDD then we use the much more efficient + // leftZipJoin + case other: VertexSetRDD[_] => { + leftZipJoin(other)(f) } case _ => { - val indexedOther: VertexSetRDD[W] = VertexSetRDD(other, index, cleanMerge) - leftZipJoin(indexedOther)(cleanF) + val indexedOther: VertexSetRDD[VD2] = VertexSetRDD(other, this.index, merge) + leftZipJoin(indexedOther)(f) } } } // end of leftJoin -} // End of VertexSetRDD +} // end of VertexSetRDD /** @@ -425,25 +385,25 @@ object VertexSetRDD { * Construct a vertex set from an RDD of vertex-attribute pairs. * Duplicate entries are removed arbitrarily. * - * @tparam V the vertex attribute type + * @tparam VD the vertex attribute type * * @param rdd the collection of vertex-attribute pairs */ - def apply[V: ClassManifest](rdd: RDD[(Vid,V)]): VertexSetRDD[V] = - apply(rdd, (a:V, b:V) => a ) + def apply[VD: ClassManifest](rdd: RDD[(Vid, VD)]): VertexSetRDD[VD] = + apply(rdd, (a: VD, b: VD) => a) /** * Construct a vertex set from an RDD of vertex-attribute pairs * where duplicate entries are merged using the reduceFunc * - * @tparam V the vertex attribute type + * @tparam VD the vertex attribute type * * @param rdd the collection of vertex-attribute pairs * @param reduceFunc the function used to merge attributes of * duplicate vertices. */ - def apply[V: ClassManifest]( - rdd: RDD[(Vid,V)], reduceFunc: (V, V) => V): VertexSetRDD[V] = { + def apply[VD: ClassManifest]( + rdd: RDD[(Vid, VD)], reduceFunc: (VD, VD) => VD): VertexSetRDD[VD] = { val cReduceFunc = rdd.context.clean(reduceFunc) // Preaggregate and shuffle if necessary val preAgg = rdd.partitioner match { @@ -451,25 +411,19 @@ object VertexSetRDD { case None => val partitioner = new HashPartitioner(rdd.partitions.size) // Preaggregation. - val aggregator = new Aggregator[Vid, V, V](v => v, cReduceFunc, cReduceFunc) + val aggregator = new Aggregator[Vid, VD, VD](v => v, cReduceFunc, cReduceFunc) rdd.mapPartitions(aggregator.combineValuesByKey, true).partitionBy(partitioner) } - val groups = preAgg.mapPartitions( iter => { - val hashMap = new PrimitiveKeyOpenHashMap[Vid, V] - for ((k,v) <- iter) { + val partitionsRDD = preAgg.mapPartitions(iter => { + val hashMap = new PrimitiveKeyOpenHashMap[Vid, VD] + for ((k, v) <- iter) { hashMap.setMerge(k, v, cReduceFunc) } - val index = hashMap.keySet - val values = hashMap._values - val bs = index.getBitSet - Iterator( (index, (values, bs)) ) - }, true).cache - // extract the index and the values - val index = groups.mapPartitions(_.map{ case (kMap, vAr) => kMap }, true) - val values: RDD[(Array[V], BitSet)] = - groups.mapPartitions(_.map{ case (kMap,vAr) => vAr }, true) - new VertexSetRDD[V](new VertexSetIndex(index), values) + val part = new VertexPartition(hashMap.keySet, hashMap._values, hashMap.keySet.getBitSet) + Iterator(part) + }, preservesPartitioning = true).cache + new VertexSetRDD(partitionsRDD) } // end of apply /** @@ -477,57 +431,62 @@ object VertexSetRDD { * * @note duplicate vertices are discarded arbitrarily * - * @tparam V the vertex attribute type + * @tparam VD the vertex attribute type * @param rdd the rdd containing vertices - * @param index the index which must be a superset of the vertices + * @param indexPrototype a VertexSetRDD whose indexes will be reused. The + * indexes must be a superset of the vertices in rdd * in RDD */ - def apply[V: ClassManifest]( - rdd: RDD[(Vid,V)], index: VertexSetIndex): VertexSetRDD[V] = - apply(rdd, index, (a:V,b:V) => a) + def apply[VD: ClassManifest]( + rdd: RDD[(Vid, VD)], index: VertexSetIndex): VertexSetRDD[VD] = + apply(rdd, index, (a: VD, b: VD) => a) /** * Construct a vertex set from an RDD using an existing index and a * user defined `combiner` to merge duplicate vertices. * - * @tparam V the vertex attribute type + * @tparam VD the vertex attribute type * @param rdd the rdd containing vertices - * @param index the index which must be a superset of the vertices - * in RDD + * @param indexPrototype a VertexSetRDD whose indexes will be reused. The + * indexes must be a superset of the vertices in rdd * @param reduceFunc the user defined reduce function used to merge * duplicate vertex attributes. */ - def apply[V: ClassManifest]( - rdd: RDD[(Vid,V)], index: VertexSetIndex, - reduceFunc: (V, V) => V): VertexSetRDD[V] = - apply(rdd,index, (v:V) => v, reduceFunc, reduceFunc) + def apply[VD: ClassManifest]( + rdd: RDD[(Vid, VD)], + index: VertexSetIndex, + reduceFunc: (VD, VD) => VD): VertexSetRDD[VD] = + apply(rdd, index, (v: VD) => v, reduceFunc, reduceFunc) /** - * Construct a vertex set from an RDD of AggregationMsgs + * Construct a vertex set from an RDD of Product2[Vid, VD] * - * @tparam V the vertex attribute type + * @tparam VD the vertex attribute type * @param rdd the rdd containing vertices - * @param index the index which must be a superset of the vertices - * in RDD + * @param indexPrototype a VertexSetRDD whose indexes will be reused. The + * indexes must be a superset of the vertices in rdd * @param reduceFunc the user defined reduce function used to merge * duplicate vertex attributes. */ - private[spark] def aggregate[V: ClassManifest]( - rdd: RDD[AggregationMsg[V]], index: VertexSetIndex, - reduceFunc: (V, V) => V): VertexSetRDD[V] = { + private[spark] def aggregate[VD: ClassManifest, VidVDPair <: Product2[Vid, VD] : ClassManifest]( + rdd: RDD[VidVDPair], + index: VertexSetIndex, + reduceFunc: (VD, VD) => VD): VertexSetRDD[VD] = { - val cReduceFunc = index.rdd.context.clean(reduceFunc) - assert(rdd.partitioner == index.rdd.partitioner) + val cReduceFunc = rdd.context.clean(reduceFunc) + assert(rdd.partitioner == Some(index.partitioner)) // Use the index to build the new values table - val values: RDD[ (Array[V], BitSet) ] = index.rdd.zipPartitions(rdd)( (indexIter, tblIter) => { + val partitionsRDD = index.rdd.zipPartitions( + rdd, preservesPartitioning = true + ) { + (indexIter, tblIter) => // There is only one map val index = indexIter.next() - assert(!indexIter.hasNext) - val values = new Array[V](index.capacity) - val bs = new BitSet(index.capacity) - for (msg <- tblIter) { + val mask = new BitSet(index.capacity) + val values = new Array[VD](index.capacity) + for (vertexPair <- tblIter) { // Get the location of the key in the index - val pos = index.getPos(msg.vid) + val pos = index.getPos(vertexPair._1) if ((pos & OpenHashSet.NONEXISTENCE_MASK) != 0) { throw new SparkException("Error: Trying to bind an external index " + "to an RDD which contains keys that are not in the index.") @@ -535,24 +494,25 @@ object VertexSetRDD { // Get the actual index val ind = pos & OpenHashSet.POSITION_MASK // If this value has already been seen then merge - if (bs.get(ind)) { - values(ind) = cReduceFunc(values(ind), msg.data) + if (mask.get(ind)) { + values(ind) = cReduceFunc(values(ind), vertexPair._2) } else { // otherwise just store the new value - bs.set(ind) - values(ind) = msg.data + mask.set(ind) + values(ind) = vertexPair._2 } } } - Iterator((values, bs)) - }) - new VertexSetRDD(index, values) + Iterator(new VertexPartition(index, values, mask)) + } + + new VertexSetRDD(partitionsRDD) } /** * Construct a vertex set from an RDD using an existing index and a * user defined `combiner` to merge duplicate vertices. * - * @tparam V the vertex attribute type + * @tparam VD the vertex attribute type * @param rdd the rdd containing vertices * @param index the index which must be a superset of the vertices * in RDD @@ -563,59 +523,27 @@ object VertexSetRDD { * @param mergeCombiners a user defined function to merge combiners * */ - def apply[V: ClassManifest, C: ClassManifest]( - rdd: RDD[(Vid,V)], + def apply[VD: ClassManifest, C: ClassManifest]( + rdd: RDD[(Vid, VD)], index: VertexSetIndex, - createCombiner: V => C, - mergeValue: (C, V) => C, + createCombiner: VD => C, + mergeValue: (C, VD) => C, mergeCombiners: (C, C) => C): VertexSetRDD[C] = { - val cCreateCombiner = index.rdd.context.clean(createCombiner) - val cMergeValue = index.rdd.context.clean(mergeValue) - val cMergeCombiners = index.rdd.context.clean(mergeCombiners) - // Get the index Partitioner - val partitioner = index.rdd.partitioner match { - case Some(p) => p - case None => throw new SparkException("An index must have a partitioner.") - } + val cCreateCombiner = rdd.context.clean(createCombiner) + val cMergeValue = rdd.context.clean(mergeValue) + val cMergeCombiners = rdd.context.clean(mergeCombiners) + val partitioner = index.partitioner // Preaggregate and shuffle if necessary val partitioned = if (rdd.partitioner != Some(partitioner)) { // Preaggregation. - val aggregator = new Aggregator[Vid, V, C](cCreateCombiner, cMergeValue, - cMergeCombiners) + val aggregator = new Aggregator[Vid, VD, C](cCreateCombiner, cMergeValue, cMergeCombiners) rdd.mapPartitions(aggregator.combineValuesByKey).partitionBy(partitioner) } else { rdd.mapValues(x => createCombiner(x)) } - // Use the index to build the new values table - val values: RDD[ (Array[C], BitSet) ] = index.rdd.zipPartitions(partitioned)( (indexIter, tblIter) => { - // There is only one map - val index = indexIter.next() - assert(!indexIter.hasNext) - val values = new Array[C](index.capacity) - val bs = new BitSet(index.capacity) - for ((k,c) <- tblIter) { - // Get the location of the key in the index - val pos = index.getPos(k) - if ((pos & OpenHashSet.NONEXISTENCE_MASK) != 0) { - throw new SparkException("Error: Trying to bind an external index " + - "to an RDD which contains keys that are not in the index.") - } else { - // Get the actual index - val ind = pos & OpenHashSet.POSITION_MASK - // If this value has already been seen then merge - if (bs.get(ind)) { - values(ind) = cMergeCombiners(values(ind), c) - } else { // otherwise just store the new value - bs.set(ind) - values(ind) = c - } - } - } - Iterator((values, bs)) - }) - new VertexSetRDD(index, values) + aggregate(partitioned, index, mergeCombiners) } // end of apply /** @@ -623,23 +551,24 @@ object VertexSetRDD { * can be used to build VertexSets over subsets of the vertices in * the input. */ - def makeIndex(keys: RDD[Vid], partitionerOpt: Option[Partitioner] = None): VertexSetIndex = { + def makeIndex( + keys: RDD[Vid], partitionerOpt: Option[Partitioner] = None): VertexSetIndex = { val partitioner = partitionerOpt match { case Some(p) => p case None => Partitioner.defaultPartitioner(keys) } - val preAgg: RDD[(Vid, Unit)] = keys.mapPartitions( iter => { + val preAgg: RDD[(Vid, Unit)] = keys.mapPartitions(iter => { val keys = new VertexIdToIndexMap - while(iter.hasNext) { keys.add(iter.next) } + while (iter.hasNext) { keys.add(iter.next) } keys.iterator.map(k => (k, ())) - }, true).partitionBy(partitioner) + }, preservesPartitioning = true).partitionBy(partitioner) - val index = preAgg.mapPartitions( iter => { + val index = preAgg.mapPartitions(iter => { val index = new VertexIdToIndexMap - while(iter.hasNext) { index.add(iter.next._1) } + while (iter.hasNext) { index.add(iter.next._1) } Iterator(index) - }, true).cache + }, preservesPartitioning = true).cache new VertexSetIndex(index) } @@ -649,21 +578,16 @@ object VertexSetRDD { * * @param index an index over the set of vertices * @param defaultValue the default value to use when initializing the vertices - * @tparam V the type of the vertex attribute + * @tparam VD the type of the vertex attribute * @return */ - def apply[V: ClassManifest](index: VertexSetIndex, defaultValue: V): VertexSetRDD[V] = { - // Use the index to build the new values table - val values: RDD[ (Array[V], BitSet) ] = index.rdd.mapPartitions(_.map { index => + def apply[VD: ClassManifest](index: VertexSetIndex, defaultValue: VD): VertexSetRDD[VD] = { + // Use the index to build the new values tables + val partitionsRDD = index.rdd.mapPartitions(_.map { index => val values = Array.fill(index.capacity)(defaultValue) - val bs = index.getBitSet - (values, bs) + val mask = index.getBitSet + new VertexPartition(index, values, mask) }, preservesPartitioning = true) - new VertexSetRDD(index, values) + new VertexSetRDD(partitionsRDD) } // end of apply } // end of object VertexSetRDD - - - - - diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartitionBuilder.scala b/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartitionBuilder.scala index cc3a443fa2bb8..355f8f05427c2 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartitionBuilder.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartitionBuilder.scala @@ -5,7 +5,7 @@ import org.apache.spark.graph._ //private[graph] -class EdgePartitionBuilder[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) +class EdgePartitionBuilder[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED: ClassManifest]{ val srcIds = new VertexArrayList val dstIds = new VertexArrayList @@ -22,10 +22,6 @@ ED: ClassManifest]{ def toEdgePartition: EdgePartition[ED] = { new EdgePartition(srcIds.toLongArray(), dstIds.toLongArray(), dataBuilder.result()) } - - -} - - +} diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index 650bb515b5abf..337d560c2659b 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -66,44 +66,42 @@ class EdgeTripletIterator[VD: ClassManifest, ED: ClassManifest]( /** * A Graph RDD that supports computation on graphs. * - * @param localVidMap Stores the location of vertex attributes after they are - * replicated. Within each partition, localVidMap holds a map from vertex ID to - * the index where that vertex's attribute is stored. This index refers to the - * arrays in the same partition in the variants of - * [[VTableReplicatedValues]]. Therefore, localVidMap can be reused across - * changes to the vertex attributes. + * Graphs are represented using two classes of data: vertex-partitioned and + * edge-partitioned. `vTable` contains vertex attributes, which are + * vertex-partitioned. `eTable` contains edge attributes, which are + * edge-partitioned. For operations on vertex neighborhoods, vertex attributes + * are replicated to the edge partitions where they appear as sources or + * destinations. `vertexPlacement` specifies where each vertex will be + * replicated. `vTableReplicated` stores the replicated vertex attributes, which + * are co-partitioned with the relevant edges. */ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( @transient val vTable: VertexSetRDD[VD], - @transient val vid2pid: Vid2Pid, - @transient val localVidMap: RDD[(Pid, VertexIdToIndexMap)], @transient val eTable: RDD[(Pid, EdgePartition[ED])], + @transient val vertexPlacement: VertexPlacement, @transient val partitioner: PartitionStrategy) extends Graph[VD, ED] { - def this() = this(null, null, null, null, null) + def this() = this(null, null, null, null) - @transient val vTableReplicatedValues: VTableReplicatedValues[VD] = - new VTableReplicatedValues(vTable, vid2pid, localVidMap) + @transient val vTableReplicated: VTableReplicated[VD] = + new VTableReplicated(vTable, eTable, vertexPlacement) /** Return a RDD of vertices. */ @transient override val vertices = vTable /** Return a RDD of edges. */ - @transient override val edges: RDD[Edge[ED]] = { - eTable.mapPartitions( iter => iter.next()._2.iterator , true ) - } + @transient override val edges: RDD[Edge[ED]] = + eTable.mapPartitions(_.next()._2.iterator, true) /** Return a RDD that brings edges with its source and destination vertices together. */ @transient override val triplets: RDD[EdgeTriplet[VD, ED]] = - makeTriplets(localVidMap, vTableReplicatedValues.bothAttrs, eTable) + makeTriplets(vTableReplicated.bothAttrs, eTable) override def persist(newLevel: StorageLevel): Graph[VD, ED] = { - eTable.persist(newLevel) - vid2pid.persist(newLevel) vTable.persist(newLevel) - localVidMap.persist(newLevel) - // vTableReplicatedValues.persist(newLevel) + eTable.persist(newLevel) + vertexPlacement.persist(newLevel) this } @@ -113,11 +111,11 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( val numVertices = this.numVertices val numEdges = this.numEdges val replicationRatioBothAttrs = - vid2pid.bothAttrs.map(kv => kv._2.size).sum / numVertices + vertexPlacement.bothAttrs.map(_.map(_.size).sum).sum / numVertices val replicationRatioSrcAttrOnly = - vid2pid.srcAttrOnly.map(kv => kv._2.size).sum / numVertices + vertexPlacement.srcAttrOnly.map(_.map(_.size).sum).sum / numVertices val replicationRatioDstAttrOnly = - vid2pid.dstAttrOnly.map(kv => kv._2.size).sum / numVertices + vertexPlacement.dstAttrOnly.map(_.map(_.size).sum).sum / numVertices val loadArray = eTable.map{ case (pid, epart) => epart.data.size }.collect.map(x => x.toDouble / numEdges) val minLoad = loadArray.min @@ -159,73 +157,63 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( println("eTable ------------------------------------------") traverseLineage(eTable, " ") var visited = Map(eTable.id -> "eTable") - println("\n\nvTable.index ------------------------------------") - traverseLineage(vTable.index.rdd, " ", visited) - visited += (vTable.index.rdd.id -> "vTable.index") - println("\n\nvTable.values ------------------------------------") - traverseLineage(vTable.valuesRDD, " ", visited) - visited += (vTable.valuesRDD.id -> "vTable.values") println("\n\nvTable ------------------------------------------") traverseLineage(vTable, " ", visited) visited += (vTable.id -> "vTable") - println("\n\nvid2pid.bothAttrs -------------------------------") - traverseLineage(vid2pid.bothAttrs, " ", visited) - visited += (vid2pid.bothAttrs.id -> "vid2pid") - visited += (vid2pid.bothAttrs.valuesRDD.id -> "vid2pid.bothAttrs") - println("\n\nlocalVidMap -------------------------------------") - traverseLineage(localVidMap, " ", visited) - visited += (localVidMap.id -> "localVidMap") - println("\n\nvTableReplicatedValues.bothAttrs ----------------") - traverseLineage(vTableReplicatedValues.bothAttrs, " ", visited) - visited += (vTableReplicatedValues.bothAttrs.id -> "vTableReplicatedValues.bothAttrs") + println("\n\nvertexPlacement.bothAttrs -------------------------------") + traverseLineage(vertexPlacement.bothAttrs, " ", visited) + visited += (vertexPlacement.bothAttrs.id -> "vertexPlacement.bothAttrs") + println("\n\nvTableReplicated.bothAttrs ----------------") + traverseLineage(vTableReplicated.bothAttrs, " ", visited) + visited += (vTableReplicated.bothAttrs.id -> "vTableReplicated.bothAttrs") println("\n\ntriplets ----------------------------------------") traverseLineage(triplets, " ", visited) println(visited) } // end of print lineage override def reverse: Graph[VD, ED] = { - val newEtable = eTable.mapPartitions( _.map{ case (pid, epart) => (pid, epart.reverse) }, + val newETable = eTable.mapPartitions(_.map { case (pid, epart) => (pid, epart.reverse) }, preservesPartitioning = true) - new GraphImpl(vTable, vid2pid, localVidMap, newEtable, partitioner) + new GraphImpl(vTable, newETable, vertexPlacement, partitioner) } - override def mapVertices[VD2: ClassManifest](f: (Vid, VD) => VD2): Graph[VD2, ED] = { - val newVTable = vTable.mapValuesWithKeys((vid, data) => f(vid, data)) - new GraphImpl(newVTable, vid2pid, localVidMap, eTable, partitioner) - } + override def mapVertices[VD2: ClassManifest](f: (Vid, VD) => VD2): Graph[VD2, ED] = + new GraphImpl(vTable.mapVertexPartitions(_.map(f)), eTable, vertexPlacement, partitioner) override def mapEdges[ED2: ClassManifest](f: Edge[ED] => ED2): Graph[VD, ED2] = { - val newETable = eTable.mapPartitions(_.map{ case (pid, epart) => (pid, epart.map(f)) }, + val newETable = eTable.mapPartitions(_.map { case (pid, epart) => (pid, epart.map(f)) }, preservesPartitioning = true) - new GraphImpl(vTable, vid2pid, localVidMap, newETable, partitioner) + new GraphImpl(vTable, newETable, vertexPlacement, partitioner) } override def mapTriplets[ED2: ClassManifest](f: EdgeTriplet[VD, ED] => ED2): Graph[VD, ED2] = GraphImpl.mapTriplets(this, f) - override def subgraph(epred: EdgeTriplet[VD,ED] => Boolean = (x => true), - vpred: (Vid, VD) => Boolean = ((a,b) => true) ): Graph[VD, ED] = { - // Reuse the partitioner (but not the index) from this graph - val newVTable = - VertexSetRDD(vertices.filter(v => vpred(v._1, v._2)).partitionBy(vTable.index.partitioner)) + override def subgraph( + epred: EdgeTriplet[VD, ED] => Boolean = (x => true), + vpred: (Vid, VD) => Boolean = ((a,b) => true)): Graph[VD, ED] = { + + // Filter the vertices, reusing the partitioner (but not the index) from + // this graph + val newVTable = vTable.mapVertexPartitions(_.filter(vpred).reindex()) + // Restrict the set of edges to those that satisfy the vertex and the edge predicate. val newETable = createETable( - triplets.filter( - t => vpred( t.srcId, t.srcAttr ) && vpred( t.dstId, t.dstAttr ) && epred(t) - ).map( t => Edge(t.srcId, t.dstId, t.attr) ), partitioner) - // Construct the Vid2Pid map. Here we assume that the filter operation - // behaves deterministically. - // @todo reindex the vertex and edge tables - val newVid2Pid = new Vid2Pid(newETable, newVTable.index) - val newVidMap = createLocalVidMap(newETable) - new GraphImpl(newVTable, newVid2Pid, localVidMap, newETable, partitioner) + triplets.filter(t => vpred(t.srcId, t.srcAttr) && vpred(t.dstId, t.dstAttr) && epred(t)) + .map(t => Edge(t.srcId, t.dstId, t.attr)), partitioner) + + // Construct the VertexPlacement map + val newVertexPlacement = new VertexPlacement(newETable, newVTable) + + new GraphImpl(newVTable, newETable, newVertexPlacement, partitioner) } // end of subgraph - override def groupEdges(merge: (ED, ED) => ED ): Graph[VD,ED] = { + override def groupEdges(merge: (ED, ED) => ED): Graph[VD, ED] = { ClosureCleaner.clean(merge) val newETable = - eTable.mapPartitions { _.map(p => (p._1, p._2.groupEdges(merge))) } - new GraphImpl(vTable, vid2pid, localVidMap, newETable, partitioner) + eTable.mapPartitions({ _.map(p => (p._1, p._2.groupEdges(merge))) }, + preservesPartitioning = true) + new GraphImpl(vTable, newETable, vertexPlacement, partitioner) } ////////////////////////////////////////////////////////////////////////////////////////////////// @@ -234,16 +222,14 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( override def mapReduceTriplets[A: ClassManifest]( mapFunc: EdgeTriplet[VD, ED] => Iterator[(Vid, A)], - reduceFunc: (A, A) => A) - : VertexSetRDD[A] = + reduceFunc: (A, A) => A): VertexSetRDD[A] = GraphImpl.mapReduceTriplets(this, mapFunc, reduceFunc) override def outerJoinVertices[U: ClassManifest, VD2: ClassManifest] - (updates: RDD[(Vid, U)])(updateF: (Vid, VD, Option[U]) => VD2) - : Graph[VD2, ED] = { + (updates: RDD[(Vid, U)])(updateF: (Vid, VD, Option[U]) => VD2): Graph[VD2, ED] = { ClosureCleaner.clean(updateF) val newVTable = vTable.leftJoin(updates)(updateF) - new GraphImpl(newVTable, vid2pid, localVidMap, eTable, partitioner) + new GraphImpl(newVTable, eTable, vertexPlacement, partitioner) } } // end of class GraphImpl @@ -253,22 +239,20 @@ object GraphImpl { def apply[VD: ClassManifest, ED: ClassManifest]( edges: RDD[Edge[ED]], defaultValue: VD, - partitionStrategy: PartitionStrategy): - GraphImpl[VD, ED] = { + partitionStrategy: PartitionStrategy): GraphImpl[VD, ED] = { val etable = createETable(edges, partitionStrategy).cache // Get the set of all vids - val vids = etable.mapPartitions( iter => { + val vids = etable.mapPartitions(iter => { val (pid, epart) = iter.next() assert(!iter.hasNext) epart.iterator.flatMap(e => Iterator(e.srcId, e.dstId)) - }, true) + }, preservesPartitioning = true) // Index the set of all vids val index = VertexSetRDD.makeIndex(vids) // Index the vertices and fill in missing attributes with the default val vtable = VertexSetRDD(index, defaultValue) - val vid2pid = new Vid2Pid(etable, vtable.index) - val localVidMap = createLocalVidMap(etable) - new GraphImpl(vtable, vid2pid, localVidMap, etable, partitionStrategy) + val vertexPlacement = new VertexPlacement(etable, vtable) + new GraphImpl(vtable, etable, vertexPlacement, partitionStrategy) } // def apply[VD: ClassManifest, ED: ClassManifest]( @@ -295,11 +279,11 @@ object GraphImpl { // } def apply[VD: ClassManifest, ED: ClassManifest]( - vertices: RDD[(Vid, VD)], - edges: RDD[Edge[ED]], - defaultVertexAttr: VD, - mergeFunc: (VD, VD) => VD, - partitionStrategy: PartitionStrategy): GraphImpl[VD,ED] = { + vertices: RDD[(Vid, VD)], + edges: RDD[Edge[ED]], + defaultVertexAttr: VD, + mergeFunc: (VD, VD) => VD, + partitionStrategy: PartitionStrategy): GraphImpl[VD, ED] = { vertices.cache val etable = createETable(edges, partitionStrategy).cache @@ -308,7 +292,7 @@ object GraphImpl { val implicitVids = etable.flatMap { case (pid, partition) => Array.concat(partition.srcIds, partition.dstIds) }.map(vid => (vid, ())).partitionBy(partitioner) - val allVids = vertices.zipPartitions(implicitVids) { + val allVids = vertices.zipPartitions(implicitVids, preservesPartitioning = true) { (a, b) => a.map(_._1) ++ b.map(_._1) } // Index the set of all vids @@ -316,9 +300,8 @@ object GraphImpl { // Index the vertices and fill in missing attributes with the default val vtable = VertexSetRDD(vertices, index, mergeFunc).fillMissing(defaultVertexAttr) - val vid2pid = new Vid2Pid(etable, vtable.index) - val localVidMap = createLocalVidMap(etable) - new GraphImpl(vtable, vid2pid, localVidMap, etable, partitionStrategy) + val vertexPlacement = new VertexPlacement(etable, vtable) + new GraphImpl(vtable, etable, vertexPlacement, partitionStrategy) } /** @@ -330,8 +313,8 @@ object GraphImpl { * containing all the edges in a partition. */ protected def createETable[ED: ClassManifest]( - edges: RDD[Edge[ED]], - partitionStrategy: PartitionStrategy): RDD[(Pid, EdgePartition[ED])] = { + edges: RDD[Edge[ED]], + partitionStrategy: PartitionStrategy): RDD[(Pid, EdgePartition[ED])] = { // Get the number of partitions val numPartitions = edges.partitions.size @@ -353,40 +336,26 @@ object GraphImpl { }, preservesPartitioning = true).cache() } - private def createLocalVidMap( - eTable: RDD[(Pid, EdgePartition[ED])] forSome { type ED } - ): RDD[(Pid, VertexIdToIndexMap)] = { - eTable.mapPartitions( _.map{ case (pid, epart) => - val vidToIndex = new VertexIdToIndexMap - epart.foreach{ e => - vidToIndex.add(e.srcId) - vidToIndex.add(e.dstId) - } - (pid, vidToIndex) - }, preservesPartitioning = true).cache() - } - - def makeTriplets[VD: ClassManifest, ED: ClassManifest]( - localVidMap: RDD[(Pid, VertexIdToIndexMap)], - vTableReplicatedValues: RDD[(Pid, Array[VD]) ], + protected def makeTriplets[VD: ClassManifest, ED: ClassManifest]( + vTableReplicated: RDD[(Pid, (VertexIdToIndexMap, Array[VD]))], eTable: RDD[(Pid, EdgePartition[ED])]): RDD[EdgeTriplet[VD, ED]] = { - eTable.zipPartitions(localVidMap, vTableReplicatedValues) { - (eTableIter, vidMapIter, replicatedValuesIter) => - val (_, vidToIndex) = vidMapIter.next() - val (_, vertexArray) = replicatedValuesIter.next() + eTable.zipPartitions(vTableReplicated) { + (eTableIter, vTableReplicatedIter) => val (_, edgePartition) = eTableIter.next() + val (_, (vidToIndex, vertexArray)) = vTableReplicatedIter.next() new EdgeTripletIterator(vidToIndex, vertexArray, edgePartition) } } - def mapTriplets[VD: ClassManifest, ED: ClassManifest, ED2: ClassManifest]( + protected def mapTriplets[VD: ClassManifest, ED: ClassManifest, ED2: ClassManifest]( g: GraphImpl[VD, ED], f: EdgeTriplet[VD, ED] => ED2): Graph[VD, ED2] = { - val newETable = g.eTable.zipPartitions(g.localVidMap, g.vTableReplicatedValues.bothAttrs) { - (edgePartitionIter, vidToIndexIter, vertexArrayIter) => - val (pid, edgePartition) = edgePartitionIter.next() - val (_, vidToIndex) = vidToIndexIter.next() - val (_, vertexArray) = vertexArrayIter.next() + val newETable = g.eTable.zipPartitions( + g.vTableReplicated.bothAttrs, preservesPartitioning = true + ) { + (eTableIter, vTableReplicatedIter) => + val (pid, edgePartition) = eTableIter.next() + val (_, (vidToIndex, vertexArray)) = vTableReplicatedIter.next() val et = new EdgeTriplet[VD, ED] val vmap = new PrimitiveKeyOpenHashMap[Vid, VD](vidToIndex, vertexArray) val newEdgePartition = edgePartition.map { e => @@ -397,10 +366,10 @@ object GraphImpl { } Iterator((pid, newEdgePartition)) } - new GraphImpl(g.vTable, g.vid2pid, g.localVidMap, newETable, g.partitioner) + new GraphImpl(g.vTable, newETable, g.vertexPlacement, g.partitioner) } - def mapReduceTriplets[VD: ClassManifest, ED: ClassManifest, A: ClassManifest]( + protected def mapReduceTriplets[VD: ClassManifest, ED: ClassManifest, A: ClassManifest]( g: GraphImpl[VD, ED], mapFunc: EdgeTriplet[VD, ED] => Iterator[(Vid, A)], reduceFunc: (A, A) => A): VertexSetRDD[A] = { @@ -412,18 +381,12 @@ object GraphImpl { val mapFuncUsesDstAttr = accessesVertexAttr[VD, ED](mapFunc, "dstAttr") // Map and preaggregate val preAgg = g.eTable.zipPartitions( - g.localVidMap, - g.vTableReplicatedValues.get(mapFuncUsesSrcAttr, mapFuncUsesDstAttr) + g.vTableReplicated.get(mapFuncUsesSrcAttr, mapFuncUsesDstAttr) ) { - (edgePartitionIter, vidToIndexIter, vertexArrayIter) => + (edgePartitionIter, vTableReplicatedIter) => val (_, edgePartition) = edgePartitionIter.next() - val (_, vidToIndex) = vidToIndexIter.next() - val (_, vertexArray) = vertexArrayIter.next() - assert(!edgePartitionIter.hasNext) - assert(!vidToIndexIter.hasNext) - assert(!vertexArrayIter.hasNext) + val (_, (vidToIndex, vertexArray)) = vTableReplicatedIter.next() assert(vidToIndex.capacity == vertexArray.size) - // Reuse the vidToIndex map to run aggregation. val vmap = new PrimitiveKeyOpenHashMap[Vid, VD](vidToIndex, vertexArray) // TODO(jegonzal): This doesn't allow users to send messages to arbitrary vertices. val msgArray = new Array[A](vertexArray.size) @@ -460,7 +423,7 @@ object GraphImpl { msgBS.iterator.map { ind => new AggregationMsg[A](vidToIndex.getValue(ind), msgArray(ind)) } - }.partitionBy(g.vTable.index.rdd.partitioner.get) + }.partitionBy(g.vTable.partitioner.get) // do the final reduction reusing the index map VertexSetRDD.aggregate(preAgg, g.vTable.index, reduceFunc) } diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala b/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala new file mode 100644 index 0000000000000..6cbac223f7e87 --- /dev/null +++ b/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala @@ -0,0 +1,100 @@ +package org.apache.spark.graph.impl + +import org.apache.spark.SparkContext._ +import org.apache.spark.rdd.RDD +import org.apache.spark.util.collection.{OpenHashSet, PrimitiveKeyOpenHashMap} + +import org.apache.spark.graph._ +import org.apache.spark.graph.impl.MsgRDDFunctions._ + +/** + * Stores the vertex attribute values after they are replicated. + */ +class VTableReplicated[VD: ClassManifest]( + vTable: VertexSetRDD[VD], + eTable: RDD[(Pid, EdgePartition[ED])] forSome { type ED }, + vertexPlacement: VertexPlacement) { + + val bothAttrs: RDD[(Pid, (VertexIdToIndexMap, Array[VD]))] = + VTableReplicated.createVTableReplicated(vTable, eTable, vertexPlacement, true, true) + val srcAttrOnly: RDD[(Pid, (VertexIdToIndexMap, Array[VD]))] = + VTableReplicated.createVTableReplicated(vTable, eTable, vertexPlacement, true, false) + val dstAttrOnly: RDD[(Pid, (VertexIdToIndexMap, Array[VD]))] = + VTableReplicated.createVTableReplicated(vTable, eTable, vertexPlacement, false, true) + val noAttrs: RDD[(Pid, (VertexIdToIndexMap, Array[VD]))] = + VTableReplicated.createVTableReplicated(vTable, eTable, vertexPlacement, false, false) + + + def get(includeSrcAttr: Boolean, includeDstAttr: Boolean) + : RDD[(Pid, (VertexIdToIndexMap, Array[VD]))] = + (includeSrcAttr, includeDstAttr) match { + case (true, true) => bothAttrs + case (true, false) => srcAttrOnly + case (false, true) => dstAttrOnly + case (false, false) => noAttrs + } +} + +class VertexAttributeBlock[VD: ClassManifest](val vids: Array[Vid], val attrs: Array[VD]) + +object VTableReplicated { + protected def createVTableReplicated[VD: ClassManifest]( + vTable: VertexSetRDD[VD], + eTable: RDD[(Pid, EdgePartition[ED])] forSome { type ED }, + vertexPlacement: VertexPlacement, + includeSrcAttr: Boolean, + includeDstAttr: Boolean): RDD[(Pid, (VertexIdToIndexMap, Array[VD]))] = { + val placement = vertexPlacement.get(includeSrcAttr, includeDstAttr) + + // Send each edge partition the vertex attributes it wants, as specified in + // vertexPlacement + val msgsByPartition = placement.zipPartitions(vTable.partitionsRDD) { + (pid2vidIter, vertexPartIter) => + val pid2vid = pid2vidIter.next() + val vertexPart = vertexPartIter.next() + + val vmap = new PrimitiveKeyOpenHashMap(vertexPart.index, vertexPart.values) + val output = new Array[(Pid, VertexAttributeBlock[VD])](pid2vid.size) + for (pid <- 0 until pid2vid.size) { + val block = new VertexAttributeBlock(pid2vid(pid), pid2vid(pid).map(vid => vmap(vid))) + output(pid) = (pid, block) + } + output.iterator + }.partitionBy(eTable.partitioner.get).cache() + + // Within each edge partition, create a local map from vid to an index into + // the attribute array. Each map contains a superset of the vertices that it + // will receive, because it stores vids from both the source and destination + // of edges. It must always include both source and destination vids because + // some operations, such as GraphImpl.mapReduceTriplets, rely on this. + val localVidMap = eTable.mapPartitions(_.map { + case (pid, epart) => + val vidToIndex = new VertexIdToIndexMap + epart.foreach { e => + vidToIndex.add(e.srcId) + vidToIndex.add(e.dstId) + } + (pid, vidToIndex) + }, preservesPartitioning = true).cache() + + // Within each edge partition, place the vertex attributes received from + // msgsByPartition into the correct locations specified in localVidMap + localVidMap.zipPartitions(msgsByPartition) { + (mapIter, msgsIter) => + val (pid, vidToIndex) = mapIter.next() + assert(!mapIter.hasNext) + // Populate the vertex array using the vidToIndex map + val vertexArray = new Array[VD](vidToIndex.capacity) + for ((_, block) <- msgsIter) { + for (i <- 0 until block.vids.size) { + val vid = block.vids(i) + val attr = block.attrs(i) + val ind = vidToIndex.getPos(vid) & OpenHashSet.POSITION_MASK + vertexArray(ind) = attr + } + } + Iterator((pid, (vidToIndex, vertexArray))) + }.cache() + } + +} diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicatedValues.scala b/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicatedValues.scala deleted file mode 100644 index fee2d40ee4aa1..0000000000000 --- a/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicatedValues.scala +++ /dev/null @@ -1,84 +0,0 @@ -package org.apache.spark.graph.impl - -import org.apache.spark.SparkContext._ -import org.apache.spark.rdd.RDD -import org.apache.spark.util.collection.{OpenHashSet, PrimitiveKeyOpenHashMap} - -import org.apache.spark.graph._ -import org.apache.spark.graph.impl.MsgRDDFunctions._ - -/** - * Stores the vertex attribute values after they are replicated. See - * the description of localVidMap in [[GraphImpl]]. - */ -class VTableReplicatedValues[VD: ClassManifest]( - vTable: VertexSetRDD[VD], - vid2pid: Vid2Pid, - localVidMap: RDD[(Pid, VertexIdToIndexMap)]) { - - val bothAttrs: RDD[(Pid, Array[VD])] = - VTableReplicatedValues.createVTableReplicated(vTable, vid2pid, localVidMap, true, true) - val srcAttrOnly: RDD[(Pid, Array[VD])] = - VTableReplicatedValues.createVTableReplicated(vTable, vid2pid, localVidMap, true, false) - val dstAttrOnly: RDD[(Pid, Array[VD])] = - VTableReplicatedValues.createVTableReplicated(vTable, vid2pid, localVidMap, false, true) - val noAttrs: RDD[(Pid, Array[VD])] = - VTableReplicatedValues.createVTableReplicated(vTable, vid2pid, localVidMap, false, false) - - - def get(includeSrcAttr: Boolean, includeDstAttr: Boolean): RDD[(Pid, Array[VD])] = - (includeSrcAttr, includeDstAttr) match { - case (true, true) => bothAttrs - case (true, false) => srcAttrOnly - case (false, true) => dstAttrOnly - case (false, false) => noAttrs - } -} - -class VertexAttributeBlock[VD: ClassManifest](val vids: Array[Vid], val attrs: Array[VD]) - -object VTableReplicatedValues { - protected def createVTableReplicated[VD: ClassManifest]( - vTable: VertexSetRDD[VD], - vid2pid: Vid2Pid, - localVidMap: RDD[(Pid, VertexIdToIndexMap)], - includeSrcAttr: Boolean, - includeDstAttr: Boolean): RDD[(Pid, Array[VD])] = { - - val pid2vid = vid2pid.getPid2Vid(includeSrcAttr, includeDstAttr) - - val msgsByPartition = pid2vid.zipPartitions(vTable.index.rdd, vTable.valuesRDD) { - (pid2vidIter, indexIter, valuesIter) => - val pid2vid = pid2vidIter.next() - val index = indexIter.next() - val values = valuesIter.next() - val vmap = new PrimitiveKeyOpenHashMap(index, values._1) - - // Send each partition the vertex attributes it wants - val output = new Array[(Pid, VertexAttributeBlock[VD])](pid2vid.size) - for (pid <- 0 until pid2vid.size) { - val block = new VertexAttributeBlock(pid2vid(pid), pid2vid(pid).map(vid => vmap(vid))) - output(pid) = (pid, block) - } - output.iterator - }.partitionBy(localVidMap.partitioner.get).cache() - - localVidMap.zipPartitions(msgsByPartition){ - (mapIter, msgsIter) => - val (pid, vidToIndex) = mapIter.next() - assert(!mapIter.hasNext) - // Populate the vertex array using the vidToIndex map - val vertexArray = new Array[VD](vidToIndex.capacity) - for ((_, block) <- msgsIter) { - for (i <- 0 until block.vids.size) { - val vid = block.vids(i) - val attr = block.attrs(i) - val ind = vidToIndex.getPos(vid) & OpenHashSet.POSITION_MASK - vertexArray(ind) = attr - } - } - Iterator((pid, vertexArray)) - }.cache() - } - -} diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala new file mode 100644 index 0000000000000..9de57375e9117 --- /dev/null +++ b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala @@ -0,0 +1,75 @@ +package org.apache.spark.graph.impl + +import org.apache.spark.util.collection.{BitSet, PrimitiveKeyOpenHashMap} + +import org.apache.spark.graph._ + +class VertexPartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) VD: ClassManifest]( + val index: VertexIdToIndexMap, + val values: Array[VD], + val mask: BitSet) { + + /** + * Pass each vertex attribute along with the vertex id through a map + * function and retain the original RDD's partitioning and index. + * + * @tparam VD2 the type returned by the map function + * + * @param f the function applied to each vertex id and vertex + * attribute in the RDD + * + * @return a new VertexPartition with values obtained by applying `f` to + * each of the entries in the original VertexSet. The resulting + * VertexPartition retains the same index. + */ + def map[VD2: ClassManifest](f: (Vid, VD) => VD2): VertexPartition[VD2] = { + // Construct a view of the map transformation + val newValues = new Array[VD2](index.capacity) + mask.iterator.foreach { ind => + newValues(ind) = f(index.getValueSafe(ind), values(ind)) + } + new VertexPartition[VD2](index, newValues, mask) + } + + /** + * Restrict the vertex set to the set of vertices satisfying the + * given predicate. + * + * @param pred the user defined predicate + * + * @note The vertex set preserves the original index structure + * which means that the returned RDD can be easily joined with + * the original vertex-set. Furthermore, the filter only + * modifies the bitmap index and so no new values are allocated. + */ + def filter(pred: (Vid, VD) => Boolean): VertexPartition[VD] = { + // Allocate the array to store the results into + val newMask = new BitSet(index.capacity) + // Iterate over the active bits in the old bitset and + // evaluate the predicate + var ind = mask.nextSetBit(0) + while (ind >= 0) { + val k = index.getValueSafe(ind) + if (pred(k, values(ind))) { + newMask.set(ind) + } + ind = mask.nextSetBit(ind + 1) + } + new VertexPartition(index, values, newMask) + } + + /** + * Construct a new VertexPartition whose index contains only the vertices in + * the mask. + */ + def reindex(): VertexPartition[VD] = { + val hashMap = new PrimitiveKeyOpenHashMap[Vid, VD] + val arbitraryMerge = (a: VD, b: VD) => a + for ((k, v) <- this.iterator) { + hashMap.setMerge(k, v, arbitraryMerge) + } + new VertexPartition(hashMap.keySet, hashMap._values, index.getBitSet) + } + + def iterator = mask.iterator.map(ind => (index.getValueSafe(ind), values(ind))) +} diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPlacement.scala b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPlacement.scala new file mode 100644 index 0000000000000..e8734df2ed58f --- /dev/null +++ b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPlacement.scala @@ -0,0 +1,72 @@ +package org.apache.spark.graph.impl + +import scala.collection.JavaConversions._ +import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.ArrayBuilder + +import org.apache.spark.rdd.RDD +import org.apache.spark.storage.StorageLevel + +import org.apache.spark.graph._ + +/** + * Stores the layout of replicated vertex attributes for GraphImpl. Tells each + * partition of the vertex data where it should go. + */ +class VertexPlacement( + eTable: RDD[(Pid, EdgePartition[ED])] forSome { type ED }, + vTable: VertexSetRDD[_]) { + + val bothAttrs: RDD[Array[Array[Vid]]] = createPid2Vid(true, true) + val srcAttrOnly: RDD[Array[Array[Vid]]] = createPid2Vid(true, false) + val dstAttrOnly: RDD[Array[Array[Vid]]] = createPid2Vid(false, true) + val noAttrs: RDD[Array[Array[Vid]]] = createPid2Vid(false, false) + + def get(includeSrcAttr: Boolean, includeDstAttr: Boolean): RDD[Array[Array[Vid]]] = + (includeSrcAttr, includeDstAttr) match { + case (true, true) => bothAttrs + case (true, false) => srcAttrOnly + case (false, true) => dstAttrOnly + case (false, false) => noAttrs + } + + def persist(newLevel: StorageLevel) { + bothAttrs.persist(newLevel) + srcAttrOnly.persist(newLevel) + dstAttrOnly.persist(newLevel) + noAttrs.persist(newLevel) + } + + private def createPid2Vid( + includeSrcAttr: Boolean, includeDstAttr: Boolean): RDD[Array[Array[Vid]]] = { + // Determine which vertices each edge partition needs by creating a mapping + // from vid to pid + val preAgg = eTable.mapPartitions { iter => + val (pid, edgePartition) = iter.next() + val vSet = new VertexSet + if (includeSrcAttr || includeDstAttr) { + edgePartition.foreach { e => + if (includeSrcAttr) vSet.add(e.srcId) + if (includeDstAttr) vSet.add(e.dstId) + } + } + vSet.iterator.map { vid => (vid.toLong, pid) } + } + // Aggregate the mappings to determine where each vertex should go + val vid2pid = VertexSetRDD[Pid, ArrayBuffer[Pid]](preAgg, vTable.index, + (p: Pid) => ArrayBuffer(p), + (ab: ArrayBuffer[Pid], p:Pid) => {ab.append(p); ab}, + (a: ArrayBuffer[Pid], b: ArrayBuffer[Pid]) => a ++ b) + .mapValues(a => a.toArray) + // Within each vertex partition, reorganize the placement information into + // columnar format keyed on the destination partition + val numPartitions = vid2pid.partitions.size + vid2pid.mapPartitions { iter => + val pid2vid = Array.fill[ArrayBuilder[Vid]](numPartitions)(ArrayBuilder.make[Vid]) + for ((vid, pids) <- iter) { + pids.foreach { pid => pid2vid(pid) += vid } + } + Iterator(pid2vid.map(_.result)) + } + } +} diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/Vid2Pid.scala b/graph/src/main/scala/org/apache/spark/graph/impl/Vid2Pid.scala deleted file mode 100644 index 363adbbce949e..0000000000000 --- a/graph/src/main/scala/org/apache/spark/graph/impl/Vid2Pid.scala +++ /dev/null @@ -1,87 +0,0 @@ -package org.apache.spark.graph.impl - -import scala.collection.JavaConversions._ -import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.ArrayBuilder - -import org.apache.spark.rdd.RDD -import org.apache.spark.storage.StorageLevel - -import org.apache.spark.graph._ - -/** - * Stores the layout of vertex attributes for GraphImpl. - */ -class Vid2Pid( - eTable: RDD[(Pid, EdgePartition[ED])] forSome { type ED }, - vTableIndex: VertexSetIndex) { - - val bothAttrs: VertexSetRDD[Array[Pid]] = createVid2Pid(true, true) - val srcAttrOnly: VertexSetRDD[Array[Pid]] = createVid2Pid(true, false) - val dstAttrOnly: VertexSetRDD[Array[Pid]] = createVid2Pid(false, true) - val noAttrs: VertexSetRDD[Array[Pid]] = createVid2Pid(false, false) - - val pid2VidBothAttrs: RDD[Array[Array[Vid]]] = createPid2Vid(bothAttrs) - val pid2VidSrcAttrOnly: RDD[Array[Array[Vid]]] = createPid2Vid(srcAttrOnly) - val pid2VidDstAttrOnly: RDD[Array[Array[Vid]]] = createPid2Vid(dstAttrOnly) - val pid2VidNoAttrs: RDD[Array[Array[Vid]]] = createPid2Vid(noAttrs) - - def get(includeSrcAttr: Boolean, includeDstAttr: Boolean): VertexSetRDD[Array[Pid]] = - (includeSrcAttr, includeDstAttr) match { - case (true, true) => bothAttrs - case (true, false) => srcAttrOnly - case (false, true) => dstAttrOnly - case (false, false) => noAttrs - } - - def getPid2Vid(includeSrcAttr: Boolean, includeDstAttr: Boolean): RDD[Array[Array[Vid]]] = - (includeSrcAttr, includeDstAttr) match { - case (true, true) => pid2VidBothAttrs - case (true, false) => pid2VidSrcAttrOnly - case (false, true) => pid2VidDstAttrOnly - case (false, false) => pid2VidNoAttrs - } - - def persist(newLevel: StorageLevel) { - bothAttrs.persist(newLevel) - srcAttrOnly.persist(newLevel) - dstAttrOnly.persist(newLevel) - noAttrs.persist(newLevel) - } - - private def createVid2Pid( - includeSrcAttr: Boolean, - includeDstAttr: Boolean): VertexSetRDD[Array[Pid]] = { - val preAgg = eTable.mapPartitions { iter => - val (pid, edgePartition) = iter.next() - val vSet = new VertexSet - if (includeSrcAttr || includeDstAttr) { - edgePartition.foreach(e => { - if (includeSrcAttr) vSet.add(e.srcId) - if (includeDstAttr) vSet.add(e.dstId) - }) - } - vSet.iterator.map { vid => (vid.toLong, pid) } - } - VertexSetRDD[Pid, ArrayBuffer[Pid]](preAgg, vTableIndex, - (p: Pid) => ArrayBuffer(p), - (ab: ArrayBuffer[Pid], p:Pid) => {ab.append(p); ab}, - (a: ArrayBuffer[Pid], b: ArrayBuffer[Pid]) => a ++ b) - .mapValues(a => a.toArray).cache() - } - - /** - * Creates an intermediate pid2vid structure that tells each partition of the - * vertex data where it should go. - */ - private def createPid2Vid(vid2pid: VertexSetRDD[Array[Pid]]): RDD[Array[Array[Vid]]] = { - val numPartitions = vid2pid.partitions.size - vid2pid.mapPartitions { iter => - val pid2vidLocal = Array.fill[ArrayBuilder[Vid]](numPartitions)(ArrayBuilder.make[Vid]) - for ((vid, pids) <- iter) { - pids.foreach { pid => pid2vidLocal(pid) += vid } - } - Iterator(pid2vidLocal.map(_.result)) - } - } -} From 4b6b15dadde3e7dd2f1f38f895a7376656aa4852 Mon Sep 17 00:00:00 2001 From: Dan Crankshaw Date: Sun, 24 Nov 2013 16:38:38 -0800 Subject: [PATCH 226/531] Actually use partitioner command line args in Analytics. --- .../org/apache/spark/graph/Analytics.scala | 30 +++++++++++-------- 1 file changed, 18 insertions(+), 12 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala index 7d7e05e2266be..042bcd9825f41 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala @@ -298,6 +298,16 @@ object Analytics extends Logging { loggerName -> prevLevel }.toMap } + + def pickPartitioner(v: String): PartitionStrategy = { + v match { + case "RandomVertexCut" => RandomVertexCut() + case "EdgePartition1D" => EdgePartition1D() + case "EdgePartition2D" => EdgePartition2D() + case "CanonicalRandomVertexCut" => CanonicalRandomVertexCut() + case _ => throw new IllegalArgumentException("Invalid Partition Strategy: " + v) + } + } // setLogLevels(org.apache.log4j.Level.DEBUG, Seq("org.apache.spark")) val serializer = "org.apache.spark.serializer.KryoSerializer" @@ -323,15 +333,7 @@ object Analytics extends Logging { case ("output", v) => outFname = v case ("numVPart", v) => numVPart = v.toInt case ("numEPart", v) => numEPart = v.toInt - case ("partStrategy", v) => { - partitionStrategy = v match { - case "RandomVertexCut" => RandomVertexCut() - case "EdgePartition1D" => EdgePartition1D() - case "EdgePartition2D" => EdgePartition2D() - case "CanonicalRandomVertexCut" => CanonicalRandomVertexCut() - case _ => throw new IllegalArgumentException("Invalid Partition Strategy: " + v) - } - } + case ("partStrategy", v) => partitionStrategy = pickPartitioner(v) case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) } @@ -351,7 +353,7 @@ object Analytics extends Logging { val sc = new SparkContext(host, "PageRank(" + fname + ")") val graph = GraphLoader.edgeListFile(sc, fname, - minEdgePartitions = numEPart).cache() + minEdgePartitions = numEPart, partitionStrategy=partitionStrategy).cache() val startTime = System.currentTimeMillis logInfo("GRAPHX: starting tasks") @@ -377,12 +379,14 @@ object Analytics extends Logging { var numVPart = 4 var numEPart = 4 var isDynamic = false + var partitionStrategy: PartitionStrategy = RandomVertexCut() options.foreach{ case ("numIter", v) => numIter = v.toInt case ("dynamic", v) => isDynamic = v.toBoolean 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) } @@ -400,7 +404,7 @@ object Analytics extends Logging { val sc = new SparkContext(host, "ConnectedComponents(" + fname + ")") val graph = GraphLoader.edgeListFile(sc, fname, - minEdgePartitions = numEPart).cache() + minEdgePartitions = numEPart, partitionStrategy=partitionStrategy).cache() val cc = Analytics.connectedComponents(graph) println("Components: " + cc.vertices.map{ case (vid,data) => data}.distinct()) sc.stop() @@ -409,10 +413,12 @@ object Analytics extends Logging { case "triangles" => { var numVPart = 4 var numEPart = 4 + 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("======================================") @@ -420,7 +426,7 @@ object Analytics extends Logging { println("--------------------------------------") val sc = new SparkContext(host, "TriangleCount(" + fname + ")") val graph = GraphLoader.edgeListFile(sc, fname, canonicalOrientation = true, - minEdgePartitions = numEPart).cache() + minEdgePartitions = numEPart, partitionStrategy=partitionStrategy).cache() val triangles = Analytics.triangleCount(graph) println("Triangles: " + triangles.vertices.map { case (vid,data) => data.toLong From d074e4c6ab577b8206c493b9e305f4895336cc19 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 26 Nov 2013 02:49:41 -0800 Subject: [PATCH 227/531] Bring PrimitiveVector up to date. --- .../util/collection/PrimitiveVector.scala | 46 +++++++++++++------ 1 file changed, 31 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/collection/PrimitiveVector.scala b/core/src/main/scala/org/apache/spark/util/collection/PrimitiveVector.scala index 369519c5595de..20554f0aaba70 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/PrimitiveVector.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/PrimitiveVector.scala @@ -17,35 +17,51 @@ package org.apache.spark.util.collection -/** Provides a simple, non-threadsafe, array-backed vector that can store primitives. */ +/** + * An append-only, non-threadsafe, array-backed vector that is optimized for primitive types. + */ private[spark] class PrimitiveVector[@specialized(Long, Int, Double) V: ClassManifest](initialSize: Int = 64) { - private var numElements = 0 - private var array: Array[V] = _ + private var _numElements = 0 + private var _array: Array[V] = _ // NB: This must be separate from the declaration, otherwise the specialized parent class - // will get its own array with the same initial size. TODO: Figure out why... - array = new Array[V](initialSize) + // will get its own array with the same initial size. + _array = new Array[V](initialSize) def apply(index: Int): V = { - require(index < numElements) - array(index) + require(index < _numElements) + _array(index) } def +=(value: V) { - if (numElements == array.length) { resize(array.length * 2) } - array(numElements) = value - numElements += 1 + if (_numElements == _array.length) { + resize(_array.length * 2) + } + _array(_numElements) = value + _numElements += 1 } - def length = numElements + def capacity: Int = _array.length + + def length: Int = _numElements + + def size: Int = _numElements + + /** Gets the underlying array backing this vector. */ + def array: Array[V] = _array - def getUnderlyingArray = array + /** Trims this vector so that the capacity is equal to the size. */ + def trim(): PrimitiveVector[V] = resize(size) /** Resizes the array, dropping elements if the total length decreases. */ - def resize(newLength: Int) { + def resize(newLength: Int): PrimitiveVector[V] = { val newArray = new Array[V](newLength) - array.copyToArray(newArray) - array = newArray + _array.copyToArray(newArray) + _array = newArray + if (newLength < _numElements) { + _numElements = newLength + } + this } } From d58bfa85738adb57a5311f52f52ede5903a181e1 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 26 Nov 2013 13:54:46 -0800 Subject: [PATCH 228/531] Code cleaning to improve readability. --- .../spark/util/collection/OpenHashSet.scala | 2 +- .../org/apache/spark/graph/Analytics.scala | 4 +- .../scala/org/apache/spark/graph/Graph.scala | 20 +- .../spark/graph/GraphKryoRegistrator.scala | 3 +- .../org/apache/spark/graph/VertexSetRDD.scala | 121 +++---- .../graph/impl/EdgePartitionBuilder.scala | 19 +- .../graph/impl/EdgeTripletIterator.scala | 60 ++++ .../apache/spark/graph/impl/GraphImpl.scala | 311 +++++++----------- .../spark/graph/impl/VTableReplicated.scala | 81 ++--- .../spark/graph/impl/VertexPartition.scala | 44 +-- .../spark/graph/impl/VertexPlacement.scala | 25 +- .../org/apache/spark/graph/package.scala | 10 +- 12 files changed, 315 insertions(+), 385 deletions(-) create mode 100644 graph/src/main/scala/org/apache/spark/graph/impl/EdgeTripletIterator.scala diff --git a/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala b/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala index 84583bf20f9e7..b8716f1db70e1 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala @@ -158,7 +158,7 @@ class OpenHashSet[@specialized(Long, Int) T: ClassManifest]( /** Return the value at the specified position. */ def getValue(pos: Int): T = _data(pos) - def iterator() = new Iterator[T] { + def iterator = new Iterator[T] { var pos = nextPos(0) override def hasNext: Boolean = pos != INVALID_POS override def next(): T = { diff --git a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala index 7d7e05e2266be..28a0f85e92b3c 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala @@ -229,7 +229,7 @@ object Analytics extends Logging { // Construct set representations of the neighborhoods val nbrSets: VertexSetRDD[VertexSet] = - graph.collectNeighborIds(EdgeDirection.Both).mapValuesWithKeys { (vid, nbrs) => + graph.collectNeighborIds(EdgeDirection.Both).mapValues { (vid, nbrs) => val set = new VertexSet(4) var i = 0 while (i < nbrs.size) { @@ -254,7 +254,7 @@ object Analytics extends Logging { } else { (et.dstAttr, et.srcAttr) } - val iter = smallSet.iterator() + val iter = smallSet.iterator var counter: Int = 0 while (iter.hasNext) { val vid = iter.next diff --git a/graph/src/main/scala/org/apache/spark/graph/Graph.scala b/graph/src/main/scala/org/apache/spark/graph/Graph.scala index 42ede2d420a0b..2ebe5cf08300e 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Graph.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Graph.scala @@ -1,8 +1,10 @@ package org.apache.spark.graph +import org.apache.spark.graph.impl._ import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel + /** * The Graph abstractly represents a graph with arbitrary objects * associated with vertices and edges. The graph provides basic @@ -162,7 +164,6 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * Construct a new graph with all the edges reversed. If this graph * contains an edge from a to b then the returned graph contains an * edge from b to a. - * */ def reverse: Graph[VD, ED] @@ -292,9 +293,6 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { */ object Graph { - import org.apache.spark.graph.impl._ - import org.apache.spark.SparkContext._ - /** * Construct a graph from a collection of edges encoded as vertex id pairs. * @@ -324,15 +322,10 @@ object Graph { rawEdges: RDD[(Vid, Vid)], defaultValue: VD, uniqueEdges: Boolean, - partitionStrategy: PartitionStrategy): - Graph[VD, Int] = { + partitionStrategy: PartitionStrategy): Graph[VD, Int] = { val edges = rawEdges.map(p => Edge(p._1, p._2, 1)) val graph = GraphImpl(edges, defaultValue, partitionStrategy) - if (uniqueEdges) { - graph.groupEdges((a,b) => a+b) - } else { - graph - } + if (uniqueEdges) graph.groupEdges((a, b) => a + b) else graph } /** @@ -344,9 +337,8 @@ object Graph { * @return a graph with edge attributes described by `edges` and vertices * given by all vertices in `edges` with value `defaultValue` */ - def apply[VD: ClassManifest, ED: ClassManifest]( - edges: RDD[Edge[ED]], - defaultValue: VD): Graph[VD, ED] = { + def apply[VD: ClassManifest, ED: ClassManifest](edges: RDD[Edge[ED]], defaultValue: VD) + : Graph[VD, ED] = { Graph(edges, defaultValue, RandomVertexCut()) } diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphKryoRegistrator.scala b/graph/src/main/scala/org/apache/spark/graph/GraphKryoRegistrator.scala index 6f18e46ab2a54..d04e82a0a333a 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphKryoRegistrator.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphKryoRegistrator.scala @@ -5,13 +5,12 @@ import com.esotericsoftware.kryo.Kryo import org.apache.spark.graph.impl._ import org.apache.spark.serializer.KryoRegistrator import org.apache.spark.util.collection.BitSet -import org.apache.spark.graph._ + class GraphKryoRegistrator extends KryoRegistrator { def registerClasses(kryo: Kryo) { kryo.register(classOf[Edge[Object]]) - kryo.register(classOf[MutableTuple2[Object, Object]]) kryo.register(classOf[MessageToPartition[Object]]) kryo.register(classOf[VertexBroadcastMsg[Object]]) kryo.register(classOf[AggregationMsg[Object]]) diff --git a/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala b/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala index ed70402a6f6f5..a054ab0357a93 100644 --- a/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala @@ -23,23 +23,22 @@ import org.apache.spark.rdd._ import org.apache.spark.storage.StorageLevel import org.apache.spark.util.collection.{BitSet, OpenHashSet, PrimitiveKeyOpenHashMap} -import org.apache.spark.graph.impl.AggregationMsg -import org.apache.spark.graph.impl.MsgRDDFunctions._ import org.apache.spark.graph.impl.VertexPartition + /** * Maintains the per-partition mapping from vertex id to the corresponding * location in the per-partition values array. This class is meant to be an * opaque type. - * */ +private[graph] class VertexSetIndex(private[spark] val rdd: RDD[VertexIdToIndexMap]) { /** * The persist function behaves like the standard RDD persist */ def persist(newLevel: StorageLevel): VertexSetIndex = { rdd.persist(newLevel) - return this + this } /** @@ -80,6 +79,8 @@ class VertexSetRDD[@specialized VD: ClassManifest]( /** * The `VertexSetIndex` representing the layout of this `VertexSetRDD`. */ + // TOOD: Consider removing the exposure of index to outside, and implement methods in this + // class to handle any operations that would require indexing. def index = new VertexSetIndex(partitionsRDD.mapPartitions(_.map(_.index), preservesPartitioning = true)) @@ -134,13 +135,12 @@ class VertexSetRDD[@specialized VD: ClassManifest]( tuples.compute(part, context) /** - * Return a new VertexSetRDD by applying a function to each VertexPartition of - * this RDD. + * Return a new VertexSetRDD by applying a function to each VertexPartition of this RDD. */ - def mapVertexPartitions[VD2: ClassManifest]( - f: VertexPartition[VD] => VertexPartition[VD2]): VertexSetRDD[VD2] = { + def mapVertexPartitions[VD2: ClassManifest](f: VertexPartition[VD] => VertexPartition[VD2]) + : VertexSetRDD[VD2] = { val cleanF = sparkContext.clean(f) - val newPartitionsRDD = partitionsRDD.mapPartitions(_.map(f), preservesPartitioning = true) + val newPartitionsRDD = partitionsRDD.mapPartitions(_.map(cleanF), preservesPartitioning = true) new VertexSetRDD(newPartitionsRDD) } @@ -175,6 +175,7 @@ class VertexSetRDD[@specialized VD: ClassManifest]( * the original vertex-set. Furthermore, the filter only * modifies the bitmap index and so no new values are allocated. */ + // TODO: Should we consider making pred taking two arguments, instead of a tuple? override def filter(pred: Tuple2[Vid, VD] => Boolean): VertexSetRDD[VD] = this.mapVertexPartitions(_.filter(Function.untupled(pred))) @@ -190,13 +191,26 @@ class VertexSetRDD[@specialized VD: ClassManifest]( * VertexSetRDD retains the same index. */ def mapValues[VD2: ClassManifest](f: VD => VD2): VertexSetRDD[VD2] = - this.mapVertexPartitions(_.map { case (vid, attr) => f(attr) }) + this.mapVertexPartitions(_.map((vid, attr) => f(attr))) + + /** + * Pass each vertex attribute through a map function and retain the + * original RDD's partitioning and index. + * + * @tparam VD2 the type returned by the map function + * + * @param f the function applied to each value in the RDD + * @return a new VertexSetRDD with values obtained by applying `f` to + * each of the entries in the original VertexSet. The resulting + * VertexSetRDD retains the same index. + */ + def mapValues[VD2: ClassManifest](f: (Vid, VD) => VD2): VertexSetRDD[VD2] = + this.mapVertexPartitions(_.map(f)) /** * Fill in missing values for all vertices in the index. * - * @param missingValue the value to be used for vertices in the - * index that don't currently have values. + * @param missingValue the value to use for vertices that don't currently have values. * @return A VertexSetRDD with a value for all vertices. */ def fillMissing(missingValue: VD): VertexSetRDD[VD] = { @@ -213,29 +227,6 @@ class VertexSetRDD[@specialized VD: ClassManifest]( } } - /** - * Pass each vertex attribute along with the vertex id through a map - * function and retain the original RDD's partitioning and index. - * - * @tparam VD2 the type returned by the map function - * - * @param f the function applied to each vertex id and vertex - * attribute in the RDD - * @return a new VertexSet with values obtained by applying `f` to - * each of the entries in the original VertexSet. The resulting - * VertexSetRDD retains the same index. - */ - def mapValuesWithKeys[VD2: ClassManifest](f: (Vid, VD) => VD2): VertexSetRDD[VD2] = { - this.mapVertexPartitions { part => - // Construct a view of the map transformation - val newValues = new Array[VD2](part.index.capacity) - part.mask.iterator.foreach { ind => - newValues(ind) = f(part.index.getValueSafe(ind), part.values(ind)) - } - new VertexPartition(part.index, newValues, part.mask) - } - } // end of mapValuesWithKeys - /** * Inner join this VertexSet with another VertexSet which has the * same Index. This function will fail if both VertexSets do not @@ -269,35 +260,6 @@ class VertexSetRDD[@specialized VD: ClassManifest]( } } - /** - * Inner join this VertexSet with another VertexSet which has the - * same Index. This function will fail if both VertexSets do not - * share the same index. - * - * @param other the vertex set to join with this vertex set - * @param f the function mapping a vertex id and its attributes in - * this and the other vertex set to a collection of tuples. - * @tparam VD2 the type of the other vertex set attributes - * @tparam VD3 the type of the tuples emitted by `f` - * @return an RDD containing the tuples emitted by `f` - */ - def zipJoinFlatMap[VD2: ClassManifest, VD3: ClassManifest] - (other: VertexSetRDD[VD2]) - (f: (Vid, VD, VD2) => Iterator[VD3]): RDD[VD3] = { - val cleanF = sparkContext.clean(f) - partitionsRDD.zipPartitions(other.partitionsRDD) { - (thisPartIter, otherPartIter) => - val thisPart = thisPartIter.next() - val otherPart = otherPartIter.next() - if (thisPart.index != otherPart.index) { - throw new SparkException("can't zip join VertexSetRDDs with different indexes") - } - (thisPart.mask & otherPart.mask).iterator.flatMap { ind => - cleanF(thisPart.index.getValueSafe(ind), thisPart.values(ind), otherPart.values(ind)) - } - } - } - /** * Left join this VertexSet with another VertexSet which has the * same Index. This function will fail if both VertexSets do not @@ -333,7 +295,6 @@ class VertexSetRDD[@specialized VD: ClassManifest]( } } // end of leftZipJoin - /** * Left join this VertexSet with an RDD containing vertex attribute * pairs. If the other RDD is backed by a VertexSet with the same @@ -343,7 +304,7 @@ class VertexSetRDD[@specialized VD: ClassManifest]( * VertexSet then a `None` attribute is generated * * @tparam VD2 the attribute type of the other VertexSet - * @tparam VD2 the attribute type of the resulting VertexSet + * @tparam VD3 the attribute type of the resulting VertexSet * * @param other the other VertexSet with which to join. * @param f the function mapping a vertex id and its attributes in @@ -358,18 +319,14 @@ class VertexSetRDD[@specialized VD: ClassManifest]( (other: RDD[(Vid, VD2)]) (f: (Vid, VD, Option[VD2]) => VD3, merge: (VD2, VD2) => VD2 = (a: VD2, b: VD2) => a) : VertexSetRDD[VD3] = { - // Test if the other vertex is a VertexSetRDD to choose the optimal - // join strategy + // Test if the other vertex is a VertexSetRDD to choose the optimal join strategy. + // If the other set is a VertexSetRDD then we use the much more efficient leftZipJoin other match { - // If the other set is a VertexSetRDD then we use the much more efficient - // leftZipJoin - case other: VertexSetRDD[_] => { + case other: VertexSetRDD[VD2] => leftZipJoin(other)(f) - } - case _ => { + case _ => val indexedOther: VertexSetRDD[VD2] = VertexSetRDD(other, this.index, merge) leftZipJoin(indexedOther)(f) - } } } // end of leftJoin @@ -433,12 +390,11 @@ object VertexSetRDD { * * @tparam VD the vertex attribute type * @param rdd the rdd containing vertices - * @param indexPrototype a VertexSetRDD whose indexes will be reused. The + * @param index a VertexSetRDD whose indexes will be reused. The * indexes must be a superset of the vertices in rdd * in RDD */ - def apply[VD: ClassManifest]( - rdd: RDD[(Vid, VD)], index: VertexSetIndex): VertexSetRDD[VD] = + def apply[VD: ClassManifest](rdd: RDD[(Vid, VD)], index: VertexSetIndex): VertexSetRDD[VD] = apply(rdd, index, (a: VD, b: VD) => a) /** @@ -447,8 +403,8 @@ object VertexSetRDD { * * @tparam VD the vertex attribute type * @param rdd the rdd containing vertices - * @param indexPrototype a VertexSetRDD whose indexes will be reused. The - * indexes must be a superset of the vertices in rdd + * @param index a VertexSetRDD whose indexes will be reused. The + * indexes must be a superset of the vertices in rdd * @param reduceFunc the user defined reduce function used to merge * duplicate vertex attributes. */ @@ -456,6 +412,7 @@ object VertexSetRDD { rdd: RDD[(Vid, VD)], index: VertexSetIndex, reduceFunc: (VD, VD) => VD): VertexSetRDD[VD] = + // TODO: Considering removing the following apply. apply(rdd, index, (v: VD) => v, reduceFunc, reduceFunc) /** @@ -463,7 +420,7 @@ object VertexSetRDD { * * @tparam VD the vertex attribute type * @param rdd the rdd containing vertices - * @param indexPrototype a VertexSetRDD whose indexes will be reused. The + * @param index a VertexSetRDD whose indexes will be reused. The * indexes must be a superset of the vertices in rdd * @param reduceFunc the user defined reduce function used to merge * duplicate vertex attributes. @@ -476,9 +433,7 @@ object VertexSetRDD { val cReduceFunc = rdd.context.clean(reduceFunc) assert(rdd.partitioner == Some(index.partitioner)) // Use the index to build the new values table - val partitionsRDD = index.rdd.zipPartitions( - rdd, preservesPartitioning = true - ) { + val partitionsRDD = index.rdd.zipPartitions(rdd, preservesPartitioning = true) { (indexIter, tblIter) => // There is only one map val index = indexIter.next() diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartitionBuilder.scala b/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartitionBuilder.scala index 355f8f05427c2..76c11a364cfda 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartitionBuilder.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartitionBuilder.scala @@ -1,27 +1,24 @@ package org.apache.spark.graph.impl -import scala.collection.mutable.ArrayBuilder import org.apache.spark.graph._ +import org.apache.spark.util.collection.PrimitiveVector //private[graph] -class EdgePartitionBuilder[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) -ED: ClassManifest]{ - val srcIds = new VertexArrayList - val dstIds = new VertexArrayList - var dataBuilder = ArrayBuilder.make[ED] +class EdgePartitionBuilder[@specialized(Long, Int, Double) ED: ClassManifest] { + val srcIds = new PrimitiveVector[Vid] + val dstIds = new PrimitiveVector[Vid] + var dataBuilder = new PrimitiveVector[ED] /** Add a new edge to the partition. */ def add(src: Vid, dst: Vid, d: ED) { - srcIds.add(src) - dstIds.add(dst) + srcIds += src + dstIds += dst dataBuilder += d } def toEdgePartition: EdgePartition[ED] = { - new EdgePartition(srcIds.toLongArray(), dstIds.toLongArray(), dataBuilder.result()) + new EdgePartition(srcIds.trim().array, dstIds.trim().array, dataBuilder.trim().array) } - - } diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/EdgeTripletIterator.scala b/graph/src/main/scala/org/apache/spark/graph/impl/EdgeTripletIterator.scala new file mode 100644 index 0000000000000..f23f68cf22ca8 --- /dev/null +++ b/graph/src/main/scala/org/apache/spark/graph/impl/EdgeTripletIterator.scala @@ -0,0 +1,60 @@ +package org.apache.spark.graph.impl + +import scala.collection.mutable + +import org.apache.spark.graph._ +import org.apache.spark.util.collection.PrimitiveKeyOpenHashMap + + +/** + * The Iterator type returned when constructing edge triplets. This class technically could be + * an anonymous class in GraphImpl.triplets, but we name it here explicitly so it is easier to + * debug / profile. + */ +private[impl] +class EdgeTripletIterator[VD: ClassManifest, ED: ClassManifest]( + val vidToIndex: VertexIdToIndexMap, + val vertexArray: Array[VD], + val edgePartition: EdgePartition[ED]) + extends Iterator[EdgeTriplet[VD, ED]] { + + // Current position in the array. + private var pos = 0 + + // A triplet object that this iterator.next() call returns. We reuse this object to avoid + // allocating too many temporary Java objects. + private val triplet = new EdgeTriplet[VD, ED] + + private val vmap = new PrimitiveKeyOpenHashMap[Vid, VD](vidToIndex, vertexArray) + + override def hasNext: Boolean = pos < edgePartition.size + + override def next() = { + triplet.srcId = edgePartition.srcIds(pos) + // assert(vmap.containsKey(e.src.id)) + triplet.srcAttr = vmap(triplet.srcId) + triplet.dstId = edgePartition.dstIds(pos) + // assert(vmap.containsKey(e.dst.id)) + triplet.dstAttr = vmap(triplet.dstId) + triplet.attr = edgePartition.data(pos) + pos += 1 + triplet + } + + // TODO: Why do we need this? + override def toList: List[EdgeTriplet[VD, ED]] = { + val lb = new mutable.ListBuffer[EdgeTriplet[VD,ED]] + val currentEdge = new EdgeTriplet[VD, ED] + for (i <- 0 until edgePartition.size) { + currentEdge.srcId = edgePartition.srcIds(i) + // assert(vmap.containsKey(e.src.id)) + currentEdge.srcAttr = vmap(currentEdge.srcId) + currentEdge.dstId = edgePartition.dstIds(i) + // assert(vmap.containsKey(e.dst.id)) + currentEdge.dstAttr = vmap(currentEdge.dstId) + currentEdge.attr = edgePartition.data(i) + lb += currentEdge + } + lb.toList + } +} diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index 337d560c2659b..db0b940a4be30 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -1,68 +1,17 @@ package org.apache.spark.graph.impl -import scala.collection.JavaConversions._ - -import scala.collection.mutable -import scala.collection.mutable.ArrayBuffer - import org.apache.spark.SparkContext._ -import org.apache.spark.HashPartitioner -import org.apache.spark.util.ClosureCleaner -import org.apache.spark.SparkException - -import org.apache.spark.Partitioner +import org.apache.spark.{HashPartitioner, Partitioner} import org.apache.spark.graph._ import org.apache.spark.graph.impl.GraphImpl._ import org.apache.spark.graph.impl.MsgRDDFunctions._ import org.apache.spark.graph.util.BytecodeUtils import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.ClosureCleaner import org.apache.spark.util.collection.{BitSet, OpenHashSet, PrimitiveKeyOpenHashMap} -/** - * The Iterator type returned when constructing edge triplets - */ -class EdgeTripletIterator[VD: ClassManifest, ED: ClassManifest]( - val vidToIndex: VertexIdToIndexMap, - val vertexArray: Array[VD], - val edgePartition: EdgePartition[ED]) extends Iterator[EdgeTriplet[VD, ED]] { - - private var pos = 0 - private val et = new EdgeTriplet[VD, ED] - private val vmap = new PrimitiveKeyOpenHashMap[Vid, VD](vidToIndex, vertexArray) - - override def hasNext: Boolean = pos < edgePartition.size - override def next() = { - et.srcId = edgePartition.srcIds(pos) - // assert(vmap.containsKey(e.src.id)) - et.srcAttr = vmap(et.srcId) - et.dstId = edgePartition.dstIds(pos) - // assert(vmap.containsKey(e.dst.id)) - et.dstAttr = vmap(et.dstId) - et.attr = edgePartition.data(pos) - pos += 1 - et - } - - override def toList: List[EdgeTriplet[VD, ED]] = { - val lb = new mutable.ListBuffer[EdgeTriplet[VD,ED]] - val currentEdge = new EdgeTriplet[VD, ED] - for (i <- (0 until edgePartition.size)) { - currentEdge.srcId = edgePartition.srcIds(i) - // assert(vmap.containsKey(e.src.id)) - currentEdge.srcAttr = vmap(currentEdge.srcId) - currentEdge.dstId = edgePartition.dstIds(i) - // assert(vmap.containsKey(e.dst.id)) - currentEdge.dstAttr = vmap(currentEdge.dstId) - currentEdge.attr = edgePartition.data(i) - lb += currentEdge - } - lb.toList - } -} // end of Edge Triplet Iterator - - /** * A Graph RDD that supports computation on graphs. * @@ -95,8 +44,13 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( eTable.mapPartitions(_.next()._2.iterator, true) /** Return a RDD that brings edges with its source and destination vertices together. */ - @transient override val triplets: RDD[EdgeTriplet[VD, ED]] = - makeTriplets(vTableReplicated.bothAttrs, eTable) + @transient override val triplets: RDD[EdgeTriplet[VD, ED]] = { + eTable.zipPartitions(vTableReplicated.bothAttrs) { (eTableIter, vTableReplicatedIter) => + val (_, edgePartition) = eTableIter.next() + val (_, (vidToIndex, vertexArray)) = vTableReplicatedIter.next() + new EdgeTripletIterator(vidToIndex, vertexArray, edgePartition) + } + } override def persist(newLevel: StorageLevel): Graph[VD, ED] = { vTable.persist(newLevel) @@ -108,25 +62,29 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( override def cache(): Graph[VD, ED] = persist(StorageLevel.MEMORY_ONLY) override def statistics: Map[String, Any] = { - val numVertices = this.numVertices - val numEdges = this.numEdges - val replicationRatioBothAttrs = - vertexPlacement.bothAttrs.map(_.map(_.size).sum).sum / numVertices - val replicationRatioSrcAttrOnly = - vertexPlacement.srcAttrOnly.map(_.map(_.size).sum).sum / numVertices - val replicationRatioDstAttrOnly = - vertexPlacement.dstAttrOnly.map(_.map(_.size).sum).sum / numVertices - val loadArray = - eTable.map{ case (pid, epart) => epart.data.size }.collect.map(x => x.toDouble / numEdges) + // Get the total number of vertices after replication, used to compute the replication ratio. + def numReplicatedVertices(vid2pids: RDD[Array[Array[Vid]]]): Double = { + vid2pids.map(_.map(_.size).sum.toLong).reduce(_ + _).toDouble + } + + val numVertices = this.ops.numVertices + val numEdges = this.ops.numEdges + val replicationRatioBoth = numReplicatedVertices(vertexPlacement.bothAttrs) / numVertices + val replicationRatioSrcOnly = numReplicatedVertices(vertexPlacement.srcAttrOnly) / numVertices + val replicationRatioDstOnly = numReplicatedVertices(vertexPlacement.dstAttrOnly) / numVertices + // One entry for each partition, indicate the total number of edges on that partition. + val loadArray = eTable.map { case (_, e) => e.size }.collect().map(_.toDouble / numEdges) val minLoad = loadArray.min val maxLoad = loadArray.max Map( - "Num Vertices" -> numVertices, "Num Edges" -> numEdges, - "Replication (both)" -> replicationRatioBothAttrs, - "Replication (src only)" -> replicationRatioSrcAttrOnly, - "Replication (dest only)" -> replicationRatioDstAttrOnly, + "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) + "Min Load" -> minLoad, + "Max Load" -> maxLoad) } /** @@ -137,7 +95,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( rdd: RDD[_], indent: String = "", visited: Map[Int, String] = Map.empty[Int, String]) { - if(visited.contains(rdd.id)) { + if (visited.contains(rdd.id)) { println(indent + visited(rdd.id)) println(indent) } else { @@ -169,7 +127,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( println("\n\ntriplets ----------------------------------------") traverseLineage(triplets, " ", visited) println(visited) - } // end of print lineage + } // end of printLineage override def reverse: Graph[VD, ED] = { val newETable = eTable.mapPartitions(_.map { case (pid, epart) => (pid, epart.reverse) }, @@ -186,12 +144,31 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( new GraphImpl(vTable, newETable, vertexPlacement, partitioner) } - override def mapTriplets[ED2: ClassManifest](f: EdgeTriplet[VD, ED] => ED2): Graph[VD, ED2] = - GraphImpl.mapTriplets(this, f) + override def mapTriplets[ED2: ClassManifest](f: EdgeTriplet[VD, ED] => ED2): Graph[VD, ED2] = { + // Use an explicit manifest in PrimitiveKeyOpenHashMap init so we don't pull in the implicit + // manifest from GraphImpl (which would require serializing GraphImpl). + val vdManifest = classManifest[VD] + val newETable = eTable.zipPartitions(vTableReplicated.bothAttrs, preservesPartitioning = true) { + (eTableIter, vTableReplicatedIter) => + val (pid, edgePartition) = eTableIter.next() + val (_, (vidToIndex, vertexArray)) = vTableReplicatedIter.next() + val et = new EdgeTriplet[VD, ED] + val vmap = new PrimitiveKeyOpenHashMap[Vid, VD]( + vidToIndex, vertexArray)(classManifest[Vid], vdManifest) + val newEdgePartition = edgePartition.map { e => + et.set(e) + et.srcAttr = vmap(e.srcId) + et.dstAttr = vmap(e.dstId) + f(et) + } + Iterator((pid, newEdgePartition)) + } + new GraphImpl(vTable, newETable, vertexPlacement, partitioner) + } override def subgraph( - epred: EdgeTriplet[VD, ED] => Boolean = (x => true), - vpred: (Vid, VD) => Boolean = ((a,b) => true)): Graph[VD, ED] = { + epred: EdgeTriplet[VD, ED] => Boolean = x => true, + vpred: (Vid, VD) => Boolean = (a, b) => true): Graph[VD, ED] = { // Filter the vertices, reusing the partitioner (but not the index) from // this graph @@ -222,8 +199,68 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( override def mapReduceTriplets[A: ClassManifest]( mapFunc: EdgeTriplet[VD, ED] => Iterator[(Vid, A)], - reduceFunc: (A, A) => A): VertexSetRDD[A] = - GraphImpl.mapReduceTriplets(this, mapFunc, reduceFunc) + reduceFunc: (A, A) => A): VertexSetRDD[A] = { + + ClosureCleaner.clean(mapFunc) + ClosureCleaner.clean(reduceFunc) + + // Use explicit manifest in PrimitiveKeyOpenHashMap so we don't have to serialize GraphImpl. + val vdManifest = classManifest[VD] + + // 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 vs = vTableReplicated.get(mapUsesSrcAttr, mapUsesDstAttr) + + // Map and combine. + val preAgg = eTable.zipPartitions(vs) { (edgePartitionIter, vTableReplicatedIter) => + val (_, edgePartition) = edgePartitionIter.next() + val (_, (vidToIndex, vertexArray)) = vTableReplicatedIter.next() + assert(vidToIndex.capacity == vertexArray.size) + val vmap = new PrimitiveKeyOpenHashMap[Vid, VD](vidToIndex, vertexArray)( + classManifest[Vid], vdManifest) + + // Note: This doesn't allow users to send messages to arbitrary vertices. + val msgArray = new Array[A](vertexArray.size) + val msgBS = new BitSet(vertexArray.size) + // Iterate over the partition + val et = new EdgeTriplet[VD, ED] + + edgePartition.foreach { e => + et.set(e) + if (mapUsesSrcAttr) { + et.srcAttr = vmap(e.srcId) + } + if (mapUsesDstAttr) { + et.dstAttr = vmap(e.dstId) + } + // TODO(rxin): rewrite the foreach using a simple while loop to speed things up. + // Also given we are only allowing zero, one, or two messages, we can completely unroll + // the for loop. + mapFunc(et).foreach { case (vid, msg) => + // verify that the vid is valid + assert(vid == et.srcId || vid == et.dstId) + // Get the index of the key + val ind = vidToIndex.getPos(vid) & OpenHashSet.POSITION_MASK + // Populate the aggregator map + if (msgBS.get(ind)) { + msgArray(ind) = reduceFunc(msgArray(ind), msg) + } else { + msgArray(ind) = msg + msgBS.set(ind) + } + } + } + // construct an iterator of tuples Iterator[(Vid, A)] + msgBS.iterator.map { ind => + new AggregationMsg[A](vidToIndex.getValue(ind), msgArray(ind)) + } + }.partitionBy(vTable.partitioner.get) + + // do the final reduction reusing the index map + VertexSetRDD.aggregate(preAgg, vTable.index, reduceFunc) + } // end of mapReduceTriplets override def outerJoinVertices[U: ClassManifest, VD2: ClassManifest] (updates: RDD[(Vid, U)])(updateF: (Vid, VD, Option[U]) => VD2): Graph[VD2, ED] = { @@ -255,29 +292,6 @@ object GraphImpl { new GraphImpl(vtable, etable, vertexPlacement, partitionStrategy) } - // def apply[VD: ClassManifest, ED: ClassManifest]( - // vertices: RDD[(Vid, VD)], - // edges: RDD[Edge[ED]], - // defaultVertexAttr: VD): GraphImpl[VD,ED] = { - // apply(vertices, edges, defaultVertexAttr, (a:VD, b:VD) => a, RandomVertexCut()) - // } - - // def apply[VD: ClassManifest, ED: ClassManifest]( - // vertices: RDD[(Vid, VD)], - // edges: RDD[Edge[ED]], - // defaultVertexAttr: VD, - // partitionStrategy: PartitionStrategy): GraphImpl[VD,ED] = { - // apply(vertices, edges, defaultVertexAttr, (a:VD, b:VD) => a, partitionStrategy) - // } - - // def apply[VD: ClassManifest, ED: ClassManifest]( - // vertices: RDD[(Vid, VD)], - // edges: RDD[Edge[ED]], - // defaultVertexAttr: VD, - // mergeFunc: (VD, VD) => VD): GraphImpl[VD,ED] = { - // apply(vertices, edges, defaultVertexAttr, mergeFunc, RandomVertexCut()) - // } - def apply[VD: ClassManifest, ED: ClassManifest]( vertices: RDD[(Vid, VD)], edges: RDD[Edge[ED]], @@ -336,100 +350,7 @@ object GraphImpl { }, preservesPartitioning = true).cache() } - protected def makeTriplets[VD: ClassManifest, ED: ClassManifest]( - vTableReplicated: RDD[(Pid, (VertexIdToIndexMap, Array[VD]))], - eTable: RDD[(Pid, EdgePartition[ED])]): RDD[EdgeTriplet[VD, ED]] = { - eTable.zipPartitions(vTableReplicated) { - (eTableIter, vTableReplicatedIter) => - val (_, edgePartition) = eTableIter.next() - val (_, (vidToIndex, vertexArray)) = vTableReplicatedIter.next() - new EdgeTripletIterator(vidToIndex, vertexArray, edgePartition) - } - } - - protected def mapTriplets[VD: ClassManifest, ED: ClassManifest, ED2: ClassManifest]( - g: GraphImpl[VD, ED], - f: EdgeTriplet[VD, ED] => ED2): Graph[VD, ED2] = { - val newETable = g.eTable.zipPartitions( - g.vTableReplicated.bothAttrs, preservesPartitioning = true - ) { - (eTableIter, vTableReplicatedIter) => - val (pid, edgePartition) = eTableIter.next() - val (_, (vidToIndex, vertexArray)) = vTableReplicatedIter.next() - val et = new EdgeTriplet[VD, ED] - val vmap = new PrimitiveKeyOpenHashMap[Vid, VD](vidToIndex, vertexArray) - val newEdgePartition = edgePartition.map { e => - et.set(e) - et.srcAttr = vmap(e.srcId) - et.dstAttr = vmap(e.dstId) - f(et) - } - Iterator((pid, newEdgePartition)) - } - new GraphImpl(g.vTable, newETable, g.vertexPlacement, g.partitioner) - } - - protected def mapReduceTriplets[VD: ClassManifest, ED: ClassManifest, A: ClassManifest]( - g: GraphImpl[VD, ED], - mapFunc: EdgeTriplet[VD, ED] => Iterator[(Vid, A)], - reduceFunc: (A, A) => A): VertexSetRDD[A] = { - ClosureCleaner.clean(mapFunc) - ClosureCleaner.clean(reduceFunc) - // For each vertex, replicate its attribute only to partitions where it is - // in the relevant position in an edge. - val mapFuncUsesSrcAttr = accessesVertexAttr[VD, ED](mapFunc, "srcAttr") - val mapFuncUsesDstAttr = accessesVertexAttr[VD, ED](mapFunc, "dstAttr") - // Map and preaggregate - val preAgg = g.eTable.zipPartitions( - g.vTableReplicated.get(mapFuncUsesSrcAttr, mapFuncUsesDstAttr) - ) { - (edgePartitionIter, vTableReplicatedIter) => - val (_, edgePartition) = edgePartitionIter.next() - val (_, (vidToIndex, vertexArray)) = vTableReplicatedIter.next() - assert(vidToIndex.capacity == vertexArray.size) - val vmap = new PrimitiveKeyOpenHashMap[Vid, VD](vidToIndex, vertexArray) - // TODO(jegonzal): This doesn't allow users to send messages to arbitrary vertices. - val msgArray = new Array[A](vertexArray.size) - val msgBS = new BitSet(vertexArray.size) - // Iterate over the partition - val et = new EdgeTriplet[VD, ED] - - edgePartition.foreach { e => - et.set(e) - if (mapFuncUsesSrcAttr) { - et.srcAttr = vmap(e.srcId) - } - if (mapFuncUsesDstAttr) { - et.dstAttr = vmap(e.dstId) - } - // TODO(rxin): rewrite the foreach using a simple while loop to speed things up. - // Also given we are only allowing zero, one, or two messages, we can completely unroll - // the for loop. - mapFunc(et).foreach { case (vid, msg) => - // verify that the vid is valid - assert(vid == et.srcId || vid == et.dstId) - // Get the index of the key - val ind = vidToIndex.getPos(vid) & OpenHashSet.POSITION_MASK - // Populate the aggregator map - if (msgBS.get(ind)) { - msgArray(ind) = reduceFunc(msgArray(ind), msg) - } else { - msgArray(ind) = msg - msgBS.set(ind) - } - } - } - // construct an iterator of tuples Iterator[(Vid, A)] - msgBS.iterator.map { ind => - new AggregationMsg[A](vidToIndex.getValue(ind), msgArray(ind)) - } - }.partitionBy(g.vTable.partitioner.get) - // do the final reduction reusing the index map - VertexSetRDD.aggregate(preAgg, g.vTable.index, reduceFunc) - } - - private def accessesVertexAttr[VD: ClassManifest, ED: ClassManifest]( - closure: AnyRef, attrName: String): Boolean = { + private def accessesVertexAttr[VD, ED](closure: AnyRef, attrName: String): Boolean = { try { BytecodeUtils.invokedMethod(closure, classOf[EdgeTriplet[VD, ED]], attrName) } catch { diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala b/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala index 6cbac223f7e87..8d28bb5ce166c 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala @@ -5,61 +5,60 @@ import org.apache.spark.rdd.RDD import org.apache.spark.util.collection.{OpenHashSet, PrimitiveKeyOpenHashMap} import org.apache.spark.graph._ -import org.apache.spark.graph.impl.MsgRDDFunctions._ /** * Stores the vertex attribute values after they are replicated. */ +private[impl] class VTableReplicated[VD: ClassManifest]( vTable: VertexSetRDD[VD], eTable: RDD[(Pid, EdgePartition[ED])] forSome { type ED }, vertexPlacement: VertexPlacement) { val bothAttrs: RDD[(Pid, (VertexIdToIndexMap, Array[VD]))] = - VTableReplicated.createVTableReplicated(vTable, eTable, vertexPlacement, true, true) + createVTableReplicated(vTable, eTable, vertexPlacement, true, true) + val srcAttrOnly: RDD[(Pid, (VertexIdToIndexMap, Array[VD]))] = - VTableReplicated.createVTableReplicated(vTable, eTable, vertexPlacement, true, false) + createVTableReplicated(vTable, eTable, vertexPlacement, true, false) + val dstAttrOnly: RDD[(Pid, (VertexIdToIndexMap, Array[VD]))] = - VTableReplicated.createVTableReplicated(vTable, eTable, vertexPlacement, false, true) - val noAttrs: RDD[(Pid, (VertexIdToIndexMap, Array[VD]))] = - VTableReplicated.createVTableReplicated(vTable, eTable, vertexPlacement, false, false) + createVTableReplicated(vTable, eTable, vertexPlacement, false, true) + val noAttrs: RDD[(Pid, (VertexIdToIndexMap, Array[VD]))] = + createVTableReplicated(vTable, eTable, vertexPlacement, false, false) - def get(includeSrcAttr: Boolean, includeDstAttr: Boolean) - : RDD[(Pid, (VertexIdToIndexMap, Array[VD]))] = - (includeSrcAttr, includeDstAttr) match { + def get(includeSrc: Boolean, includeDst: Boolean): RDD[(Pid, (VertexIdToIndexMap, Array[VD]))] = { + (includeSrc, includeDst) match { case (true, true) => bothAttrs case (true, false) => srcAttrOnly case (false, true) => dstAttrOnly case (false, false) => noAttrs } -} + } -class VertexAttributeBlock[VD: ClassManifest](val vids: Array[Vid], val attrs: Array[VD]) + private def createVTableReplicated[VD: ClassManifest]( + vTable: VertexSetRDD[VD], + eTable: RDD[(Pid, EdgePartition[ED])] forSome { type ED }, + vertexPlacement: VertexPlacement, + includeSrcAttr: Boolean, + includeDstAttr: Boolean): RDD[(Pid, (VertexIdToIndexMap, Array[VD]))] = { -object VTableReplicated { - protected def createVTableReplicated[VD: ClassManifest]( - vTable: VertexSetRDD[VD], - eTable: RDD[(Pid, EdgePartition[ED])] forSome { type ED }, - vertexPlacement: VertexPlacement, - includeSrcAttr: Boolean, - includeDstAttr: Boolean): RDD[(Pid, (VertexIdToIndexMap, Array[VD]))] = { val placement = vertexPlacement.get(includeSrcAttr, includeDstAttr) // Send each edge partition the vertex attributes it wants, as specified in // vertexPlacement val msgsByPartition = placement.zipPartitions(vTable.partitionsRDD) { (pid2vidIter, vertexPartIter) => - val pid2vid = pid2vidIter.next() - val vertexPart = vertexPartIter.next() - - val vmap = new PrimitiveKeyOpenHashMap(vertexPart.index, vertexPart.values) - val output = new Array[(Pid, VertexAttributeBlock[VD])](pid2vid.size) - for (pid <- 0 until pid2vid.size) { - val block = new VertexAttributeBlock(pid2vid(pid), pid2vid(pid).map(vid => vmap(vid))) - output(pid) = (pid, block) - } - output.iterator + val pid2vid = pid2vidIter.next() + val vertexPart = vertexPartIter.next() + + val vmap = new PrimitiveKeyOpenHashMap(vertexPart.index, vertexPart.values) + val output = new Array[(Pid, VertexAttributeBlock[VD])](pid2vid.size) + for (pid <- 0 until pid2vid.size) { + val block = new VertexAttributeBlock(pid2vid(pid), pid2vid(pid).map(vid => vmap(vid))) + output(pid) = (pid, block) + } + output.iterator }.partitionBy(eTable.partitioner.get).cache() // Within each edge partition, create a local map from vid to an index into @@ -81,20 +80,22 @@ object VTableReplicated { // msgsByPartition into the correct locations specified in localVidMap localVidMap.zipPartitions(msgsByPartition) { (mapIter, msgsIter) => - val (pid, vidToIndex) = mapIter.next() - assert(!mapIter.hasNext) - // Populate the vertex array using the vidToIndex map - val vertexArray = new Array[VD](vidToIndex.capacity) - for ((_, block) <- msgsIter) { - for (i <- 0 until block.vids.size) { - val vid = block.vids(i) - val attr = block.attrs(i) - val ind = vidToIndex.getPos(vid) & OpenHashSet.POSITION_MASK - vertexArray(ind) = attr + val (pid, vidToIndex) = mapIter.next() + assert(!mapIter.hasNext) + // Populate the vertex array using the vidToIndex map + val vertexArray = new Array[VD](vidToIndex.capacity) + for ((_, block) <- msgsIter) { + for (i <- 0 until block.vids.size) { + val vid = block.vids(i) + val attr = block.attrs(i) + val ind = vidToIndex.getPos(vid) & OpenHashSet.POSITION_MASK + vertexArray(ind) = attr + } } - } - Iterator((pid, (vidToIndex, vertexArray))) + Iterator((pid, (vidToIndex, vertexArray))) }.cache() } } + +class VertexAttributeBlock[VD: ClassManifest](val vids: Array[Vid], val attrs: Array[VD]) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala index 9de57375e9117..1df845a341748 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala @@ -4,11 +4,17 @@ import org.apache.spark.util.collection.{BitSet, PrimitiveKeyOpenHashMap} import org.apache.spark.graph._ -class VertexPartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) VD: ClassManifest]( +private[graph] +class VertexPartition[@specialized(Long, Int, Double) VD: ClassManifest]( val index: VertexIdToIndexMap, val values: Array[VD], val mask: BitSet) { + // TODO: Encapsulate the internal data structures in this class so callers don't need to + // understand the internal data structures. This can possibly be achieved by implementing + // the aggregate and join functions in this class, and VertexSetRDD can simply call into + // that. + /** * Pass each vertex attribute along with the vertex id through a map * function and retain the original RDD's partitioning and index. @@ -25,42 +31,39 @@ class VertexPartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double def map[VD2: ClassManifest](f: (Vid, VD) => VD2): VertexPartition[VD2] = { // Construct a view of the map transformation val newValues = new Array[VD2](index.capacity) - mask.iterator.foreach { ind => - newValues(ind) = f(index.getValueSafe(ind), values(ind)) + var i = mask.nextSetBit(0) + while (i >= 0) { + newValues(i) = f(index.getValue(i), values(i)) + i = mask.nextSetBit(i + 1) } new VertexPartition[VD2](index, newValues, mask) } /** - * Restrict the vertex set to the set of vertices satisfying the - * given predicate. + * Restrict the vertex set to the set of vertices satisfying the given predicate. * * @param pred the user defined predicate * - * @note The vertex set preserves the original index structure - * which means that the returned RDD can be easily joined with - * the original vertex-set. Furthermore, the filter only - * modifies the bitmap index and so no new values are allocated. + * @note The vertex set preserves the original index structure which means that the returned + * RDD can be easily joined with the original vertex-set. Furthermore, the filter only + * modifies the bitmap index and so no new values are allocated. */ def filter(pred: (Vid, VD) => Boolean): VertexPartition[VD] = { // Allocate the array to store the results into val newMask = new BitSet(index.capacity) - // Iterate over the active bits in the old bitset and - // evaluate the predicate - var ind = mask.nextSetBit(0) - while (ind >= 0) { - val k = index.getValueSafe(ind) - if (pred(k, values(ind))) { - newMask.set(ind) + // Iterate over the active bits in the old mask and evaluate the predicate + var i = mask.nextSetBit(0) + while (i >= 0) { + if (pred(index.getValue(i), values(i))) { + newMask.set(i) } - ind = mask.nextSetBit(ind + 1) + i = mask.nextSetBit(i + 1) } new VertexPartition(index, values, newMask) } /** - * Construct a new VertexPartition whose index contains only the vertices in - * the mask. + * Construct a new VertexPartition whose index contains only the vertices in the mask. */ def reindex(): VertexPartition[VD] = { val hashMap = new PrimitiveKeyOpenHashMap[Vid, VD] @@ -68,8 +71,9 @@ class VertexPartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double for ((k, v) <- this.iterator) { hashMap.setMerge(k, v, arbitraryMerge) } + // TODO: Is this a bug? Why are we using index.getBitSet here? new VertexPartition(hashMap.keySet, hashMap._values, index.getBitSet) } - def iterator = mask.iterator.map(ind => (index.getValueSafe(ind), values(ind))) + def iterator = mask.iterator.map(ind => (index.getValue(ind), values(ind))) } diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPlacement.scala b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPlacement.scala index e8734df2ed58f..5b90fc0cf1059 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPlacement.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPlacement.scala @@ -1,6 +1,5 @@ package org.apache.spark.graph.impl -import scala.collection.JavaConversions._ import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.ArrayBuilder @@ -39,18 +38,26 @@ class VertexPlacement( private def createPid2Vid( includeSrcAttr: Boolean, includeDstAttr: Boolean): RDD[Array[Array[Vid]]] = { - // Determine which vertices each edge partition needs by creating a mapping - // from vid to pid + // Determine which vertices each edge partition needs by creating a mapping from vid to pid. val preAgg = eTable.mapPartitions { iter => - val (pid, edgePartition) = iter.next() + val (pid: Pid, edgePartition: EdgePartition[_]) = iter.next() + val numEdges = edgePartition.size val vSet = new VertexSet - if (includeSrcAttr || includeDstAttr) { - edgePartition.foreach { e => - if (includeSrcAttr) vSet.add(e.srcId) - if (includeDstAttr) vSet.add(e.dstId) + if (includeSrcAttr) { // Add src vertices to the set. + var i = 0 + while (i < numEdges) { + vSet.add(edgePartition.srcIds(i)) + i += 1 } } - vSet.iterator.map { vid => (vid.toLong, pid) } + if (includeDstAttr) { // Add dst vertices to the set. + var i = 0 + while (i < numEdges) { + vSet.add(edgePartition.dstIds(i)) + i += 1 + } + } + vSet.iterator.map { vid => (vid, pid) } } // Aggregate the mappings to determine where each vertex should go val vid2pid = VertexSetRDD[Pid, ArrayBuffer[Pid]](preAgg, vTable.index, diff --git a/graph/src/main/scala/org/apache/spark/graph/package.scala b/graph/src/main/scala/org/apache/spark/graph/package.scala index 7b53e9cce82a3..655ae53bf8bc5 100644 --- a/graph/src/main/scala/org/apache/spark/graph/package.scala +++ b/graph/src/main/scala/org/apache/spark/graph/package.scala @@ -6,10 +6,11 @@ import org.apache.spark.util.collection.OpenHashSet package object graph { type Vid = Long + + // TODO: Consider using Char. type Pid = Int type VertexSet = OpenHashSet[Vid] - type VertexArrayList = it.unimi.dsi.fastutil.longs.LongArrayList // type VertexIdToIndexMap = it.unimi.dsi.fastutil.longs.Long2IntOpenHashMap type VertexIdToIndexMap = OpenHashSet[Vid] @@ -18,11 +19,4 @@ package object graph { * Return the default null-like value for a data type T. */ def nullValue[T] = null.asInstanceOf[T] - - - private[graph] - case class MutableTuple2[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) U, - @specialized(Char, Int, Boolean, Byte, Long, Float, Double) V]( - var _1: U, var _2: V) - } From 137294e2abeacda85bdf9036f5b09ab2a96d9cdf Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Tue, 26 Nov 2013 15:32:47 -0800 Subject: [PATCH 229/531] Test GraphImpl.subgraph and fix bug --- .../org/apache/spark/graph/impl/VertexPartition.scala | 2 +- .../test/scala/org/apache/spark/graph/GraphSuite.scala | 10 ++++++++++ 2 files changed, 11 insertions(+), 1 deletion(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala index 9de57375e9117..c82a290680422 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala @@ -68,7 +68,7 @@ class VertexPartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double for ((k, v) <- this.iterator) { hashMap.setMerge(k, v, arbitraryMerge) } - new VertexPartition(hashMap.keySet, hashMap._values, index.getBitSet) + new VertexPartition(hashMap.keySet, hashMap._values, hashMap.keySet.getBitSet) } def iterator = mask.iterator.map(ind => (index.getValueSafe(ind), values(ind))) diff --git a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala index da7b2bdd997cd..ffb47fa3107cf 100644 --- a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala @@ -132,4 +132,14 @@ class GraphSuite extends FunSuite with LocalSparkContext { } } + test("subgraph") { + withSpark(new SparkContext("local", "test")) { sc => + val n = 10 + val star = Graph(sc.parallelize((1 to n).map(x => (0: Vid, x: Vid))), "defaultValue") + val subgraph = star.subgraph(vpred = (vid, attr) => vid % 2 == 0) + assert(subgraph.vertices.collect().toSet === + (0 to n / 2).map(x => (x * 2, "defaultValue")).toSet) + } + } + } From 9e896be375a5c0270bbdf45a2532e59bcb813efa Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Tue, 26 Nov 2013 15:58:55 -0800 Subject: [PATCH 230/531] Test edge filtering in subgraph (test fails) --- graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala index ffb47fa3107cf..8eab9b2bc6bec 100644 --- a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala @@ -139,6 +139,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { val subgraph = star.subgraph(vpred = (vid, attr) => vid % 2 == 0) assert(subgraph.vertices.collect().toSet === (0 to n / 2).map(x => (x * 2, "defaultValue")).toSet) + assert(subgraph.edges.collect().toSet === (1 to n / 2).map(x => Edge(0, x * 2)).toSet) } } From caba1628612603ebf3e8e71b0d89b81e64a8ef87 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 26 Nov 2013 21:02:39 -0800 Subject: [PATCH 231/531] Added join and aggregateUsingIndex to VertexPartition. --- .../apache/spark/rdd/PairRDDFunctions.scala | 2 +- .../org/apache/spark/graph/VertexSetRDD.scala | 117 +++++++----------- .../apache/spark/graph/impl/GraphImpl.scala | 4 +- .../spark/graph/impl/VertexPartition.scala | 73 ++++++++++- 4 files changed, 117 insertions(+), 79 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 93b78e123267c..53184a6b8aaf9 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -245,7 +245,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) if (getKeyClass().isArray && partitioner.isInstanceOf[HashPartitioner]) { throw new SparkException("Default partitioner cannot partition array keys.") } - new ShuffledRDD[K, V, (K, V)](self, partitioner) + if (self.partitioner == partitioner) self else new ShuffledRDD[K, V, (K, V)](self, partitioner) } /** diff --git a/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala b/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala index a054ab0357a93..ed4584bd1eb36 100644 --- a/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala @@ -24,6 +24,7 @@ import org.apache.spark.storage.StorageLevel import org.apache.spark.util.collection.{BitSet, OpenHashSet, PrimitiveKeyOpenHashMap} import org.apache.spark.graph.impl.VertexPartition +import org.apache.spark.util.ClosureCleaner /** @@ -148,14 +149,13 @@ class VertexSetRDD[@specialized VD: ClassManifest]( * Return a new VertexSetRDD by applying a function to corresponding * VertexPartitions of this VertexSetRDD and another one. */ - def zipVertexPartitions[VD2: ClassManifest, VD3: ClassManifest] + private def zipVertexPartitions[VD2: ClassManifest, VD3: ClassManifest] (other: VertexSetRDD[VD2]) (f: (VertexPartition[VD], VertexPartition[VD2]) => VertexPartition[VD3]): VertexSetRDD[VD3] = { val cleanF = sparkContext.clean(f) val newPartitionsRDD = partitionsRDD.zipPartitions( other.partitionsRDD, preservesPartitioning = true - ) { - (thisIter, otherIter) => + ) { (thisIter, otherIter) => val thisPart = thisIter.next() val otherPart = otherIter.next() Iterator(cleanF(thisPart, otherPart)) @@ -175,7 +175,6 @@ class VertexSetRDD[@specialized VD: ClassManifest]( * the original vertex-set. Furthermore, the filter only * modifies the bitmap index and so no new values are allocated. */ - // TODO: Should we consider making pred taking two arguments, instead of a tuple? override def filter(pred: Tuple2[Vid, VD] => Boolean): VertexSetRDD[VD] = this.mapVertexPartitions(_.filter(Function.untupled(pred))) @@ -214,6 +213,7 @@ class VertexSetRDD[@specialized VD: ClassManifest]( * @return A VertexSetRDD with a value for all vertices. */ def fillMissing(missingValue: VD): VertexSetRDD[VD] = { + // TODO: I think this can be done using a join. this.mapVertexPartitions { part => // Allocate a new values array with missing value as the default val newValues = Array.fill(part.values.size)(missingValue) @@ -244,19 +244,10 @@ class VertexSetRDD[@specialized VD: ClassManifest]( * */ def zipJoin[VD2: ClassManifest, VD3: ClassManifest] - (other: VertexSetRDD[VD2])(f: (Vid, VD, VD2) => VD3): VertexSetRDD[VD3] = { - this.zipVertexPartitions(other) { - (thisPart, otherPart) => - if (thisPart.index != otherPart.index) { - throw new SparkException("can't zip join VertexSetRDDs with different indexes") - } - val newValues = new Array[VD3](thisPart.index.capacity) - val newMask = thisPart.mask & otherPart.mask - newMask.iterator.foreach { ind => - newValues(ind) = - f(thisPart.index.getValueSafe(ind), thisPart.values(ind), otherPart.values(ind)) - } - new VertexPartition(thisPart.index, newValues, newMask) + (other: VertexSetRDD[VD2])(f: (Vid, VD, VD2) => VD3): VertexSetRDD[VD3] = + { + this.zipVertexPartitions(other) { (thisPart, otherPart) => + thisPart.join(otherPart)(f) } } @@ -279,21 +270,12 @@ class VertexSetRDD[@specialized VD: ClassManifest]( * */ def leftZipJoin[VD2: ClassManifest, VD3: ClassManifest] - (other: VertexSetRDD[VD2])(f: (Vid, VD, Option[VD2]) => VD3): VertexSetRDD[VD3] = { - this.zipVertexPartitions(other) { - (thisPart, otherPart) => - if (thisPart.index != otherPart.index) { - throw new SparkException("can't zip join VertexSetRDDs with different indexes") - } - val newValues = new Array[VD3](thisPart.index.capacity) - thisPart.mask.iterator.foreach { ind => - val otherV = if (otherPart.mask.get(ind)) Option(otherPart.values(ind)) else None - newValues(ind) = f( - thisPart.index.getValueSafe(ind), thisPart.values(ind), otherV) - } - new VertexPartition(thisPart.index, newValues, thisPart.mask) + (other: VertexSetRDD[VD2])(f: (Vid, VD, Option[VD2]) => VD3): VertexSetRDD[VD3] = + { + this.zipVertexPartitions(other) { (thisPart, otherPart) => + thisPart.leftJoin(otherPart)(f) } - } // end of leftZipJoin + } /** * Left join this VertexSet with an RDD containing vertex attribute @@ -301,7 +283,9 @@ class VertexSetRDD[@specialized VD: ClassManifest]( * index than the efficient leftZipJoin implementation is used. The * resulting vertex set contains an entry for each vertex in this * set. If the other VertexSet is missing any vertex in this - * VertexSet then a `None` attribute is generated + * VertexSet then a `None` attribute is generated. + * + * If there are duplicates, the vertex is picked at random. * * @tparam VD2 the attribute type of the other VertexSet * @tparam VD3 the attribute type of the resulting VertexSet @@ -309,26 +293,36 @@ class VertexSetRDD[@specialized VD: ClassManifest]( * @param other the other VertexSet with which to join. * @param f the function mapping a vertex id and its attributes in * this and the other vertex set to a new vertex attribute. - * @param merge the function used combine duplicate vertex - * attributes * @return a VertexSetRDD containing all the vertices in this * VertexSet with the attribute emitted by f. - * */ def leftJoin[VD2: ClassManifest, VD3: ClassManifest] - (other: RDD[(Vid, VD2)]) - (f: (Vid, VD, Option[VD2]) => VD3, merge: (VD2, VD2) => VD2 = (a: VD2, b: VD2) => a) - : VertexSetRDD[VD3] = { + (other: RDD[(Vid, VD2)]) + (f: (Vid, VD, Option[VD2]) => VD3) + : VertexSetRDD[VD3] = + { // Test if the other vertex is a VertexSetRDD to choose the optimal join strategy. // If the other set is a VertexSetRDD then we use the much more efficient leftZipJoin other match { case other: VertexSetRDD[VD2] => leftZipJoin(other)(f) case _ => - val indexedOther: VertexSetRDD[VD2] = VertexSetRDD(other, this.index, merge) + val indexedOther: VertexSetRDD[VD2] = VertexSetRDD(other, this.index, (a, b) => a) leftZipJoin(indexedOther)(f) } - } // end of leftJoin + } + + def aggregateUsingIndex[VD2: ClassManifest, VidVDPair <: Product2[Vid, VD2] : ClassManifest]( + messages: RDD[VidVDPair], reduceFunc: (VD2, VD2) => VD2): VertexSetRDD[VD2] = + { + // TODO: use specialized shuffle serializer. + val shuffled = new ShuffledRDD[Vid, VD2, VidVDPair](messages, partitionsRDD.partitioner.get) + val parts = partitionsRDD.zipPartitions(shuffled, true) { (thisIter, msgIter) => + val vertextPartition: VertexPartition[VD] = thisIter.next() + Iterator(vertextPartition.aggregateUsingIndex(msgIter, reduceFunc)) + } + new VertexSetRDD[VD2](parts) + } } // end of VertexSetRDD @@ -346,42 +340,16 @@ object VertexSetRDD { * * @param rdd the collection of vertex-attribute pairs */ - def apply[VD: ClassManifest](rdd: RDD[(Vid, VD)]): VertexSetRDD[VD] = - apply(rdd, (a: VD, b: VD) => a) - - /** - * Construct a vertex set from an RDD of vertex-attribute pairs - * where duplicate entries are merged using the reduceFunc - * - * @tparam VD the vertex attribute type - * - * @param rdd the collection of vertex-attribute pairs - * @param reduceFunc the function used to merge attributes of - * duplicate vertices. - */ - def apply[VD: ClassManifest]( - rdd: RDD[(Vid, VD)], reduceFunc: (VD, VD) => VD): VertexSetRDD[VD] = { - val cReduceFunc = rdd.context.clean(reduceFunc) - // Preaggregate and shuffle if necessary - val preAgg = rdd.partitioner match { + def apply[VD: ClassManifest](rdd: RDD[(Vid, VD)]): VertexSetRDD[VD] = { + val partitioned: RDD[(Vid, VD)] = rdd.partitioner match { case Some(p) => rdd - case None => - val partitioner = new HashPartitioner(rdd.partitions.size) - // Preaggregation. - val aggregator = new Aggregator[Vid, VD, VD](v => v, cReduceFunc, cReduceFunc) - rdd.mapPartitions(aggregator.combineValuesByKey, true).partitionBy(partitioner) + case None => rdd.partitionBy(new HashPartitioner(rdd.partitions.size)) } - - val partitionsRDD = preAgg.mapPartitions(iter => { - val hashMap = new PrimitiveKeyOpenHashMap[Vid, VD] - for ((k, v) <- iter) { - hashMap.setMerge(k, v, cReduceFunc) - } - val part = new VertexPartition(hashMap.keySet, hashMap._values, hashMap.keySet.getBitSet) - Iterator(part) - }, preservesPartitioning = true).cache - new VertexSetRDD(partitionsRDD) - } // end of apply + val vertexPartitions = partitioned.mapPartitions( + iter => Iterator(VertexPartition(iter)), + preservesPartitioning = true) + new VertexSetRDD(vertexPartitions) + } /** * Construct a vertex set from an RDD using an existing index. @@ -394,6 +362,7 @@ object VertexSetRDD { * indexes must be a superset of the vertices in rdd * in RDD */ + // TODO: only used in testing. Consider removing. def apply[VD: ClassManifest](rdd: RDD[(Vid, VD)], index: VertexSetIndex): VertexSetRDD[VD] = apply(rdd, index, (a: VD, b: VD) => a) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index db0b940a4be30..154466d7a66a5 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -256,10 +256,10 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( msgBS.iterator.map { ind => new AggregationMsg[A](vidToIndex.getValue(ind), msgArray(ind)) } - }.partitionBy(vTable.partitioner.get) + } // do the final reduction reusing the index map - VertexSetRDD.aggregate(preAgg, vTable.index, reduceFunc) + vTable.aggregateUsingIndex(preAgg, reduceFunc) } // end of mapReduceTriplets override def outerJoinVertices[U: ClassManifest, VD2: ClassManifest] diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala index 1df845a341748..c6147b89a946f 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala @@ -2,8 +2,22 @@ package org.apache.spark.graph.impl import org.apache.spark.util.collection.{BitSet, PrimitiveKeyOpenHashMap} +import org.apache.spark.SparkException import org.apache.spark.graph._ + +private[graph] object VertexPartition { + + def apply[VD: ClassManifest](iter: Iterator[(Vid, VD)]): VertexPartition[VD] = { + val map = new PrimitiveKeyOpenHashMap[Vid, VD] + iter.foreach { case (k, v) => + map(k) = v + } + new VertexPartition(map.keySet, map._values, map.keySet.getBitSet) + } +} + + private[graph] class VertexPartition[@specialized(Long, Int, Double) VD: ClassManifest]( val index: VertexIdToIndexMap, @@ -15,6 +29,8 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassManifest]( // the aggregate and join functions in this class, and VertexSetRDD can simply call into // that. + val capacity: Int = index.capacity + /** * Pass each vertex attribute along with the vertex id through a map * function and retain the original RDD's partitioning and index. @@ -30,7 +46,7 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassManifest]( */ def map[VD2: ClassManifest](f: (Vid, VD) => VD2): VertexPartition[VD2] = { // Construct a view of the map transformation - val newValues = new Array[VD2](index.capacity) + val newValues = new Array[VD2](capacity) var i = mask.nextSetBit(0) while (i >= 0) { newValues(i) = f(index.getValue(i), values(i)) @@ -50,7 +66,7 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassManifest]( */ def filter(pred: (Vid, VD) => Boolean): VertexPartition[VD] = { // Allocate the array to store the results into - val newMask = new BitSet(index.capacity) + val newMask = new BitSet(capacity) // Iterate over the active bits in the old mask and evaluate the predicate var i = mask.nextSetBit(0) while (i >= 0) { @@ -62,6 +78,59 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassManifest]( new VertexPartition(index, values, newMask) } + /** Inner join another VertexPartition. */ + def join[VD2: ClassManifest, VD3: ClassManifest] + (other: VertexPartition[VD2]) + (f: (Vid, VD, VD2) => VD3): VertexPartition[VD3] = { + if (index != other.index) { + throw new SparkException("can't zip join VertexSetRDDs with different indexes") + } + val newValues = new Array[VD3](capacity) + val newMask = mask & other.mask + + var i = newMask.nextSetBit(0) + while (i >= 0) { + newValues(i) = f(index.getValue(i), values(i), other.values(i)) + i = mask.nextSetBit(i + 1) + } + new VertexPartition(index, newValues, newMask) + } + + /** Left outer join another VertexPartition. */ + def leftJoin[VD2: ClassManifest, VD3: ClassManifest] + (other: VertexPartition[VD2]) + (f: (Vid, VD, Option[VD2]) => VD3): VertexPartition[VD3] = { + if (index != other.index) { + throw new SparkException("can't zip join VertexSetRDDs with different indexes") + } + val newValues = new Array[VD3](capacity) + + var i = mask.nextSetBit(0) + while (i >= 0) { + val otherV: Option[VD2] = if (other.mask.get(i)) Some(other.values(i)) else None + newValues(i) = f(index.getValue(i), values(i), otherV) + i = mask.nextSetBit(i + 1) + } + new VertexPartition(index, newValues, mask) + } + + def aggregateUsingIndex[VD2: ClassManifest]( + iter: Iterator[Product2[Vid, VD2]], reduceFunc: (VD2, VD2) => VD2): VertexPartition[VD2] = + { + val newMask = new BitSet(capacity) + val newValues = new Array[VD2](capacity) + iter.foreach { case (vid, vdata) => + val pos = index.getPos(vid) + if (newMask.get(pos)) { + newValues(pos) = reduceFunc(newValues(pos), vdata) + } else { // otherwise just store the new value + newMask.set(pos) + newValues(pos) = vdata + } + } + new VertexPartition[VD2](index, newValues, newMask) + } + /** * Construct a new VertexPartition whose index contains only the vertices in the mask. */ From 95e83af209b322a54fd405b515602cf74020e2f0 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 27 Nov 2013 00:30:26 -0800 Subject: [PATCH 232/531] More, bigger cleaning for better encapsulation of VertexSetRDD and VertexPartition. This is work in progress as stuff doesn't really run. --- .../collection/PrimitiveKeyOpenHashMap.scala | 2 +- .../org/apache/spark/graph/Analytics.scala | 14 +- .../scala/org/apache/spark/graph/Graph.scala | 112 ++------- .../org/apache/spark/graph/GraphLoader.scala | 8 +- .../org/apache/spark/graph/GraphOps.scala | 1 - .../spark/graph/PartitionStrategy.scala | 8 +- .../org/apache/spark/graph/VertexSetRDD.scala | 221 +++--------------- .../apache/spark/graph/impl/GraphImpl.scala | 62 ++--- .../apache/spark/graph/impl/Serializers.scala | 21 ++ .../spark/graph/impl/VertexPartition.scala | 36 ++- .../spark/graph/impl/VertexPlacement.scala | 29 +-- .../spark/graph/util/GraphGenerators.scala | 4 +- .../apache/spark/graph/AnalyticsSuite.scala | 18 +- .../org/apache/spark/graph/GraphSuite.scala | 30 ++- 14 files changed, 203 insertions(+), 363 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMap.scala b/core/src/main/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMap.scala index 987077dd8afea..ee1b168028b4a 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMap.scala @@ -71,7 +71,7 @@ class PrimitiveKeyOpenHashMap[@specialized(Long, Int) K: ClassManifest, /** Set the value for a key */ - def setMerge(k: K, v: V, mergeF: (V,V) => V) { + def setMerge(k: K, v: V, mergeF: (V, V) => V) { val pos = keySet.addWithoutResize(k) val ind = pos & OpenHashSet.POSITION_MASK if ((pos & OpenHashSet.NONEXISTENCE_MASK) != 0) { // if first add diff --git a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala index 62de470c96043..ea501c74355d1 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala @@ -301,10 +301,10 @@ object Analytics extends Logging { def pickPartitioner(v: String): PartitionStrategy = { v match { - case "RandomVertexCut" => RandomVertexCut() - case "EdgePartition1D" => EdgePartition1D() - case "EdgePartition2D" => EdgePartition2D() - case "CanonicalRandomVertexCut" => CanonicalRandomVertexCut() + case "RandomVertexCut" => RandomVertexCut + case "EdgePartition1D" => EdgePartition1D + case "EdgePartition2D" => EdgePartition2D + case "CanonicalRandomVertexCut" => CanonicalRandomVertexCut case _ => throw new IllegalArgumentException("Invalid Partition Strategy: " + v) } } @@ -324,7 +324,7 @@ object Analytics extends Logging { var outFname = "" var numVPart = 4 var numEPart = 4 - var partitionStrategy: PartitionStrategy = RandomVertexCut() + var partitionStrategy: PartitionStrategy = RandomVertexCut options.foreach{ case ("numIter", v) => numIter = v.toInt @@ -379,7 +379,7 @@ object Analytics extends Logging { var numVPart = 4 var numEPart = 4 var isDynamic = false - var partitionStrategy: PartitionStrategy = RandomVertexCut() + var partitionStrategy: PartitionStrategy = RandomVertexCut options.foreach{ case ("numIter", v) => numIter = v.toInt @@ -413,7 +413,7 @@ object Analytics extends Logging { case "triangles" => { var numVPart = 4 var numEPart = 4 - var partitionStrategy: PartitionStrategy = RandomVertexCut() + var partitionStrategy: PartitionStrategy = RandomVertexCut options.foreach{ case ("numEPart", v) => numEPart = v.toInt diff --git a/graph/src/main/scala/org/apache/spark/graph/Graph.scala b/graph/src/main/scala/org/apache/spark/graph/Graph.scala index 2ebe5cf08300e..425c9edefeb23 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Graph.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Graph.scala @@ -287,23 +287,10 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { /** - * The Graph object contains a collection of routines used to - * construct graphs from RDDs. - * + * The Graph object contains a collection of routines used to construct graphs from RDDs. */ object Graph { - /** - * Construct a graph from a collection of edges encoded as vertex id pairs. - * - * @param rawEdges the RDD containing the set of edges in the graph - * - * @return a graph with edge attributes containing the count of duplicate edges. - */ - def apply[VD: ClassManifest](rawEdges: RDD[(Vid, Vid)], defaultValue: VD): Graph[VD, Int] = { - Graph(rawEdges, defaultValue, false, RandomVertexCut()) - } - /** * Construct a graph from a collection of edges encoded as vertex id * pairs. @@ -316,13 +303,12 @@ object Graph { * @return a graph with edge attributes containing either the count * of duplicate edges or 1 (if `uniqueEdges=false`) and vertex * attributes containing the total degree of each vertex. - * */ - def apply[VD: ClassManifest]( + def fromEdgeTuples[VD: ClassManifest]( rawEdges: RDD[(Vid, Vid)], defaultValue: VD, - uniqueEdges: Boolean, - partitionStrategy: PartitionStrategy): Graph[VD, Int] = { + uniqueEdges: Boolean = false, + partitionStrategy: PartitionStrategy = RandomVertexCut): Graph[VD, Int] = { val edges = rawEdges.map(p => Edge(p._1, p._2, 1)) val graph = GraphImpl(edges, defaultValue, partitionStrategy) if (uniqueEdges) graph.groupEdges((a, b) => a + b) else graph @@ -337,106 +323,42 @@ object Graph { * @return a graph with edge attributes described by `edges` and vertices * given by all vertices in `edges` with value `defaultValue` */ - def apply[VD: ClassManifest, ED: ClassManifest](edges: RDD[Edge[ED]], defaultValue: VD) - : Graph[VD, ED] = { - Graph(edges, defaultValue, RandomVertexCut()) - } - - /** - * Construct a graph from a collection of edges. - * - * @param edges the RDD containing the set of edges in the graph - * @param defaultValue the default vertex attribute to use for each vertex - * - * @return a graph with edge attributes described by `edges` and vertices - * given by all vertices in `edges` with value `defaultValue` - */ - def apply[VD: ClassManifest, ED: ClassManifest]( + def fromEdges[VD: ClassManifest, ED: ClassManifest]( edges: RDD[Edge[ED]], defaultValue: VD, - partitionStrategy: PartitionStrategy): Graph[VD, ED] = { + partitionStrategy: PartitionStrategy = RandomVertexCut): Graph[VD, ED] = { GraphImpl(edges, defaultValue, partitionStrategy) } /** * Construct a graph from a collection attributed vertices and - * edges. - * - * @note Duplicate vertices are removed arbitrarily and missing - * vertices (vertices in the edge collection that are not in the - * vertex collection) are replaced by null vertex attributes. - * - * @tparam VD the vertex attribute type - * @tparam ED the edge attribute type - * @param vertices the "set" of vertices and their attributes - * @param edges the collection of edges in the graph - * - */ - def apply[VD: ClassManifest, ED: ClassManifest]( - vertices: RDD[(Vid,VD)], - edges: RDD[Edge[ED]]): Graph[VD, ED] = { - val defaultAttr: VD = null.asInstanceOf[VD] - Graph(vertices, edges, defaultAttr, (a:VD,b:VD) => a, RandomVertexCut()) - } - - /** - * Construct a graph from a collection attributed vertices and - * edges. Duplicate vertices are combined using the `mergeFunc` and + * edges. Duplicate vertices are picked arbitrarily and * vertices found in the edge collection but not in the input - * vertices are the default attribute `defautVertexAttr`. - * - * @note Duplicate vertices are removed arbitrarily . + * vertices are the default attribute. * * @tparam VD the vertex attribute type * @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` - * - */ - def apply[VD: ClassManifest, ED: ClassManifest]( - vertices: RDD[(Vid,VD)], - edges: RDD[Edge[ED]], - defaultVertexAttr: VD): Graph[VD, ED] = { - Graph(vertices, edges, defaultVertexAttr, (a,b) => a, RandomVertexCut()) - } - - /** - * Construct a graph from a collection attributed vertices and - * edges. Duplicate vertices are combined using the `mergeFunc` and - * vertices found in the edge collection but not in the input - * vertices are the default attribute `defautVertexAttr`. - * - * @tparam VD the vertex attribute type - * @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 mergeFunc the function used to merge duplicate vertices - * in the `vertices` collection. + * vertices that are mentioned in edges but not in vertices * @param partitionStrategy the partition strategy to use when * partitioning the edges. - * */ def apply[VD: ClassManifest, ED: ClassManifest]( - vertices: RDD[(Vid,VD)], + vertices: RDD[(Vid, VD)], edges: RDD[Edge[ED]], - defaultVertexAttr: VD, - mergeFunc: (VD, VD) => VD, - partitionStrategy: PartitionStrategy): Graph[VD, ED] = { - GraphImpl(vertices, edges, defaultVertexAttr, mergeFunc, partitionStrategy) + defaultVertexAttr: VD = null.asInstanceOf[VD], + partitionStrategy: PartitionStrategy = RandomVertexCut): Graph[VD, ED] = { + GraphImpl(vertices, edges, defaultVertexAttr, partitionStrategy) } /** - * The implicit graphToGraphOPs function extracts the GraphOps - * member from a graph. + * The implicit graphToGraphOPs function 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: ClassManifest, ED: ClassManifest](g: Graph[VD, ED]) = g.ops } // end of Graph object diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala b/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala index d97c028faae35..7623b2b596156 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala @@ -26,7 +26,7 @@ object GraphLoader { path: String, edgeParser: Array[String] => ED, minEdgePartitions: Int = 1, - partitionStrategy: PartitionStrategy = RandomVertexCut()): + partitionStrategy: PartitionStrategy = RandomVertexCut): Graph[Int, ED] = { // Parse the edge data table val edges = sc.textFile(path, minEdgePartitions).mapPartitions( iter => @@ -43,7 +43,7 @@ object GraphLoader { Edge(source, target, edata) }) val defaultVertexAttr = 1 - Graph(edges, defaultVertexAttr, partitionStrategy) + Graph.fromEdges(edges, defaultVertexAttr, partitionStrategy) } /** @@ -78,7 +78,7 @@ object GraphLoader { path: String, canonicalOrientation: Boolean = false, minEdgePartitions: Int = 1, - partitionStrategy: PartitionStrategy = RandomVertexCut()): + partitionStrategy: PartitionStrategy = RandomVertexCut): Graph[Int, Int] = { // Parse the edge data table val edges = sc.textFile(path, minEdgePartitions).mapPartitions( iter => @@ -97,7 +97,7 @@ object GraphLoader { } }) val defaultVertexAttr = 1 - Graph(edges, defaultVertexAttr, partitionStrategy) + Graph.fromEdges(edges, defaultVertexAttr, partitionStrategy) } // end of edgeListFile } diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala b/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala index b7e28186c6527..3bfad2131e246 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala @@ -240,7 +240,6 @@ class GraphOps[VD: ClassManifest, ED: ClassManifest](graph: Graph[VD, ED]) { case None => data } } - ClosureCleaner.clean(uf) graph.outerJoinVertices(table)(uf) } diff --git a/graph/src/main/scala/org/apache/spark/graph/PartitionStrategy.scala b/graph/src/main/scala/org/apache/spark/graph/PartitionStrategy.scala index cf65f5065786a..293a9d588afda 100644 --- a/graph/src/main/scala/org/apache/spark/graph/PartitionStrategy.scala +++ b/graph/src/main/scala/org/apache/spark/graph/PartitionStrategy.scala @@ -50,7 +50,7 @@ sealed trait PartitionStrategy extends Serializable { * * */ -case class EdgePartition2D() extends PartitionStrategy { +case object EdgePartition2D extends PartitionStrategy { override def getPartition(src: Vid, dst: Vid, numParts: Pid): Pid = { val ceilSqrtNumParts: Pid = math.ceil(math.sqrt(numParts)).toInt val mixingPrime: Vid = 1125899906842597L @@ -61,7 +61,7 @@ case class EdgePartition2D() extends PartitionStrategy { } -case class EdgePartition1D() extends PartitionStrategy { +case object EdgePartition1D extends PartitionStrategy { override def getPartition(src: Vid, dst: Vid, numParts: Pid): Pid = { val mixingPrime: Vid = 1125899906842597L (math.abs(src) * mixingPrime).toInt % numParts @@ -73,7 +73,7 @@ case class EdgePartition1D() extends PartitionStrategy { * Assign edges to an aribtrary machine corresponding to a * random vertex cut. */ -case class RandomVertexCut() extends PartitionStrategy { +case object RandomVertexCut extends PartitionStrategy { override def getPartition(src: Vid, dst: Vid, numParts: Pid): Pid = { math.abs((src, dst).hashCode()) % numParts } @@ -85,7 +85,7 @@ case class RandomVertexCut() extends PartitionStrategy { * function ensures that edges of opposite direction between the same two vertices * will end up on the same partition. */ -case class CanonicalRandomVertexCut() extends PartitionStrategy { +case object CanonicalRandomVertexCut extends PartitionStrategy { override def getPartition(src: Vid, dst: Vid, numParts: Pid): Pid = { val lower = math.min(src, dst) val higher = math.max(src, dst) diff --git a/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala b/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala index ed4584bd1eb36..c3b15bed671e7 100644 --- a/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala @@ -27,28 +27,6 @@ import org.apache.spark.graph.impl.VertexPartition import org.apache.spark.util.ClosureCleaner -/** - * Maintains the per-partition mapping from vertex id to the corresponding - * location in the per-partition values array. This class is meant to be an - * opaque type. - */ -private[graph] -class VertexSetIndex(private[spark] val rdd: RDD[VertexIdToIndexMap]) { - /** - * The persist function behaves like the standard RDD persist - */ - def persist(newLevel: StorageLevel): VertexSetIndex = { - rdd.persist(newLevel) - this - } - - /** - * Returns the partitioner object of the underlying RDD. This is - * used by the VertexSetRDD to partition the values RDD. - */ - def partitioner: Partitioner = rdd.partitioner.get -} // end of VertexSetIndex - /** * A `VertexSetRDD[VD]` extends the `RDD[(Vid, VD)]` by ensuring that there is * only one entry for each vertex and by pre-indexing the entries for fast, @@ -77,26 +55,20 @@ class VertexSetRDD[@specialized VD: ClassManifest]( @transient val partitionsRDD: RDD[VertexPartition[VD]]) extends RDD[(Vid, VD)](partitionsRDD.context, List(new OneToOneDependency(partitionsRDD))) { - /** - * The `VertexSetIndex` representing the layout of this `VertexSetRDD`. - */ - // TOOD: Consider removing the exposure of index to outside, and implement methods in this - // class to handle any operations that would require indexing. - def index = new VertexSetIndex(partitionsRDD.mapPartitions(_.map(_.index), - preservesPartitioning = true)) + require(partitionsRDD.partitioner.isDefined) /** * Construct a new VertexSetRDD that is indexed by only the keys in the RDD. * The resulting VertexSet will be based on a different index and can * no longer be quickly joined with this RDD. */ - def reindex(): VertexSetRDD[VD] = VertexSetRDD(this) + def reindex(): VertexSetRDD[VD] = new VertexSetRDD(partitionsRDD.map(_.reindex())) /** * An internal representation which joins the block indices with the values * This is used by the compute function to emulate `RDD[(Vid, VD)]` */ - protected[spark] val tuples = partitionsRDD.flatMap(_.iterator) + protected[spark] val tuples: RDD[(Vid, VD)] = partitionsRDD.flatMap(_.iterator) /** * The partitioner is defined by the index. @@ -305,10 +277,18 @@ class VertexSetRDD[@specialized VD: ClassManifest]( // If the other set is a VertexSetRDD then we use the much more efficient leftZipJoin other match { case other: VertexSetRDD[VD2] => + println("!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!") leftZipJoin(other)(f) case _ => - val indexedOther: VertexSetRDD[VD2] = VertexSetRDD(other, this.index, (a, b) => a) - leftZipJoin(indexedOther)(f) + println("------------------------------------------------------") + new VertexSetRDD[VD3]( + partitionsRDD.zipPartitions( + other.partitionBy(this.partitioner.get), preservesPartitioning = true) + { (part, msgs) => + val vertexPartition: VertexPartition[VD] = part.next() + Iterator(vertexPartition.leftJoin(msgs)(f)) + } + ) } } @@ -316,7 +296,7 @@ class VertexSetRDD[@specialized VD: ClassManifest]( messages: RDD[VidVDPair], reduceFunc: (VD2, VD2) => VD2): VertexSetRDD[VD2] = { // TODO: use specialized shuffle serializer. - val shuffled = new ShuffledRDD[Vid, VD2, VidVDPair](messages, partitionsRDD.partitioner.get) + val shuffled = new ShuffledRDD[Vid, VD2, VidVDPair](messages, this.partitioner.get) val parts = partitionsRDD.zipPartitions(shuffled, true) { (thisIter, msgIter) => val vertextPartition: VertexPartition[VD] = thisIter.next() Iterator(vertextPartition.aggregateUsingIndex(msgIter, reduceFunc)) @@ -352,166 +332,31 @@ object VertexSetRDD { } /** - * Construct a vertex set from an RDD using an existing index. - * - * @note duplicate vertices are discarded arbitrarily - * - * @tparam VD the vertex attribute type - * @param rdd the rdd containing vertices - * @param index a VertexSetRDD whose indexes will be reused. The - * indexes must be a superset of the vertices in rdd - * in RDD - */ - // TODO: only used in testing. Consider removing. - def apply[VD: ClassManifest](rdd: RDD[(Vid, VD)], index: VertexSetIndex): VertexSetRDD[VD] = - apply(rdd, index, (a: VD, b: VD) => a) - - /** - * Construct a vertex set from an RDD using an existing index and a - * user defined `combiner` to merge duplicate vertices. + * Construct a vertex set from an RDD of vertex-attribute pairs. + * Duplicate entries are merged using mergeFunc. * * @tparam VD the vertex attribute type - * @param rdd the rdd containing vertices - * @param index a VertexSetRDD whose indexes will be reused. The - * indexes must be a superset of the vertices in rdd - * @param reduceFunc the user defined reduce function used to merge - * duplicate vertex attributes. - */ - def apply[VD: ClassManifest]( - rdd: RDD[(Vid, VD)], - index: VertexSetIndex, - reduceFunc: (VD, VD) => VD): VertexSetRDD[VD] = - // TODO: Considering removing the following apply. - apply(rdd, index, (v: VD) => v, reduceFunc, reduceFunc) - - /** - * Construct a vertex set from an RDD of Product2[Vid, VD] * - * @tparam VD the vertex attribute type - * @param rdd the rdd containing vertices - * @param index a VertexSetRDD whose indexes will be reused. The - * indexes must be a superset of the vertices in rdd - * @param reduceFunc the user defined reduce function used to merge - * duplicate vertex attributes. + * @param rdd the collection of vertex-attribute pairs + * @param mergeFunc the associative, commutative merge function. */ - private[spark] def aggregate[VD: ClassManifest, VidVDPair <: Product2[Vid, VD] : ClassManifest]( - rdd: RDD[VidVDPair], - index: VertexSetIndex, - reduceFunc: (VD, VD) => VD): VertexSetRDD[VD] = { - - val cReduceFunc = rdd.context.clean(reduceFunc) - assert(rdd.partitioner == Some(index.partitioner)) - // Use the index to build the new values table - val partitionsRDD = index.rdd.zipPartitions(rdd, preservesPartitioning = true) { - (indexIter, tblIter) => - // There is only one map - val index = indexIter.next() - val mask = new BitSet(index.capacity) - val values = new Array[VD](index.capacity) - for (vertexPair <- tblIter) { - // Get the location of the key in the index - val pos = index.getPos(vertexPair._1) - if ((pos & OpenHashSet.NONEXISTENCE_MASK) != 0) { - throw new SparkException("Error: Trying to bind an external index " + - "to an RDD which contains keys that are not in the index.") - } else { - // Get the actual index - val ind = pos & OpenHashSet.POSITION_MASK - // If this value has already been seen then merge - if (mask.get(ind)) { - values(ind) = cReduceFunc(values(ind), vertexPair._2) - } else { // otherwise just store the new value - mask.set(ind) - values(ind) = vertexPair._2 - } - } - } - Iterator(new VertexPartition(index, values, mask)) + def apply[VD: ClassManifest](rdd: RDD[(Vid, VD)], mergeFunc: (VD, VD) => VD): VertexSetRDD[VD] = + { + val partitioned: RDD[(Vid, VD)] = rdd.partitioner match { + case Some(p) => rdd + case None => rdd.partitionBy(new HashPartitioner(rdd.partitions.size)) } - - new VertexSetRDD(partitionsRDD) + val vertexPartitions = partitioned.mapPartitions( + iter => Iterator(VertexPartition(iter)), + preservesPartitioning = true) + new VertexSetRDD(vertexPartitions) } - /** - * Construct a vertex set from an RDD using an existing index and a - * user defined `combiner` to merge duplicate vertices. - * - * @tparam VD the vertex attribute type - * @param rdd the rdd containing vertices - * @param index the index which must be a superset of the vertices - * in RDD - * @param createCombiner a user defined function to create a combiner - * from a vertex attribute - * @param mergeValue a user defined function to merge a vertex - * attribute into an existing combiner - * @param mergeCombiners a user defined function to merge combiners - * - */ - def apply[VD: ClassManifest, C: ClassManifest]( - rdd: RDD[(Vid, VD)], - index: VertexSetIndex, - createCombiner: VD => C, - mergeValue: (C, VD) => C, - mergeCombiners: (C, C) => C): VertexSetRDD[C] = { - val cCreateCombiner = rdd.context.clean(createCombiner) - val cMergeValue = rdd.context.clean(mergeValue) - val cMergeCombiners = rdd.context.clean(mergeCombiners) - val partitioner = index.partitioner - // Preaggregate and shuffle if necessary - val partitioned = - if (rdd.partitioner != Some(partitioner)) { - // Preaggregation. - val aggregator = new Aggregator[Vid, VD, C](cCreateCombiner, cMergeValue, cMergeCombiners) - rdd.mapPartitions(aggregator.combineValuesByKey).partitionBy(partitioner) - } else { - rdd.mapValues(x => createCombiner(x)) - } - - aggregate(partitioned, index, mergeCombiners) - } // end of apply - - /** - * Construct an index of the unique vertices. The resulting index - * can be used to build VertexSets over subsets of the vertices in - * the input. - */ - def makeIndex( - keys: RDD[Vid], partitionerOpt: Option[Partitioner] = None): VertexSetIndex = { - val partitioner = partitionerOpt match { - case Some(p) => p - case None => Partitioner.defaultPartitioner(keys) + def apply[VD: ClassManifest](vids: RDD[Vid], rdd: RDD[(Vid, VD)], defaultVal: VD) + : VertexSetRDD[VD] = + { + VertexSetRDD(vids.map(vid => (vid, defaultVal))).leftJoin(rdd) { (vid, default, value) => + value.getOrElse(default) } - - val preAgg: RDD[(Vid, Unit)] = keys.mapPartitions(iter => { - val keys = new VertexIdToIndexMap - while (iter.hasNext) { keys.add(iter.next) } - keys.iterator.map(k => (k, ())) - }, preservesPartitioning = true).partitionBy(partitioner) - - val index = preAgg.mapPartitions(iter => { - val index = new VertexIdToIndexMap - while (iter.hasNext) { index.add(iter.next._1) } - Iterator(index) - }, preservesPartitioning = true).cache - - new VertexSetIndex(index) } - - /** - * Create a VertexSetRDD with all vertices initialized to the default value. - * - * @param index an index over the set of vertices - * @param defaultValue the default value to use when initializing the vertices - * @tparam VD the type of the vertex attribute - * @return - */ - def apply[VD: ClassManifest](index: VertexSetIndex, defaultValue: VD): VertexSetRDD[VD] = { - // Use the index to build the new values tables - val partitionsRDD = index.rdd.mapPartitions(_.map { index => - val values = Array.fill(index.capacity)(defaultValue) - val mask = index.getBitSet - new VertexPartition(index, values, mask) - }, preservesPartitioning = true) - new VertexSetRDD(partitionsRDD) - } // end of apply -} // end of object VertexSetRDD +} diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index 154466d7a66a5..b445c8ad2ba8c 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -6,7 +6,7 @@ import org.apache.spark.graph._ import org.apache.spark.graph.impl.GraphImpl._ import org.apache.spark.graph.impl.MsgRDDFunctions._ import org.apache.spark.graph.util.BytecodeUtils -import org.apache.spark.rdd.RDD +import org.apache.spark.rdd.{ShuffledRDD, RDD} import org.apache.spark.storage.StorageLevel import org.apache.spark.util.ClosureCleaner import org.apache.spark.util.collection.{BitSet, OpenHashSet, PrimitiveKeyOpenHashMap} @@ -252,7 +252,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( } } } - // construct an iterator of tuples Iterator[(Vid, A)] + // construct an iterator of tuples. Iterator[(Vid, A)] msgBS.iterator.map { ind => new AggregationMsg[A](vidToIndex.getValue(ind), msgArray(ind)) } @@ -265,6 +265,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( override def outerJoinVertices[U: ClassManifest, VD2: ClassManifest] (updates: RDD[(Vid, U)])(updateF: (Vid, VD, Option[U]) => VD2): Graph[VD2, ED] = { ClosureCleaner.clean(updateF) + println("type of --------------------------- " + updates) val newVTable = vTable.leftJoin(updates)(updateF) new GraphImpl(newVTable, eTable, vertexPlacement, partitioner) } @@ -276,18 +277,24 @@ object GraphImpl { def apply[VD: ClassManifest, ED: ClassManifest]( edges: RDD[Edge[ED]], defaultValue: VD, - partitionStrategy: PartitionStrategy): GraphImpl[VD, ED] = { - val etable = createETable(edges, partitionStrategy).cache + partitionStrategy: PartitionStrategy): GraphImpl[VD, ED] = + { + val etable = createETable(edges, partitionStrategy).cache() + // Get the set of all vids - val vids = etable.mapPartitions(iter => { - val (pid, epart) = iter.next() + val vids = etable.mapPartitions { iter => + val (_, epart) = iter.next() assert(!iter.hasNext) - epart.iterator.flatMap(e => Iterator(e.srcId, e.dstId)) - }, preservesPartitioning = true) - // Index the set of all vids - val index = VertexSetRDD.makeIndex(vids) - // Index the vertices and fill in missing attributes with the default - val vtable = VertexSetRDD(index, defaultValue) + epart.iterator.flatMap(e => Iterator((e.srcId, 0), (e.dstId, 0))) + } + + // Shuffle the vids and create the VertexSetRDD. + // TODO: Consider doing map side distinct before shuffle. + val shuffled = new ShuffledRDD[Vid, Int, (Vid, Int)]( + vids, new HashPartitioner(edges.partitions.size)) + shuffled.setSerializer(classOf[VidMsgSerializer].getName) + val vtable = VertexSetRDD(shuffled.mapValues(x => defaultValue)) + val vertexPlacement = new VertexPlacement(etable, vtable) new GraphImpl(vtable, etable, vertexPlacement, partitionStrategy) } @@ -296,23 +303,24 @@ object GraphImpl { vertices: RDD[(Vid, VD)], edges: RDD[Edge[ED]], defaultVertexAttr: VD, - mergeFunc: (VD, VD) => VD, - partitionStrategy: PartitionStrategy): GraphImpl[VD, ED] = { - - vertices.cache - val etable = createETable(edges, partitionStrategy).cache - // Get the set of all vids, preserving partitions + partitionStrategy: PartitionStrategy): GraphImpl[VD, ED] = + { + vertices.cache() + val etable = createETable(edges, partitionStrategy).cache() + // Get the set of all vids val partitioner = Partitioner.defaultPartitioner(vertices) - val implicitVids = etable.flatMap { - case (pid, partition) => Array.concat(partition.srcIds, partition.dstIds) - }.map(vid => (vid, ())).partitionBy(partitioner) - val allVids = vertices.zipPartitions(implicitVids, preservesPartitioning = true) { - (a, b) => a.map(_._1) ++ b.map(_._1) + + val vPartitioned = vertices.partitionBy(partitioner) + + val vidsFromEdges = etable.flatMap { case (_, p) => Array.concat(p.srcIds, p.dstIds) } + .map(vid => (vid, 0)) + .partitionBy(partitioner) + + val vids = vPartitioned.zipPartitions(vidsFromEdges) { (vertexIter, vidsFromEdgesIter) => + vertexIter.map(_._1) ++ vidsFromEdgesIter.map(_._1) } - // Index the set of all vids - val index = VertexSetRDD.makeIndex(allVids, Some(partitioner)) - // Index the vertices and fill in missing attributes with the default - val vtable = VertexSetRDD(vertices, index, mergeFunc).fillMissing(defaultVertexAttr) + + val vtable = VertexSetRDD(vids, vPartitioned, defaultVertexAttr) val vertexPlacement = new VertexPlacement(etable, vtable) new GraphImpl(vtable, etable, vertexPlacement, partitionStrategy) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala b/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala index 2e768e85cfb47..9143820e13b4f 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala @@ -3,9 +3,30 @@ package org.apache.spark.graph.impl import java.io.{EOFException, InputStream, OutputStream} import java.nio.ByteBuffer +import org.apache.spark.graph._ import org.apache.spark.serializer._ +class VidMsgSerializer extends Serializer { + override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { + + override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { + def writeObject[T](t: T) = { + val msg = t.asInstanceOf[(Vid, _)] + writeVarLong(msg._1, optimizePositive = false) + this + } + } + + override def deserializeStream(s: InputStream) = new ShuffleDeserializationStream(s) { + override def readObject[T](): T = { + (readVarLong(optimizePositive = false), null).asInstanceOf[T] + } + } + } +} + + /** A special shuffle serializer for VertexBroadcastMessage[Int]. */ class IntVertexBroadcastMsgSerializer extends Serializer { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala index c6147b89a946f..7007427e0dd9f 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala @@ -15,6 +15,16 @@ private[graph] object VertexPartition { } new VertexPartition(map.keySet, map._values, map.keySet.getBitSet) } + + def apply[VD: ClassManifest](iter: Iterator[(Vid, VD)], mergeFunc: (VD, VD) => VD) + : VertexPartition[VD] = + { + val map = new PrimitiveKeyOpenHashMap[Vid, VD] + iter.foreach { case (k, v) => + map.setMerge(k, v, mergeFunc) + } + new VertexPartition(map.keySet, map._values, map.keySet.getBitSet) + } } @@ -114,12 +124,36 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassManifest]( new VertexPartition(index, newValues, mask) } + /** Left outer join another iterator of messages. */ + def leftJoin[VD2: ClassManifest, VD3: ClassManifest] + (other: Iterator[(Vid, VD2)]) + (f: (Vid, VD, Option[VD2]) => VD3): VertexPartition[VD3] = { + leftJoin(createUsingIndex(other))(f) + } + + /** + * Similar effect as aggregateUsingIndex((a, b) => a) + */ + def createUsingIndex[VD2: ClassManifest](iter: Iterator[Product2[Vid, VD2]]) + : VertexPartition[VD2] = { + val newMask = new BitSet(capacity) + val newValues = new Array[VD2](capacity) + iter.foreach { case (vid, vdata) => + val pos = index.getPos(vid) + newMask.set(pos) + newValues(pos) = vdata + } + new VertexPartition[VD2](index, newValues, newMask) + } + def aggregateUsingIndex[VD2: ClassManifest]( iter: Iterator[Product2[Vid, VD2]], reduceFunc: (VD2, VD2) => VD2): VertexPartition[VD2] = { val newMask = new BitSet(capacity) val newValues = new Array[VD2](capacity) - iter.foreach { case (vid, vdata) => + iter.foreach { product => + val vid = product._1 + val vdata = product._2 val pos = index.getPos(vid) if (newMask.get(pos)) { newValues(pos) = reduceFunc(newValues(pos), vdata) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPlacement.scala b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPlacement.scala index 5b90fc0cf1059..f47b2cb58785e 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPlacement.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPlacement.scala @@ -1,12 +1,11 @@ package org.apache.spark.graph.impl -import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.ArrayBuilder - +import org.apache.spark.SparkContext._ import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel import org.apache.spark.graph._ +import org.apache.spark.util.collection.PrimitiveVector /** * Stores the layout of replicated vertex attributes for GraphImpl. Tells each @@ -39,7 +38,7 @@ class VertexPlacement( private def createPid2Vid( includeSrcAttr: Boolean, includeDstAttr: Boolean): RDD[Array[Array[Vid]]] = { // Determine which vertices each edge partition needs by creating a mapping from vid to pid. - val preAgg = eTable.mapPartitions { iter => + val vid2pid: RDD[(Vid, Pid)] = eTable.mapPartitions { iter => val (pid: Pid, edgePartition: EdgePartition[_]) = iter.next() val numEdges = edgePartition.size val vSet = new VertexSet @@ -59,21 +58,15 @@ class VertexPlacement( } vSet.iterator.map { vid => (vid, pid) } } - // Aggregate the mappings to determine where each vertex should go - val vid2pid = VertexSetRDD[Pid, ArrayBuffer[Pid]](preAgg, vTable.index, - (p: Pid) => ArrayBuffer(p), - (ab: ArrayBuffer[Pid], p:Pid) => {ab.append(p); ab}, - (a: ArrayBuffer[Pid], b: ArrayBuffer[Pid]) => a ++ b) - .mapValues(a => a.toArray) - // Within each vertex partition, reorganize the placement information into - // columnar format keyed on the destination partition - val numPartitions = vid2pid.partitions.size - vid2pid.mapPartitions { iter => - val pid2vid = Array.fill[ArrayBuilder[Vid]](numPartitions)(ArrayBuilder.make[Vid]) - for ((vid, pids) <- iter) { - pids.foreach { pid => pid2vid(pid) += vid } + + val numPartitions = vTable.partitions.size + vid2pid.partitionBy(vTable.partitioner.get).mapPartitions { iter => + val pid2vid = Array.fill(numPartitions)(new PrimitiveVector[Vid]) + for ((vid, pid) <- iter) { + pid2vid(pid) += vid } - Iterator(pid2vid.map(_.result)) + + Iterator(pid2vid.map(_.trim().array)) } } } diff --git a/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala b/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala index 4c17bab0c47ab..a1e285816b9b7 100644 --- a/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala +++ b/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala @@ -268,14 +268,14 @@ object GraphGenerators { * Create a star graph with vertex 0 being the center. * * @param sc the spark context in which to construct the graph - * @param the number of vertices in the star + * @param nverts the number of vertices in the star * * @return A star graph containing `nverts` vertices with vertex 0 * being the center vertex. */ def starGraph(sc: SparkContext, nverts: Int): Graph[Int, Int] = { val edges: RDD[(Vid, Vid)] = sc.parallelize(1 until nverts).map(vid => (vid, 0)) - Graph(edges, 1) + Graph.fromEdgeTuples(edges, 1) } // end of starGraph diff --git a/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala b/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala index e1ff8df4eaae0..d098c17c74fbf 100644 --- a/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala @@ -132,9 +132,9 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { val chain1 = (0 until 9).map(x => (x, x+1) ) val chain2 = (10 until 20).map(x => (x, x+1) ) val rawEdges = sc.parallelize(chain1 ++ chain2, 3).map { case (s,d) => (s.toLong, d.toLong) } - val twoChains = Graph(rawEdges, 1.0) + val twoChains = Graph.fromEdgeTuples(rawEdges, 1.0) val ccGraph = Analytics.connectedComponents(twoChains).cache() - val vertices = ccGraph.vertices.collect + val vertices = ccGraph.vertices.collect() for ( (id, cc) <- vertices ) { if(id < 10) { assert(cc === 0) } else { assert(cc === 10) } @@ -156,7 +156,7 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { val chain1 = (0 until 9).map(x => (x, x+1) ) val chain2 = (10 until 20).map(x => (x, x+1) ) val rawEdges = sc.parallelize(chain1 ++ chain2, 3).map { case (s,d) => (s.toLong, d.toLong) } - val twoChains = Graph(rawEdges, true).reverse + val twoChains = Graph.fromEdgeTuples(rawEdges, true).reverse val ccGraph = Analytics.connectedComponents(twoChains).cache() val vertices = ccGraph.vertices.collect for ( (id, cc) <- vertices ) { @@ -181,7 +181,7 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { test("Count a single triangle") { withSpark(new SparkContext("local", "test")) { sc => val rawEdges = sc.parallelize(Array( 0L->1L, 1L->2L, 2L->0L ), 2) - val graph = Graph(rawEdges, true).cache + val graph = Graph.fromEdgeTuples(rawEdges, true).cache() val triangleCount = Analytics.triangleCount(graph) val verts = triangleCount.vertices verts.collect.foreach { case (vid, count) => assert(count === 1) } @@ -193,10 +193,10 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { val triangles = Array(0L -> 1L, 1L -> 2L, 2L -> 0L) ++ Array(0L -> -1L, -1L -> -2L, -2L -> 0L) val rawEdges = sc.parallelize(triangles, 2) - val graph = Graph(rawEdges, true).cache + val graph = Graph.fromEdgeTuples(rawEdges, true).cache() val triangleCount = Analytics.triangleCount(graph) val verts = triangleCount.vertices - verts.collect.foreach { case (vid, count) => + verts.collect().foreach { case (vid, count) => if (vid == 0) { assert(count === 2) } else { @@ -213,10 +213,10 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { Array(0L -> -1L, -1L -> -2L, -2L -> 0L) val revTriangles = triangles.map { case (a,b) => (b,a) } val rawEdges = sc.parallelize(triangles ++ revTriangles, 2) - val graph = Graph(rawEdges, true).cache + val graph = Graph.fromEdgeTuples(rawEdges, true).cache() val triangleCount = Analytics.triangleCount(graph) val verts = triangleCount.vertices - verts.collect.foreach { case (vid, count) => + verts.collect().foreach { case (vid, count) => if (vid == 0) { assert(count === 4) } else { @@ -230,7 +230,7 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { withSpark(new SparkContext("local", "test")) { sc => val rawEdges = sc.parallelize(Array(0L -> 1L, 1L -> 2L, 2L -> 0L) ++ Array(0L -> 1L, 1L -> 2L, 2L -> 0L), 2) - val graph = Graph(rawEdges, true).cache + val graph = Graph.fromEdgeTuples(rawEdges, true).cache() val triangleCount = Analytics.triangleCount(graph) val verts = triangleCount.vertices verts.collect.foreach { case (vid, count) => assert(count === 1) } diff --git a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala index da7b2bdd997cd..8f4f926d718f2 100644 --- a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala @@ -15,11 +15,20 @@ class GraphSuite extends FunSuite with LocalSparkContext { withSpark(new SparkContext("local", "test")) { sc => val rawEdges = (0L to 100L).zip((1L to 99L) :+ 0L) val edges = sc.parallelize(rawEdges) - val graph = Graph(edges, 1.0F) + val graph = Graph.fromEdgeTuples(edges, 1.0F) assert(graph.edges.count() === rawEdges.size) } } + test("mapReduceTriplets") { + withSpark(new SparkContext("local", "test")) { sc => + val edges = sc.parallelize((0L to 100L).zip((1L to 99L) :+ 0L)) + val graph = Graph.fromEdgeTuples(edges, 1.0F) + + val d = graph.mapReduceTriplets[Int](et => Iterator((et.srcId, 0)), (a, b) => a + b) + } + } + test("Graph Creation with invalid vertices") { withSpark(new SparkContext("local", "test")) { sc => val rawEdges = (0L to 98L).zip((1L to 99L) :+ 0L) @@ -38,7 +47,9 @@ class GraphSuite extends FunSuite with LocalSparkContext { test("mapEdges") { withSpark(new SparkContext("local", "test")) { sc => val n = 3 - val star = Graph(sc.parallelize((1 to n).map(x => (0: Vid, x: Vid))), "defaultValue") + val star = Graph.fromEdgeTuples( + sc.parallelize((1 to n).map(x => (0: Vid, x: Vid))), + "defaultValue") val starWithEdgeAttrs = star.mapEdges(e => e.dstId) // map(_.copy()) is a workaround for https://github.com/amplab/graphx/issues/25 @@ -51,7 +62,14 @@ class GraphSuite extends FunSuite with LocalSparkContext { test("mapReduceTriplets") { withSpark(new SparkContext("local", "test")) { sc => val n = 3 - val star = Graph(sc.parallelize((1 to n).map(x => (0: Vid, x: Vid))), 0) + val star = Graph.fromEdgeTuples(sc.parallelize((1 to n).map(x => (0: Vid, x: Vid))), 0) + + println("--------------------------------------- star vertices") + println(star.vertices.partitionsRDD.map { v => v.index.toString }.collect().toSeq) + + println("--------------------------------------- starDeg") + println(star.degrees.partitionsRDD.map { v => v.index.toString }.collect().toSeq) + val starDeg = star.joinVertices(star.degrees){ (vid, oldV, deg) => deg } val neighborDegreeSums = starDeg.mapReduceTriplets( edge => Iterator((edge.srcId, edge.dstAttr), (edge.dstId, edge.srcAttr)), @@ -63,7 +81,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { test("aggregateNeighbors") { withSpark(new SparkContext("local", "test")) { sc => val n = 3 - val star = Graph(sc.parallelize((1 to n).map(x => (0: Vid, x: Vid))), 1) + val star = Graph.fromEdgeTuples(sc.parallelize((1 to n).map(x => (0: Vid, x: Vid))), 1) val indegrees = star.aggregateNeighbors( (vid, edge) => Some(1), @@ -103,7 +121,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { withSpark(new SparkContext("local", "test")) { sc => 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(rawEdges, 1.0) + val graph = Graph.fromEdgeTuples(rawEdges, 1.0) val nbrs = graph.collectNeighborIds(EdgeDirection.Both) assert(nbrs.count === chain.size) assert(graph.numVertices === nbrs.count) @@ -122,7 +140,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { val b = VertexSetRDD(a).mapValues(x => -x) assert(b.count === 101) assert(b.leftJoin(a){ (id, a, bOpt) => a + bOpt.get }.map(x=> x._2).reduce(_+_) === 0) - val c = VertexSetRDD(a, b.index) + val c = b.aggregateUsingIndex[Long, (Long, Long)](a, (x, y) => x) assert(b.leftJoin(c){ (id, b, cOpt) => b + cOpt.get }.map(x=> x._2).reduce(_+_) === 0) val d = c.filter(q => ((q._2 % 2) == 0)) val e = a.filter(q => ((q._2 % 2) == 0)) From 10c0f9b0bb80a2ced2cd5e0c6a3561937f9ec9db Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 30 Nov 2013 13:18:43 -0800 Subject: [PATCH 233/531] Added a log4j properties file for graphx unit tests. --- graph/src/test/resources/log4j.properties | 28 +++++++++++++++++++++++ 1 file changed, 28 insertions(+) create mode 100644 graph/src/test/resources/log4j.properties diff --git a/graph/src/test/resources/log4j.properties b/graph/src/test/resources/log4j.properties new file mode 100644 index 0000000000000..896936d8c4f29 --- /dev/null +++ b/graph/src/test/resources/log4j.properties @@ -0,0 +1,28 @@ +# +# 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. +# + +# Set everything to be logged to the file core/target/unit-tests.log +log4j.rootCategory=INFO, file +log4j.appender.file=org.apache.log4j.FileAppender +log4j.appender.file.append=false +log4j.appender.file.file=graph/target/unit-tests.log +log4j.appender.file.layout=org.apache.log4j.PatternLayout +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n + +# Ignore messages below warning level from Jetty, because it's a bit verbose +log4j.logger.org.eclipse.jetty=WARN +org.eclipse.jetty.LEVEL=WARN From dee1318d3d4e37489dc2b0bb373857cc13b4d533 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Sat, 30 Nov 2013 13:37:19 -0800 Subject: [PATCH 234/531] Fix join error by caching vTable in mapReduceTriplets --- .../src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index b445c8ad2ba8c..c68f8c2130edb 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -258,6 +258,9 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( } } + // Permit joining the result of mapReduceTriplets back with vTable + vTable.cache() + // do the final reduction reusing the index map vTable.aggregateUsingIndex(preAgg, reduceFunc) } // end of mapReduceTriplets From eed31950380f96f40b7cc70a7ebcf0f2c8d5ffbb Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Sat, 30 Nov 2013 13:50:37 -0800 Subject: [PATCH 235/531] Fix VertexSetRDD test by enabling index reuse --- graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala index 8f4f926d718f2..e095eedce62a0 100644 --- a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala @@ -137,7 +137,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { test("VertexSetRDD") { withSpark(new SparkContext("local", "test")) { sc => val a = sc.parallelize((0 to 100).map(x => (x.toLong, x.toLong)), 5) - val b = VertexSetRDD(a).mapValues(x => -x) + val b = VertexSetRDD(a).mapValues(x => -x).cache() // Allow joining b with a derived RDD of b assert(b.count === 101) assert(b.leftJoin(a){ (id, a, bOpt) => a + bOpt.get }.map(x=> x._2).reduce(_+_) === 0) val c = b.aggregateUsingIndex[Long, (Long, Long)](a, (x, y) => x) From 7528e6d5f15fd0e01a206f60a6db218858cac4d3 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 30 Nov 2013 14:04:16 -0800 Subject: [PATCH 236/531] Enable joining arbitrary VertexPartitions (with different indexes). --- .../org/apache/spark/graph/VertexSetRDD.scala | 2 - .../spark/graph/impl/VertexPartition.scala | 51 ++++++++++--------- 2 files changed, 28 insertions(+), 25 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala b/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala index c3b15bed671e7..cf1fa00ed9fcb 100644 --- a/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala @@ -277,10 +277,8 @@ class VertexSetRDD[@specialized VD: ClassManifest]( // If the other set is a VertexSetRDD then we use the much more efficient leftZipJoin other match { case other: VertexSetRDD[VD2] => - println("!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!") leftZipJoin(other)(f) case _ => - println("------------------------------------------------------") new VertexSetRDD[VD3]( partitionsRDD.zipPartitions( other.partitionBy(this.partitioner.get), preservesPartitioning = true) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala index 7007427e0dd9f..f5047e7b9e851 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala @@ -2,7 +2,7 @@ package org.apache.spark.graph.impl import org.apache.spark.util.collection.{BitSet, PrimitiveKeyOpenHashMap} -import org.apache.spark.SparkException +import org.apache.spark.{Logging, SparkException} import org.apache.spark.graph._ @@ -32,7 +32,8 @@ private[graph] class VertexPartition[@specialized(Long, Int, Double) VD: ClassManifest]( val index: VertexIdToIndexMap, val values: Array[VD], - val mask: BitSet) { + val mask: BitSet) + extends Logging { // TODO: Encapsulate the internal data structures in this class so callers don't need to // understand the internal data structures. This can possibly be achieved by implementing @@ -93,17 +94,19 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassManifest]( (other: VertexPartition[VD2]) (f: (Vid, VD, VD2) => VD3): VertexPartition[VD3] = { if (index != other.index) { - throw new SparkException("can't zip join VertexSetRDDs with different indexes") - } - val newValues = new Array[VD3](capacity) - val newMask = mask & other.mask - - var i = newMask.nextSetBit(0) - while (i >= 0) { - newValues(i) = f(index.getValue(i), values(i), other.values(i)) - i = mask.nextSetBit(i + 1) + logWarning("Joining two VertexPartitions with different indexes is slow.") + join(createUsingIndex(other.iterator))(f) + } else { + val newValues = new Array[VD3](capacity) + val newMask = mask & other.mask + + var i = newMask.nextSetBit(0) + while (i >= 0) { + newValues(i) = f(index.getValue(i), values(i), other.values(i)) + i = mask.nextSetBit(i + 1) + } + new VertexPartition(index, newValues, newMask) } - new VertexPartition(index, newValues, newMask) } /** Left outer join another VertexPartition. */ @@ -111,17 +114,19 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassManifest]( (other: VertexPartition[VD2]) (f: (Vid, VD, Option[VD2]) => VD3): VertexPartition[VD3] = { if (index != other.index) { - throw new SparkException("can't zip join VertexSetRDDs with different indexes") - } - val newValues = new Array[VD3](capacity) - - var i = mask.nextSetBit(0) - while (i >= 0) { - val otherV: Option[VD2] = if (other.mask.get(i)) Some(other.values(i)) else None - newValues(i) = f(index.getValue(i), values(i), otherV) - i = mask.nextSetBit(i + 1) + logWarning("Joining two VertexPartitions with different indexes is slow.") + leftJoin(createUsingIndex(other.iterator))(f) + } else { + val newValues = new Array[VD3](capacity) + + var i = mask.nextSetBit(0) + while (i >= 0) { + val otherV: Option[VD2] = if (other.mask.get(i)) Some(other.values(i)) else None + newValues(i) = f(index.getValue(i), values(i), otherV) + i = mask.nextSetBit(i + 1) + } + new VertexPartition(index, newValues, mask) } - new VertexPartition(index, newValues, mask) } /** Left outer join another iterator of messages. */ @@ -178,5 +183,5 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassManifest]( new VertexPartition(hashMap.keySet, hashMap._values, index.getBitSet) } - def iterator = mask.iterator.map(ind => (index.getValue(ind), values(ind))) + def iterator: Iterator[(Vid, VD)] = mask.iterator.map(ind => (index.getValue(ind), values(ind))) } From 3292cb0f9c1b0b6efbf4a7596dfddfc1a54c004f Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Sat, 30 Nov 2013 14:05:32 -0800 Subject: [PATCH 237/531] Revert "Fix join error by caching vTable in mapReduceTriplets" This reverts commit dee1318d3d4e37489dc2b0bb373857cc13b4d533, which is unnecessary due to 7528e6d5f15fd0e01a206f60a6db218858cac4d3. --- .../src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala | 3 --- 1 file changed, 3 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index c68f8c2130edb..b445c8ad2ba8c 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -258,9 +258,6 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( } } - // Permit joining the result of mapReduceTriplets back with vTable - vTable.cache() - // do the final reduction reusing the index map vTable.aggregateUsingIndex(preAgg, reduceFunc) } // end of mapReduceTriplets From 4d3d68b8fb9cb09c32a4fa24a025dcc0761d548b Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 30 Nov 2013 14:15:47 -0800 Subject: [PATCH 238/531] Minor update to tests. --- .../scala/org/apache/spark/graph/VertexSetRDD.scala | 2 +- .../org/apache/spark/graph/impl/GraphImpl.scala | 1 - .../org/apache/spark/graph/AnalyticsSuite.scala | 12 ++++++------ 3 files changed, 7 insertions(+), 8 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala b/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala index cf1fa00ed9fcb..328dafa632108 100644 --- a/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala @@ -276,7 +276,7 @@ class VertexSetRDD[@specialized VD: ClassManifest]( // Test if the other vertex is a VertexSetRDD to choose the optimal join strategy. // If the other set is a VertexSetRDD then we use the much more efficient leftZipJoin other match { - case other: VertexSetRDD[VD2] => + case other: VertexSetRDD[_] => leftZipJoin(other)(f) case _ => new VertexSetRDD[VD3]( diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index c68f8c2130edb..b257b9e603e50 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -268,7 +268,6 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( override def outerJoinVertices[U: ClassManifest, VD2: ClassManifest] (updates: RDD[(Vid, U)])(updateF: (Vid, VD, Option[U]) => VD2): Graph[VD2, ED] = { ClosureCleaner.clean(updateF) - println("type of --------------------------- " + updates) val newVTable = vTable.leftJoin(updates)(updateF) new GraphImpl(newVTable, eTable, vertexPlacement, partitioner) } diff --git a/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala b/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala index d098c17c74fbf..b74835216aa4d 100644 --- a/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala @@ -54,7 +54,7 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { test("Star PageRank") { withSpark(new SparkContext("local", "test")) { sc => val nVertices = 100 - val starGraph = GraphGenerators.starGraph(sc, nVertices) + val starGraph = GraphGenerators.starGraph(sc, nVertices).cache() val resetProb = 0.15 val prGraph1 = Analytics.pagerank(starGraph, 1, resetProb) val prGraph2 = Analytics.pagerank(starGraph, 2, resetProb) @@ -86,7 +86,7 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { test("Grid PageRank") { withSpark(new SparkContext("local", "test")) { sc => - val gridGraph = GraphGenerators.gridGraph(sc, 10, 10) + val gridGraph = GraphGenerators.gridGraph(sc, 10, 10).cache() val resetProb = 0.15 val prGraph1 = Analytics.pagerank(gridGraph, 50, resetProb).cache() val prGraph2 = Analytics.deltaPagerank(gridGraph, 0.0001, resetProb).cache() @@ -109,7 +109,7 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { test("Grid Connected Components") { withSpark(new SparkContext("local", "test")) { sc => - val gridGraph = GraphGenerators.gridGraph(sc, 10, 10) + val gridGraph = GraphGenerators.gridGraph(sc, 10, 10).cache() val ccGraph = Analytics.connectedComponents(gridGraph).cache() val maxCCid = ccGraph.vertices.map { case (vid, ccId) => ccId }.sum assert(maxCCid === 0) @@ -119,7 +119,7 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { test("Reverse Grid Connected Components") { withSpark(new SparkContext("local", "test")) { sc => - val gridGraph = GraphGenerators.gridGraph(sc, 10, 10).reverse + val gridGraph = GraphGenerators.gridGraph(sc, 10, 10).reverse.cache() val ccGraph = Analytics.connectedComponents(gridGraph).cache() val maxCCid = ccGraph.vertices.map { case (vid, ccId) => ccId }.sum assert(maxCCid === 0) @@ -132,7 +132,7 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { val chain1 = (0 until 9).map(x => (x, x+1) ) val chain2 = (10 until 20).map(x => (x, x+1) ) val rawEdges = sc.parallelize(chain1 ++ chain2, 3).map { case (s,d) => (s.toLong, d.toLong) } - val twoChains = Graph.fromEdgeTuples(rawEdges, 1.0) + val twoChains = Graph.fromEdgeTuples(rawEdges, 1.0).cache() val ccGraph = Analytics.connectedComponents(twoChains).cache() val vertices = ccGraph.vertices.collect() for ( (id, cc) <- vertices ) { @@ -156,7 +156,7 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { val chain1 = (0 until 9).map(x => (x, x+1) ) val chain2 = (10 until 20).map(x => (x, x+1) ) val rawEdges = sc.parallelize(chain1 ++ chain2, 3).map { case (s,d) => (s.toLong, d.toLong) } - val twoChains = Graph.fromEdgeTuples(rawEdges, true).reverse + val twoChains = Graph.fromEdgeTuples(rawEdges, true).reverse.cache() val ccGraph = Analytics.connectedComponents(twoChains).cache() val vertices = ccGraph.vertices.collect for ( (id, cc) <- vertices ) { From b30e0ae0351be1cbc0b1cf179293587b466ee026 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 30 Nov 2013 14:24:18 -0800 Subject: [PATCH 239/531] Added an optimized count to VertexSetRDD. --- graph/src/main/scala/org/apache/spark/graph/Pregel.scala | 2 -- .../src/main/scala/org/apache/spark/graph/VertexSetRDD.scala | 5 +++++ .../scala/org/apache/spark/graph/impl/VertexPartition.scala | 2 ++ 3 files changed, 7 insertions(+), 2 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala index 29d6225f33838..55b3464b56989 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala @@ -139,8 +139,6 @@ object Pregel { * @param initialMsg the message each vertex will receive at the on * the first iteration. * - * @param numIter the number of iterations to run this computation. - * * @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 diff --git a/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala b/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala index 328dafa632108..4c8128f3e0f1d 100644 --- a/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala @@ -101,6 +101,11 @@ class VertexSetRDD[@specialized VD: ClassManifest]( /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */ override def cache(): VertexSetRDD[VD] = persist() + /** Return the number of vertices in this set. */ + override def count(): Long = { + partitionsRDD.map(_.size).reduce(_ + _) + } + /** * Provide the `RDD[(Vid, VD)]` equivalent output. */ diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala index f5047e7b9e851..3d80ab1bb9313 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala @@ -42,6 +42,8 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassManifest]( val capacity: Int = index.capacity + def size: Int = mask.cardinality + /** * Pass each vertex attribute along with the vertex id through a map * function and retain the original RDD's partitioning and index. From 229022891fead3a8addbd9f1fd10b250baa79676 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 30 Nov 2013 14:45:56 -0800 Subject: [PATCH 240/531] Made all VertexPartition internal data structures private. --- .../org/apache/spark/graph/VertexSetRDD.scala | 21 ------------------- .../spark/graph/impl/VTableReplicated.scala | 10 ++++----- .../spark/graph/impl/VertexPartition.scala | 11 ++++++---- .../org/apache/spark/graph/GraphSuite.scala | 16 -------------- 4 files changed, 12 insertions(+), 46 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala b/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala index 4c8128f3e0f1d..0206dc9b376a8 100644 --- a/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala @@ -183,27 +183,6 @@ class VertexSetRDD[@specialized VD: ClassManifest]( def mapValues[VD2: ClassManifest](f: (Vid, VD) => VD2): VertexSetRDD[VD2] = this.mapVertexPartitions(_.map(f)) - /** - * Fill in missing values for all vertices in the index. - * - * @param missingValue the value to use for vertices that don't currently have values. - * @return A VertexSetRDD with a value for all vertices. - */ - def fillMissing(missingValue: VD): VertexSetRDD[VD] = { - // TODO: I think this can be done using a join. - this.mapVertexPartitions { part => - // Allocate a new values array with missing value as the default - val newValues = Array.fill(part.values.size)(missingValue) - // Copy over the old values - part.mask.iterator.foreach { ind => - newValues(ind) = part.values(ind) - } - // Create a new mask with all vertices in the index - val newMask = part.index.getBitSet - new VertexPartition(part.index, newValues, newMask) - } - } - /** * Inner join this VertexSet with another VertexSet which has the * same Index. This function will fail if both VertexSets do not diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala b/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala index 8d28bb5ce166c..dbd1ece6cdad3 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala @@ -49,17 +49,17 @@ class VTableReplicated[VD: ClassManifest]( // vertexPlacement val msgsByPartition = placement.zipPartitions(vTable.partitionsRDD) { (pid2vidIter, vertexPartIter) => - val pid2vid = pid2vidIter.next() - val vertexPart = vertexPartIter.next() - - val vmap = new PrimitiveKeyOpenHashMap(vertexPart.index, vertexPart.values) + val pid2vid: Array[Array[Vid]] = pid2vidIter.next() + val vertexPart: VertexPartition[VD] = vertexPartIter.next() val output = new Array[(Pid, VertexAttributeBlock[VD])](pid2vid.size) for (pid <- 0 until pid2vid.size) { - val block = new VertexAttributeBlock(pid2vid(pid), pid2vid(pid).map(vid => vmap(vid))) + val block = new VertexAttributeBlock( + pid2vid(pid), pid2vid(pid).map(vid => vertexPart(vid))) output(pid) = (pid, block) } output.iterator }.partitionBy(eTable.partitioner.get).cache() + // TODO: Consider using a specialized shuffler. // Within each edge partition, create a local map from vid to an index into // the attribute array. Each map contains a superset of the vertices that it diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala index 3d80ab1bb9313..d89143505e048 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala @@ -30,9 +30,9 @@ private[graph] object VertexPartition { private[graph] class VertexPartition[@specialized(Long, Int, Double) VD: ClassManifest]( - val index: VertexIdToIndexMap, - val values: Array[VD], - val mask: BitSet) + private val index: VertexIdToIndexMap, + private val values: Array[VD], + private val mask: BitSet) extends Logging { // TODO: Encapsulate the internal data structures in this class so callers don't need to @@ -42,7 +42,10 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassManifest]( val capacity: Int = index.capacity - def size: Int = mask.cardinality + def size: Int = mask.cardinality() + + /** Return the vertex attribute for the given vertex ID. */ + def apply(vid: Vid): VD = values(index.getPos(vid)) /** * Pass each vertex attribute along with the vertex id through a map diff --git a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala index e095eedce62a0..47b98cfd80b15 100644 --- a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala @@ -20,15 +20,6 @@ class GraphSuite extends FunSuite with LocalSparkContext { } } - test("mapReduceTriplets") { - withSpark(new SparkContext("local", "test")) { sc => - val edges = sc.parallelize((0L to 100L).zip((1L to 99L) :+ 0L)) - val graph = Graph.fromEdgeTuples(edges, 1.0F) - - val d = graph.mapReduceTriplets[Int](et => Iterator((et.srcId, 0)), (a, b) => a + b) - } - } - test("Graph Creation with invalid vertices") { withSpark(new SparkContext("local", "test")) { sc => val rawEdges = (0L to 98L).zip((1L to 99L) :+ 0L) @@ -63,13 +54,6 @@ class GraphSuite extends FunSuite with LocalSparkContext { withSpark(new SparkContext("local", "test")) { sc => val n = 3 val star = Graph.fromEdgeTuples(sc.parallelize((1 to n).map(x => (0: Vid, x: Vid))), 0) - - println("--------------------------------------- star vertices") - println(star.vertices.partitionsRDD.map { v => v.index.toString }.collect().toSeq) - - println("--------------------------------------- starDeg") - println(star.degrees.partitionsRDD.map { v => v.index.toString }.collect().toSeq) - val starDeg = star.joinVertices(star.degrees){ (vid, oldV, deg) => deg } val neighborDegreeSums = starDeg.mapReduceTriplets( edge => Iterator((edge.srcId, edge.dstAttr), (edge.dstId, edge.srcAttr)), From 34ee81415eacc990e3e709daa833318868aff763 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 30 Nov 2013 15:10:30 -0800 Subject: [PATCH 241/531] Merged Ankur's pull request #80 and fixed subgraph. --- .../main/scala/org/apache/spark/graph/Edge.scala | 2 -- .../scala/org/apache/spark/graph/EdgeTriplet.scala | 1 - .../org/apache/spark/graph/impl/GraphImpl.scala | 13 +++++++++---- .../scala/org/apache/spark/graph/GraphSuite.scala | 13 +++++++++---- 4 files changed, 18 insertions(+), 11 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/Edge.scala b/graph/src/main/scala/org/apache/spark/graph/Edge.scala index 509a734338c32..f64983e79b510 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Edge.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Edge.scala @@ -30,7 +30,6 @@ case class Edge[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED] def otherVertexId(vid: Vid): Vid = if (srcId == vid) dstId else { assert(dstId == vid); srcId } - /** * Return the relative direction of the edge to the corresponding * vertex. @@ -41,5 +40,4 @@ case class Edge[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED] */ def relativeDirection(vid: Vid): EdgeDirection = if (vid == srcId) EdgeDirection.Out else { assert(vid == dstId); EdgeDirection.In } - } diff --git a/graph/src/main/scala/org/apache/spark/graph/EdgeTriplet.scala b/graph/src/main/scala/org/apache/spark/graph/EdgeTriplet.scala index aace6e54fe314..39119fc455b84 100644 --- a/graph/src/main/scala/org/apache/spark/graph/EdgeTriplet.scala +++ b/graph/src/main/scala/org/apache/spark/graph/EdgeTriplet.scala @@ -53,5 +53,4 @@ class EdgeTriplet[VD, ED] extends Edge[ED] { */ def vertexAttr(vid: Vid): VD = if (srcId == vid) srcAttr else { assert(dstId == vid); dstAttr } - } diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index 27c2dfa61609f..49273b3566094 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -45,10 +45,13 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( /** Return a RDD that brings edges with its source and destination vertices together. */ @transient override val triplets: RDD[EdgeTriplet[VD, ED]] = { + val vdManifest = classManifest[VD] + val edManifest = classManifest[ED] + eTable.zipPartitions(vTableReplicated.bothAttrs) { (eTableIter, vTableReplicatedIter) => val (_, edgePartition) = eTableIter.next() val (_, (vidToIndex, vertexArray)) = vTableReplicatedIter.next() - new EdgeTripletIterator(vidToIndex, vertexArray, edgePartition) + new EdgeTripletIterator(vidToIndex, vertexArray, edgePartition)(vdManifest, edManifest) } } @@ -174,10 +177,12 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( // this graph val newVTable = vTable.mapVertexPartitions(_.filter(vpred).reindex()) + val edges = triplets.filter { et => + vpred(et.srcId, et.srcAttr) && vpred(et.dstId, et.dstAttr) && epred(et) + }.map(et => Edge(et.srcId, et.dstId, et.attr)) + // Restrict the set of edges to those that satisfy the vertex and the edge predicate. - val newETable = createETable( - triplets.filter(t => vpred(t.srcId, t.srcAttr) && vpred(t.dstId, t.dstAttr) && epred(t)) - .map(t => Edge(t.srcId, t.dstId, t.attr)), partitioner) + val newETable = createETable(edges, partitioner) // Construct the VertexPlacement map val newVertexPlacement = new VertexPlacement(newETable, newVTable) diff --git a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala index 2a040de7fe990..e176cf71e9301 100644 --- a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala @@ -136,12 +136,17 @@ class GraphSuite extends FunSuite with LocalSparkContext { test("subgraph") { withSpark(new SparkContext("local", "test")) { sc => + // Create a star graph of 10 veritces. val n = 10 - val star = Graph(sc.parallelize((1 to n).map(x => (0: Vid, x: Vid))), "defaultValue") + val star = Graph.fromEdgeTuples(sc.parallelize((1 to n).map(x => (0: Vid, x: Vid))), "v") + // Take only vertices whose vids are even val subgraph = star.subgraph(vpred = (vid, attr) => vid % 2 == 0) - assert(subgraph.vertices.collect().toSet === - (0 to n / 2).map(x => (x * 2, "defaultValue")).toSet) - assert(subgraph.edges.collect().toSet === (1 to n / 2).map(x => Edge(0, x * 2)).toSet) + + // We should have 5 vertices. + assert(subgraph.vertices.collect().toSet === (0 to n / 2).map(x => (x * 2, "v")).toSet) + + // And 4 edges. + assert(subgraph.edges.map(_.copy()).collect().toSet === (1 to n / 2).map(x => Edge(0, x * 2, 1)).toSet) } } From 6eeadb667de5b05bb4877ba4c5d1cb46091eefc7 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 30 Nov 2013 16:53:54 -0800 Subject: [PATCH 242/531] Created EdgeRDD. --- .../org/apache/spark/graph/Analytics.scala | 4 +- .../scala/org/apache/spark/graph/Edge.scala | 2 +- .../org/apache/spark/graph/EdgeRDD.scala | 58 +++++++++ .../org/apache/spark/graph/EdgeTriplet.scala | 2 + .../scala/org/apache/spark/graph/Graph.scala | 4 +- .../org/apache/spark/graph/GraphOps.scala | 14 +- .../{VertexSetRDD.scala => VertexRDD.scala} | 121 +++++++++-------- .../apache/spark/graph/impl/GraphImpl.scala | 122 +++++++++--------- .../spark/graph/impl/VTableReplicated.scala | 10 +- .../spark/graph/impl/VertexPartition.scala | 5 - .../spark/graph/impl/VertexPlacement.scala | 10 +- .../org/apache/spark/graph/GraphSuite.scala | 2 +- 12 files changed, 199 insertions(+), 155 deletions(-) create mode 100644 graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala rename graph/src/main/scala/org/apache/spark/graph/{VertexSetRDD.scala => VertexRDD.scala} (74%) diff --git a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala index ea501c74355d1..db1bb8e2bec8c 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala @@ -228,7 +228,7 @@ object Analytics extends Logging { val graph = rawGraph.groupEdges( (a,b) => a ).cache // Construct set representations of the neighborhoods - val nbrSets: VertexSetRDD[VertexSet] = + val nbrSets: VertexRDD[VertexSet] = graph.collectNeighborIds(EdgeDirection.Both).mapValues { (vid, nbrs) => val set = new VertexSet(4) var i = 0 @@ -263,7 +263,7 @@ object Analytics extends Logging { Iterator((et.srcId, counter), (et.dstId, counter)) } // compute the intersection along edges - val counters: VertexSetRDD[Int] = setGraph.mapReduceTriplets(edgeFunc, _ + _) + val counters: VertexRDD[Int] = setGraph.mapReduceTriplets(edgeFunc, _ + _) // Merge counters with the graph and divide by two since each triangle is counted twice graph.outerJoinVertices(counters) { (vid, _, optCounter: Option[Int]) => diff --git a/graph/src/main/scala/org/apache/spark/graph/Edge.scala b/graph/src/main/scala/org/apache/spark/graph/Edge.scala index f64983e79b510..1aa1b36b4725a 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Edge.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Edge.scala @@ -18,7 +18,7 @@ case class Edge[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED] var dstId: Vid = 0, /** * The attribute associated with the edge. - */ + */ var attr: ED = nullValue[ED]) { /** diff --git a/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala b/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala new file mode 100644 index 0000000000000..e9174df53d826 --- /dev/null +++ b/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala @@ -0,0 +1,58 @@ +package org.apache.spark.graph + + +import org.apache.spark.{TaskContext, Partition, OneToOneDependency} +import org.apache.spark.graph.impl.EdgePartition +import org.apache.spark.rdd.RDD +import org.apache.spark.storage.StorageLevel + + +class EdgeRDD[@specialized ED: ClassManifest]( + val partitionsRDD: RDD[(Pid, EdgePartition[ED])]) + extends RDD[Edge[ED]](partitionsRDD.context, List(new OneToOneDependency(partitionsRDD))) { + + override val partitioner = partitionsRDD.partitioner + + override protected def getPartitions: Array[Partition] = partitionsRDD.partitions + + override def compute(split: Partition, context: TaskContext): Iterator[Edge[ED]] = { + val edgePartition = partitionsRDD.compute(split, context).next()._2 + edgePartition.iterator + } + + override def collect(): Array[Edge[ED]] = this.map(_.copy()).collect() + + /** + * Caching a VertexRDD causes the index and values to be cached separately. + */ + override def persist(newLevel: StorageLevel): EdgeRDD[ED] = { + partitionsRDD.persist(newLevel) + this + } + + /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */ + override def persist(): EdgeRDD[ED] = persist(StorageLevel.MEMORY_ONLY) + + /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */ + override def cache(): EdgeRDD[ED] = persist() + + def mapEdgePartitions[ED2: ClassManifest](f: EdgePartition[ED] => EdgePartition[ED2]) + : EdgeRDD[ED2]= { + val cleanF = sparkContext.clean(f) + new EdgeRDD[ED2](partitionsRDD.mapPartitions({ iter => + val (pid, ep) = iter.next() + Iterator(Tuple2(pid, cleanF(ep))) + }, preservesPartitioning = true)) + } + + def zipEdgePartitions[T: ClassManifest, U: ClassManifest] + (other: RDD[T]) + (f: (EdgePartition[ED], Iterator[T]) => Iterator[U]): RDD[U] = { + val cleanF = sparkContext.clean(f) + partitionsRDD.zipPartitions(other, preservesPartitioning = true) { (ePartIter, otherIter) => + val (_, edgePartition) = ePartIter.next() + cleanF(edgePartition, otherIter) + } + } + +} diff --git a/graph/src/main/scala/org/apache/spark/graph/EdgeTriplet.scala b/graph/src/main/scala/org/apache/spark/graph/EdgeTriplet.scala index 39119fc455b84..b8df94c419420 100644 --- a/graph/src/main/scala/org/apache/spark/graph/EdgeTriplet.scala +++ b/graph/src/main/scala/org/apache/spark/graph/EdgeTriplet.scala @@ -53,4 +53,6 @@ class EdgeTriplet[VD, ED] extends Edge[ED] { */ def vertexAttr(vid: Vid): VD = if (srcId == vid) srcAttr else { assert(dstId == vid); dstAttr } + + override def toString() = ((srcId, srcAttr), (dstId, dstAttr), attr).toString() } diff --git a/graph/src/main/scala/org/apache/spark/graph/Graph.scala b/graph/src/main/scala/org/apache/spark/graph/Graph.scala index 425c9edefeb23..d81275c5ba042 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Graph.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Graph.scala @@ -34,7 +34,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * @see Vertex for the vertex type. * */ - val vertices: VertexSetRDD[VD] + val vertices: VertexRDD[VD] /** * Get the Edges and their data as an RDD. The entries in the RDD @@ -243,7 +243,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { def mapReduceTriplets[A: ClassManifest]( mapFunc: EdgeTriplet[VD, ED] => Iterator[(Vid, A)], reduceFunc: (A, A) => A) - : VertexSetRDD[A] + : VertexRDD[A] /** * Join the vertices with an RDD and then apply a function from the diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala b/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala index 3bfad2131e246..ee74a36c21b71 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala @@ -35,14 +35,14 @@ class GraphOps[VD: ClassManifest, ED: ClassManifest](graph: Graph[VD, ED]) { * RDD. * @note Vertices with no in edges are not returned in the resulting RDD. */ - lazy val inDegrees: VertexSetRDD[Int] = degreesRDD(EdgeDirection.In) + lazy val inDegrees: VertexRDD[Int] = degreesRDD(EdgeDirection.In) /** * Compute the out-degree of each vertex in the Graph returning an RDD. * @note Vertices with no out edges are not returned in the resulting RDD. */ - lazy val outDegrees: VertexSetRDD[Int] = degreesRDD(EdgeDirection.Out) + lazy val outDegrees: VertexRDD[Int] = degreesRDD(EdgeDirection.Out) /** @@ -50,7 +50,7 @@ class GraphOps[VD: ClassManifest, ED: ClassManifest](graph: Graph[VD, ED]) { * @note Vertices with no edges are not returned in the resulting * RDD. */ - lazy val degrees: VertexSetRDD[Int] = degreesRDD(EdgeDirection.Both) + lazy val degrees: VertexRDD[Int] = degreesRDD(EdgeDirection.Both) /** @@ -59,7 +59,7 @@ class GraphOps[VD: ClassManifest, ED: ClassManifest](graph: Graph[VD, ED]) { * @param edgeDirection the direction along which to collect * neighboring vertex attributes. */ - private def degreesRDD(edgeDirection: EdgeDirection): VertexSetRDD[Int] = { + private def degreesRDD(edgeDirection: EdgeDirection): VertexRDD[Int] = { if (edgeDirection == EdgeDirection.In) { graph.mapReduceTriplets(et => Iterator((et.dstId,1)), _ + _) } else if (edgeDirection == EdgeDirection.Out) { @@ -114,7 +114,7 @@ class GraphOps[VD: ClassManifest, ED: ClassManifest](graph: Graph[VD, ED]) { mapFunc: (Vid, EdgeTriplet[VD, ED]) => Option[A], reduceFunc: (A, A) => A, dir: EdgeDirection) - : VertexSetRDD[A] = { + : VertexRDD[A] = { ClosureCleaner.clean(mapFunc) ClosureCleaner.clean(reduceFunc) @@ -154,7 +154,7 @@ class GraphOps[VD: ClassManifest, ED: ClassManifest](graph: Graph[VD, ED]) { * @return the vertex set of neighboring ids for each vertex. */ def collectNeighborIds(edgeDirection: EdgeDirection) : - VertexSetRDD[Array[Vid]] = { + VertexRDD[Array[Vid]] = { val nbrs = if (edgeDirection == EdgeDirection.Both) { graph.mapReduceTriplets[Array[Vid]]( @@ -190,7 +190,7 @@ class GraphOps[VD: ClassManifest, ED: ClassManifest](graph: Graph[VD, ED]) { * vertex. */ def collectNeighbors(edgeDirection: EdgeDirection) : - VertexSetRDD[ Array[(Vid, VD)] ] = { + VertexRDD[ Array[(Vid, VD)] ] = { val nbrs = graph.aggregateNeighbors[Array[(Vid,VD)]]( (vid, edge) => Some(Array( (edge.otherVertexId(vid), edge.otherVertexAttr(vid)) )), diff --git a/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala b/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala similarity index 74% rename from graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala rename to graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala index 0206dc9b376a8..5f9672e07b540 100644 --- a/graph/src/main/scala/org/apache/spark/graph/VertexSetRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala @@ -28,47 +28,41 @@ import org.apache.spark.util.ClosureCleaner /** - * A `VertexSetRDD[VD]` extends the `RDD[(Vid, VD)]` by ensuring that there is + * A `VertexRDD[VD]` extends the `RDD[(Vid, VD)]` by ensuring that there is * only one entry for each vertex and by pre-indexing the entries for fast, * efficient joins. * * @tparam VD the vertex attribute associated with each vertex in the set. * - * To construct a `VertexSetRDD` use the singleton object: + * To construct a `VertexRDD` use the singleton object: * - * @example Construct a `VertexSetRDD` from a plain RDD + * @example Construct a `VertexRDD` from a plain RDD * {{{ * // Construct an intial vertex set * val someData: RDD[(Vid, SomeType)] = loadData(someFile) - * val vset = VertexSetRDD(someData) + * val vset = VertexRDD(someData) * // If there were redundant values in someData we would use a reduceFunc - * val vset2 = VertexSetRDD(someData, reduceFunc) - * // Finally we can use the VertexSetRDD to index another dataset + * val vset2 = VertexRDD(someData, reduceFunc) + * // Finally we can use the VertexRDD to index another dataset * val otherData: RDD[(Vid, OtherType)] = loadData(otherFile) - * val vset3 = VertexSetRDD(otherData, vset.index) + * val vset3 = VertexRDD(otherData, vset.index) * // Now we can construct very fast joins between the two sets - * val vset4: VertexSetRDD[(SomeType, OtherType)] = vset.leftJoin(vset3) + * val vset4: VertexRDD[(SomeType, OtherType)] = vset.leftJoin(vset3) * }}} * */ -class VertexSetRDD[@specialized VD: ClassManifest]( - @transient val partitionsRDD: RDD[VertexPartition[VD]]) +class VertexRDD[@specialized VD: ClassManifest]( + val partitionsRDD: RDD[VertexPartition[VD]]) extends RDD[(Vid, VD)](partitionsRDD.context, List(new OneToOneDependency(partitionsRDD))) { require(partitionsRDD.partitioner.isDefined) /** - * Construct a new VertexSetRDD that is indexed by only the keys in the RDD. + * Construct a new VertexRDD that is indexed by only the keys in the RDD. * The resulting VertexSet will be based on a different index and can * no longer be quickly joined with this RDD. */ - def reindex(): VertexSetRDD[VD] = new VertexSetRDD(partitionsRDD.map(_.reindex())) - - /** - * An internal representation which joins the block indices with the values - * This is used by the compute function to emulate `RDD[(Vid, VD)]` - */ - protected[spark] val tuples: RDD[(Vid, VD)] = partitionsRDD.flatMap(_.iterator) + def reindex(): VertexRDD[VD] = new VertexRDD(partitionsRDD.map(_.reindex())) /** * The partitioner is defined by the index. @@ -78,28 +72,28 @@ class VertexSetRDD[@specialized VD: ClassManifest]( /** * The actual partitions are defined by the tuples. */ - override def getPartitions: Array[Partition] = tuples.partitions + override protected def getPartitions: Array[Partition] = partitionsRDD.partitions /** * The preferred locations are computed based on the preferred * locations of the tuples. */ - override def getPreferredLocations(s: Partition): Seq[String] = - tuples.preferredLocations(s) + override protected def getPreferredLocations(s: Partition): Seq[String] = + partitionsRDD.preferredLocations(s) /** - * Caching a VertexSetRDD causes the index and values to be cached separately. + * Caching a VertexRDD causes the index and values to be cached separately. */ - override def persist(newLevel: StorageLevel): VertexSetRDD[VD] = { + override def persist(newLevel: StorageLevel): VertexRDD[VD] = { partitionsRDD.persist(newLevel) this } /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */ - override def persist(): VertexSetRDD[VD] = persist(StorageLevel.MEMORY_ONLY) + override def persist(): VertexRDD[VD] = persist(StorageLevel.MEMORY_ONLY) /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */ - override def cache(): VertexSetRDD[VD] = persist() + override def cache(): VertexRDD[VD] = persist() /** Return the number of vertices in this set. */ override def count(): Long = { @@ -109,26 +103,27 @@ class VertexSetRDD[@specialized VD: ClassManifest]( /** * Provide the `RDD[(Vid, VD)]` equivalent output. */ - override def compute(part: Partition, context: TaskContext): Iterator[(Vid, VD)] = - tuples.compute(part, context) + override def compute(part: Partition, context: TaskContext): Iterator[(Vid, VD)] = { + partitionsRDD.compute(part, context).next().iterator + } /** - * Return a new VertexSetRDD by applying a function to each VertexPartition of this RDD. + * Return a new VertexRDD by applying a function to each VertexPartition of this RDD. */ def mapVertexPartitions[VD2: ClassManifest](f: VertexPartition[VD] => VertexPartition[VD2]) - : VertexSetRDD[VD2] = { + : VertexRDD[VD2] = { val cleanF = sparkContext.clean(f) val newPartitionsRDD = partitionsRDD.mapPartitions(_.map(cleanF), preservesPartitioning = true) - new VertexSetRDD(newPartitionsRDD) + new VertexRDD(newPartitionsRDD) } /** - * Return a new VertexSetRDD by applying a function to corresponding - * VertexPartitions of this VertexSetRDD and another one. + * Return a new VertexRDD by applying a function to corresponding + * VertexPartitions of this VertexRDD and another one. */ private def zipVertexPartitions[VD2: ClassManifest, VD3: ClassManifest] - (other: VertexSetRDD[VD2]) - (f: (VertexPartition[VD], VertexPartition[VD2]) => VertexPartition[VD3]): VertexSetRDD[VD3] = { + (other: VertexRDD[VD2]) + (f: (VertexPartition[VD], VertexPartition[VD2]) => VertexPartition[VD3]): VertexRDD[VD3] = { val cleanF = sparkContext.clean(f) val newPartitionsRDD = partitionsRDD.zipPartitions( other.partitionsRDD, preservesPartitioning = true @@ -137,7 +132,7 @@ class VertexSetRDD[@specialized VD: ClassManifest]( val otherPart = otherIter.next() Iterator(cleanF(thisPart, otherPart)) } - new VertexSetRDD(newPartitionsRDD) + new VertexRDD(newPartitionsRDD) } /** @@ -152,7 +147,7 @@ class VertexSetRDD[@specialized VD: ClassManifest]( * the original vertex-set. Furthermore, the filter only * modifies the bitmap index and so no new values are allocated. */ - override def filter(pred: Tuple2[Vid, VD] => Boolean): VertexSetRDD[VD] = + override def filter(pred: Tuple2[Vid, VD] => Boolean): VertexRDD[VD] = this.mapVertexPartitions(_.filter(Function.untupled(pred))) /** @@ -162,11 +157,11 @@ class VertexSetRDD[@specialized VD: ClassManifest]( * @tparam VD2 the type returned by the map function * * @param f the function applied to each value in the RDD - * @return a new VertexSetRDD with values obtained by applying `f` to + * @return a new VertexRDD with values obtained by applying `f` to * each of the entries in the original VertexSet. The resulting - * VertexSetRDD retains the same index. + * VertexRDD retains the same index. */ - def mapValues[VD2: ClassManifest](f: VD => VD2): VertexSetRDD[VD2] = + def mapValues[VD2: ClassManifest](f: VD => VD2): VertexRDD[VD2] = this.mapVertexPartitions(_.map((vid, attr) => f(attr))) /** @@ -176,11 +171,11 @@ class VertexSetRDD[@specialized VD: ClassManifest]( * @tparam VD2 the type returned by the map function * * @param f the function applied to each value in the RDD - * @return a new VertexSetRDD with values obtained by applying `f` to + * @return a new VertexRDD with values obtained by applying `f` to * each of the entries in the original VertexSet. The resulting - * VertexSetRDD retains the same index. + * VertexRDD retains the same index. */ - def mapValues[VD2: ClassManifest](f: (Vid, VD) => VD2): VertexSetRDD[VD2] = + def mapValues[VD2: ClassManifest](f: (Vid, VD) => VD2): VertexRDD[VD2] = this.mapVertexPartitions(_.map(f)) /** @@ -195,12 +190,12 @@ class VertexSetRDD[@specialized VD: ClassManifest]( * @param other the other VertexSet with which to join. * @param f the function mapping a vertex id and its attributes in * this and the other vertex set to a new vertex attribute. - * @return a VertexSetRDD containing only the vertices in both this + * @return a VertexRDD containing only the vertices in both this * and the other VertexSet and with tuple attributes. * */ def zipJoin[VD2: ClassManifest, VD3: ClassManifest] - (other: VertexSetRDD[VD2])(f: (Vid, VD, VD2) => VD3): VertexSetRDD[VD3] = + (other: VertexRDD[VD2])(f: (Vid, VD, VD2) => VD3): VertexRDD[VD3] = { this.zipVertexPartitions(other) { (thisPart, otherPart) => thisPart.join(otherPart)(f) @@ -220,13 +215,13 @@ class VertexSetRDD[@specialized VD: ClassManifest]( * @param other the other VertexSet with which to join. * @param f the function mapping a vertex id and its attributes in * this and the other vertex set to a new vertex attribute. - * @return a VertexSetRDD containing all the vertices in this + * @return a VertexRDD containing all the vertices in this * VertexSet with `None` attributes used for Vertices missing in the * other VertexSet. * */ def leftZipJoin[VD2: ClassManifest, VD3: ClassManifest] - (other: VertexSetRDD[VD2])(f: (Vid, VD, Option[VD2]) => VD3): VertexSetRDD[VD3] = + (other: VertexRDD[VD2])(f: (Vid, VD, Option[VD2]) => VD3): VertexRDD[VD3] = { this.zipVertexPartitions(other) { (thisPart, otherPart) => thisPart.leftJoin(otherPart)(f) @@ -249,21 +244,21 @@ class VertexSetRDD[@specialized VD: ClassManifest]( * @param other the other VertexSet with which to join. * @param f the function mapping a vertex id and its attributes in * this and the other vertex set to a new vertex attribute. - * @return a VertexSetRDD containing all the vertices in this + * @return a VertexRDD containing all the vertices in this * VertexSet with the attribute emitted by f. */ def leftJoin[VD2: ClassManifest, VD3: ClassManifest] (other: RDD[(Vid, VD2)]) (f: (Vid, VD, Option[VD2]) => VD3) - : VertexSetRDD[VD3] = + : VertexRDD[VD3] = { - // Test if the other vertex is a VertexSetRDD to choose the optimal join strategy. - // If the other set is a VertexSetRDD then we use the much more efficient leftZipJoin + // Test if the other vertex is a VertexRDD to choose the optimal join strategy. + // If the other set is a VertexRDD then we use the much more efficient leftZipJoin other match { - case other: VertexSetRDD[_] => + case other: VertexRDD[_] => leftZipJoin(other)(f) case _ => - new VertexSetRDD[VD3]( + new VertexRDD[VD3]( partitionsRDD.zipPartitions( other.partitionBy(this.partitioner.get), preservesPartitioning = true) { (part, msgs) => @@ -275,7 +270,7 @@ class VertexSetRDD[@specialized VD: ClassManifest]( } def aggregateUsingIndex[VD2: ClassManifest, VidVDPair <: Product2[Vid, VD2] : ClassManifest]( - messages: RDD[VidVDPair], reduceFunc: (VD2, VD2) => VD2): VertexSetRDD[VD2] = + messages: RDD[VidVDPair], reduceFunc: (VD2, VD2) => VD2): VertexRDD[VD2] = { // TODO: use specialized shuffle serializer. val shuffled = new ShuffledRDD[Vid, VD2, VidVDPair](messages, this.partitioner.get) @@ -283,16 +278,16 @@ class VertexSetRDD[@specialized VD: ClassManifest]( val vertextPartition: VertexPartition[VD] = thisIter.next() Iterator(vertextPartition.aggregateUsingIndex(msgIter, reduceFunc)) } - new VertexSetRDD[VD2](parts) + new VertexRDD[VD2](parts) } -} // end of VertexSetRDD +} // end of VertexRDD /** - * The VertexSetRDD singleton is used to construct VertexSets + * The VertexRDD singleton is used to construct VertexSets */ -object VertexSetRDD { +object VertexRDD { /** * Construct a vertex set from an RDD of vertex-attribute pairs. @@ -302,7 +297,7 @@ object VertexSetRDD { * * @param rdd the collection of vertex-attribute pairs */ - def apply[VD: ClassManifest](rdd: RDD[(Vid, VD)]): VertexSetRDD[VD] = { + def apply[VD: ClassManifest](rdd: RDD[(Vid, VD)]): VertexRDD[VD] = { val partitioned: RDD[(Vid, VD)] = rdd.partitioner match { case Some(p) => rdd case None => rdd.partitionBy(new HashPartitioner(rdd.partitions.size)) @@ -310,7 +305,7 @@ object VertexSetRDD { val vertexPartitions = partitioned.mapPartitions( iter => Iterator(VertexPartition(iter)), preservesPartitioning = true) - new VertexSetRDD(vertexPartitions) + new VertexRDD(vertexPartitions) } /** @@ -322,7 +317,7 @@ object VertexSetRDD { * @param rdd the collection of vertex-attribute pairs * @param mergeFunc the associative, commutative merge function. */ - def apply[VD: ClassManifest](rdd: RDD[(Vid, VD)], mergeFunc: (VD, VD) => VD): VertexSetRDD[VD] = + def apply[VD: ClassManifest](rdd: RDD[(Vid, VD)], mergeFunc: (VD, VD) => VD): VertexRDD[VD] = { val partitioned: RDD[(Vid, VD)] = rdd.partitioner match { case Some(p) => rdd @@ -331,13 +326,13 @@ object VertexSetRDD { val vertexPartitions = partitioned.mapPartitions( iter => Iterator(VertexPartition(iter)), preservesPartitioning = true) - new VertexSetRDD(vertexPartitions) + new VertexRDD(vertexPartitions) } def apply[VD: ClassManifest](vids: RDD[Vid], rdd: RDD[(Vid, VD)], defaultVal: VD) - : VertexSetRDD[VD] = + : VertexRDD[VD] = { - VertexSetRDD(vids.map(vid => (vid, defaultVal))).leftJoin(rdd) { (vid, default, value) => + VertexRDD(vids.map(vid => (vid, defaultVal))).leftJoin(rdd) { (vid, default, value) => value.getOrElse(default) } } diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index 49273b3566094..b89336fcc1ab8 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -25,39 +25,43 @@ import org.apache.spark.util.collection.{BitSet, OpenHashSet, PrimitiveKeyOpenHa * are co-partitioned with the relevant edges. */ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( - @transient val vTable: VertexSetRDD[VD], - @transient val eTable: RDD[(Pid, EdgePartition[ED])], + @transient val vertices: VertexRDD[VD], + @transient val edges: EdgeRDD[ED], @transient val vertexPlacement: VertexPlacement, - @transient val partitioner: PartitionStrategy) + @transient val partitionStrategy: PartitionStrategy) extends Graph[VD, ED] { - def this() = this(null, null, null, null) + //def this() = this(null, null, null, null) - @transient val vTableReplicated: VTableReplicated[VD] = - new VTableReplicated(vTable, eTable, vertexPlacement) + def this( + vertices: RDD[VertexPartition[VD]], + edges: RDD[(Pid, EdgePartition[ED])], + vertexPlacement: VertexPlacement, + partitionStrategy: PartitionStrategy) = { + this(new VertexRDD(vertices), new EdgeRDD(edges), vertexPlacement, partitionStrategy) + } - /** Return a RDD of vertices. */ - @transient override val vertices = vTable + @transient val vTableReplicated: VTableReplicated[VD] = + new VTableReplicated(vertices, edges, vertexPlacement) /** Return a RDD of edges. */ - @transient override val edges: RDD[Edge[ED]] = - eTable.mapPartitions(_.next()._2.iterator, true) +// @transient override val edges: RDD[Edge[ED]] = +// edges.mapPartitions(_.next()._2.iterator, true) /** Return a RDD that brings edges with its source and destination vertices together. */ @transient override val triplets: RDD[EdgeTriplet[VD, ED]] = { val vdManifest = classManifest[VD] val edManifest = classManifest[ED] - eTable.zipPartitions(vTableReplicated.bothAttrs) { (eTableIter, vTableReplicatedIter) => - val (_, edgePartition) = eTableIter.next() + edges.zipEdgePartitions(vTableReplicated.bothAttrs) { (edgePartition, vTableReplicatedIter) => val (_, (vidToIndex, vertexArray)) = vTableReplicatedIter.next() new EdgeTripletIterator(vidToIndex, vertexArray, edgePartition)(vdManifest, edManifest) } } override def persist(newLevel: StorageLevel): Graph[VD, ED] = { - vTable.persist(newLevel) - eTable.persist(newLevel) + vertices.persist(newLevel) + edges.persist(newLevel) vertexPlacement.persist(newLevel) this } @@ -76,7 +80,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( val replicationRatioSrcOnly = numReplicatedVertices(vertexPlacement.srcAttrOnly) / numVertices val replicationRatioDstOnly = numReplicatedVertices(vertexPlacement.dstAttrOnly) / numVertices // One entry for each partition, indicate the total number of edges on that partition. - val loadArray = eTable.map { case (_, e) => e.size }.collect().map(_.toDouble / numEdges) + val loadArray = edges.partitionsRDD.map(_._2.size).collect().map(_.toDouble / numEdges) val minLoad = loadArray.min val maxLoad = loadArray.max Map( @@ -116,11 +120,11 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( } } println("eTable ------------------------------------------") - traverseLineage(eTable, " ") - var visited = Map(eTable.id -> "eTable") + traverseLineage(edges, " ") + var visited = Map(edges.id -> "eTable") println("\n\nvTable ------------------------------------------") - traverseLineage(vTable, " ", visited) - visited += (vTable.id -> "vTable") + traverseLineage(vertices, " ", visited) + visited += (vertices.id -> "vTable") println("\n\nvertexPlacement.bothAttrs -------------------------------") traverseLineage(vertexPlacement.bothAttrs, " ", visited) visited += (vertexPlacement.bothAttrs.id -> "vertexPlacement.bothAttrs") @@ -132,29 +136,22 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( println(visited) } // end of printLineage - override def reverse: Graph[VD, ED] = { - val newETable = eTable.mapPartitions(_.map { case (pid, epart) => (pid, epart.reverse) }, - preservesPartitioning = true) - new GraphImpl(vTable, newETable, vertexPlacement, partitioner) - } + override def reverse: Graph[VD, ED] = + new GraphImpl(vertices, edges.mapEdgePartitions(_.reverse), vertexPlacement, partitionStrategy) override def mapVertices[VD2: ClassManifest](f: (Vid, VD) => VD2): Graph[VD2, ED] = - new GraphImpl(vTable.mapVertexPartitions(_.map(f)), eTable, vertexPlacement, partitioner) + new GraphImpl(vertices.mapVertexPartitions(_.map(f)), edges, vertexPlacement, partitionStrategy) - override def mapEdges[ED2: ClassManifest](f: Edge[ED] => ED2): Graph[VD, ED2] = { - val newETable = eTable.mapPartitions(_.map { case (pid, epart) => (pid, epart.map(f)) }, - preservesPartitioning = true) - new GraphImpl(vTable, newETable, vertexPlacement, partitioner) - } + override def mapEdges[ED2: ClassManifest](f: Edge[ED] => ED2): Graph[VD, ED2] = + new GraphImpl(vertices, edges.mapEdgePartitions(_.map(f)), vertexPlacement, partitionStrategy) override def mapTriplets[ED2: ClassManifest](f: EdgeTriplet[VD, ED] => ED2): Graph[VD, ED2] = { // Use an explicit manifest in PrimitiveKeyOpenHashMap init so we don't pull in the implicit // manifest from GraphImpl (which would require serializing GraphImpl). val vdManifest = classManifest[VD] - val newETable = eTable.zipPartitions(vTableReplicated.bothAttrs, preservesPartitioning = true) { - (eTableIter, vTableReplicatedIter) => - val (pid, edgePartition) = eTableIter.next() - val (_, (vidToIndex, vertexArray)) = vTableReplicatedIter.next() + val newETable = + edges.zipEdgePartitions(vTableReplicated.bothAttrs) { (edgePartition, vTableReplicatedIter) => + val (pid, (vidToIndex, vertexArray)) = vTableReplicatedIter.next() val et = new EdgeTriplet[VD, ED] val vmap = new PrimitiveKeyOpenHashMap[Vid, VD]( vidToIndex, vertexArray)(classManifest[Vid], vdManifest) @@ -166,7 +163,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( } Iterator((pid, newEdgePartition)) } - new GraphImpl(vTable, newETable, vertexPlacement, partitioner) + new GraphImpl(vertices, new EdgeRDD(newETable), vertexPlacement, partitionStrategy) } override def subgraph( @@ -175,27 +172,26 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( // Filter the vertices, reusing the partitioner (but not the index) from // this graph - val newVTable = vTable.mapVertexPartitions(_.filter(vpred).reindex()) + val newVTable = vertices.mapVertexPartitions(_.filter(vpred).reindex()) - val edges = triplets.filter { et => + val newEdges = triplets.filter { et => vpred(et.srcId, et.srcAttr) && vpred(et.dstId, et.dstAttr) && epred(et) }.map(et => Edge(et.srcId, et.dstId, et.attr)) // Restrict the set of edges to those that satisfy the vertex and the edge predicate. - val newETable = createETable(edges, partitioner) + // TODO don't use createETable here as it is inefficient. + val newETable = createETable(newEdges, partitionStrategy) // Construct the VertexPlacement map val newVertexPlacement = new VertexPlacement(newETable, newVTable) - new GraphImpl(newVTable, newETable, newVertexPlacement, partitioner) + new GraphImpl(newVTable, newETable, newVertexPlacement, partitionStrategy) } // end of subgraph override def groupEdges(merge: (ED, ED) => ED): Graph[VD, ED] = { ClosureCleaner.clean(merge) - val newETable = - eTable.mapPartitions({ _.map(p => (p._1, p._2.groupEdges(merge))) }, - preservesPartitioning = true) - new GraphImpl(vTable, newETable, vertexPlacement, partitioner) + val newETable = edges.mapEdgePartitions(_.groupEdges(merge)) + new GraphImpl(vertices, newETable, vertexPlacement, partitionStrategy) } ////////////////////////////////////////////////////////////////////////////////////////////////// @@ -204,7 +200,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( override def mapReduceTriplets[A: ClassManifest]( mapFunc: EdgeTriplet[VD, ED] => Iterator[(Vid, A)], - reduceFunc: (A, A) => A): VertexSetRDD[A] = { + reduceFunc: (A, A) => A): VertexRDD[A] = { ClosureCleaner.clean(mapFunc) ClosureCleaner.clean(reduceFunc) @@ -216,11 +212,10 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( // in the relevant position in an edge. val mapUsesSrcAttr = accessesVertexAttr[VD, ED](mapFunc, "srcAttr") val mapUsesDstAttr = accessesVertexAttr[VD, ED](mapFunc, "dstAttr") - val vs = vTableReplicated.get(mapUsesSrcAttr, mapUsesDstAttr) + val vs: RDD[(Pid, (VertexIdToIndexMap, Array[VD]))] = vTableReplicated.get(mapUsesSrcAttr, mapUsesDstAttr) // Map and combine. - val preAgg = eTable.zipPartitions(vs) { (edgePartitionIter, vTableReplicatedIter) => - val (_, edgePartition) = edgePartitionIter.next() + val preAgg = edges.zipEdgePartitions(vs) { (edgePartition, vTableReplicatedIter) => val (_, (vidToIndex, vertexArray)) = vTableReplicatedIter.next() assert(vidToIndex.capacity == vertexArray.size) val vmap = new PrimitiveKeyOpenHashMap[Vid, VD](vidToIndex, vertexArray)( @@ -264,14 +259,14 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( } // do the final reduction reusing the index map - vTable.aggregateUsingIndex(preAgg, reduceFunc) + vertices.aggregateUsingIndex(preAgg, reduceFunc) } // end of mapReduceTriplets override def outerJoinVertices[U: ClassManifest, VD2: ClassManifest] (updates: RDD[(Vid, U)])(updateF: (Vid, VD, Option[U]) => VD2): Graph[VD2, ED] = { ClosureCleaner.clean(updateF) - val newVTable = vTable.leftJoin(updates)(updateF) - new GraphImpl(newVTable, eTable, vertexPlacement, partitioner) + val newVTable = vertices.leftJoin(updates)(updateF) + new GraphImpl(newVTable, edges, vertexPlacement, partitionStrategy) } } // end of class GraphImpl @@ -286,18 +281,16 @@ object GraphImpl { val etable = createETable(edges, partitionStrategy).cache() // Get the set of all vids - val vids = etable.mapPartitions { iter => - val (_, epart) = iter.next() - assert(!iter.hasNext) - epart.iterator.flatMap(e => Iterator((e.srcId, 0), (e.dstId, 0))) + val vids = etable.flatMap { e => + Iterator((e.srcId, 0), (e.dstId, 0)) } - // Shuffle the vids and create the VertexSetRDD. + // Shuffle the vids and create the VertexRDD. // TODO: Consider doing map side distinct before shuffle. val shuffled = new ShuffledRDD[Vid, Int, (Vid, Int)]( vids, new HashPartitioner(edges.partitions.size)) shuffled.setSerializer(classOf[VidMsgSerializer].getName) - val vtable = VertexSetRDD(shuffled.mapValues(x => defaultValue)) + val vtable = VertexRDD(shuffled.mapValues(x => defaultValue)) val vertexPlacement = new VertexPlacement(etable, vtable) new GraphImpl(vtable, etable, vertexPlacement, partitionStrategy) @@ -316,15 +309,17 @@ object GraphImpl { val vPartitioned = vertices.partitionBy(partitioner) - val vidsFromEdges = etable.flatMap { case (_, p) => Array.concat(p.srcIds, p.dstIds) } - .map(vid => (vid, 0)) - .partitionBy(partitioner) + val vidsFromEdges = { + etable.partitionsRDD.flatMap { case (_, p) => Array.concat(p.srcIds, p.dstIds) } + .map(vid => (vid, 0)) + .partitionBy(partitioner) + } val vids = vPartitioned.zipPartitions(vidsFromEdges) { (vertexIter, vidsFromEdgesIter) => vertexIter.map(_._1) ++ vidsFromEdgesIter.map(_._1) } - val vtable = VertexSetRDD(vids, vPartitioned, defaultVertexAttr) + val vtable = VertexRDD(vids, vPartitioned, defaultVertexAttr) val vertexPlacement = new VertexPlacement(etable, vtable) new GraphImpl(vtable, etable, vertexPlacement, partitionStrategy) @@ -340,18 +335,18 @@ object GraphImpl { */ protected def createETable[ED: ClassManifest]( edges: RDD[Edge[ED]], - partitionStrategy: PartitionStrategy): RDD[(Pid, EdgePartition[ED])] = { + partitionStrategy: PartitionStrategy): EdgeRDD[ED] = { // Get the number of partitions val numPartitions = edges.partitions.size - edges.map { e => + val eTable = edges.map { e => val part: Pid = partitionStrategy.getPartition(e.srcId, e.dstId, numPartitions) // Should we be using 3-tuple or an optimized class new MessageToPartition(part, (e.srcId, e.dstId, e.attr)) } .partitionBy(new HashPartitioner(numPartitions)) - .mapPartitionsWithIndex( (pid, iter) => { + .mapPartitionsWithIndex( { (pid, iter) => val builder = new EdgePartitionBuilder[ED] iter.foreach { message => val data = message.data @@ -360,6 +355,7 @@ object GraphImpl { val edgePartition = builder.toEdgePartition Iterator((pid, edgePartition)) }, preservesPartitioning = true).cache() + new EdgeRDD(eTable) } private def accessesVertexAttr[VD, ED](closure: AnyRef, attrName: String): Boolean = { diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala b/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala index dbd1ece6cdad3..148f016aeae93 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala @@ -11,8 +11,8 @@ import org.apache.spark.graph._ */ private[impl] class VTableReplicated[VD: ClassManifest]( - vTable: VertexSetRDD[VD], - eTable: RDD[(Pid, EdgePartition[ED])] forSome { type ED }, + vTable: VertexRDD[VD], + eTable: EdgeRDD[_], vertexPlacement: VertexPlacement) { val bothAttrs: RDD[(Pid, (VertexIdToIndexMap, Array[VD]))] = @@ -37,8 +37,8 @@ class VTableReplicated[VD: ClassManifest]( } private def createVTableReplicated[VD: ClassManifest]( - vTable: VertexSetRDD[VD], - eTable: RDD[(Pid, EdgePartition[ED])] forSome { type ED }, + vTable: VertexRDD[VD], + eTable: EdgeRDD[_], vertexPlacement: VertexPlacement, includeSrcAttr: Boolean, includeDstAttr: Boolean): RDD[(Pid, (VertexIdToIndexMap, Array[VD]))] = { @@ -66,7 +66,7 @@ class VTableReplicated[VD: ClassManifest]( // will receive, because it stores vids from both the source and destination // of edges. It must always include both source and destination vids because // some operations, such as GraphImpl.mapReduceTriplets, rely on this. - val localVidMap = eTable.mapPartitions(_.map { + val localVidMap = eTable.partitionsRDD.mapPartitions(_.map { case (pid, epart) => val vidToIndex = new VertexIdToIndexMap epart.foreach { e => diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala index 2ab3759e4c4b8..56f0a544822d5 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala @@ -35,11 +35,6 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassManifest]( private val mask: BitSet) extends Logging { - // TODO: Encapsulate the internal data structures in this class so callers don't need to - // understand the internal data structures. This can possibly be achieved by implementing - // the aggregate and join functions in this class, and VertexSetRDD can simply call into - // that. - val capacity: Int = index.capacity def size: Int = mask.cardinality() diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPlacement.scala b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPlacement.scala index f47b2cb58785e..b7aa9b4b49943 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPlacement.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPlacement.scala @@ -11,9 +11,7 @@ import org.apache.spark.util.collection.PrimitiveVector * Stores the layout of replicated vertex attributes for GraphImpl. Tells each * partition of the vertex data where it should go. */ -class VertexPlacement( - eTable: RDD[(Pid, EdgePartition[ED])] forSome { type ED }, - vTable: VertexSetRDD[_]) { +class VertexPlacement(eTable: EdgeRDD[_], vTable: VertexRDD[_]) { val bothAttrs: RDD[Array[Array[Vid]]] = createPid2Vid(true, true) val srcAttrOnly: RDD[Array[Array[Vid]]] = createPid2Vid(true, false) @@ -38,19 +36,19 @@ class VertexPlacement( private def createPid2Vid( includeSrcAttr: Boolean, includeDstAttr: Boolean): RDD[Array[Array[Vid]]] = { // Determine which vertices each edge partition needs by creating a mapping from vid to pid. - val vid2pid: RDD[(Vid, Pid)] = eTable.mapPartitions { iter => + val vid2pid: RDD[(Vid, Pid)] = eTable.partitionsRDD.mapPartitions { iter => val (pid: Pid, edgePartition: EdgePartition[_]) = iter.next() val numEdges = edgePartition.size val vSet = new VertexSet if (includeSrcAttr) { // Add src vertices to the set. - var i = 0 + var i = 0 while (i < numEdges) { vSet.add(edgePartition.srcIds(i)) i += 1 } } if (includeDstAttr) { // Add dst vertices to the set. - var i = 0 + var i = 0 while (i < numEdges) { vSet.add(edgePartition.dstIds(i)) i += 1 diff --git a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala index e176cf71e9301..cf8884bd047cf 100644 --- a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala @@ -121,7 +121,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { test("VertexSetRDD") { withSpark(new SparkContext("local", "test")) { sc => val a = sc.parallelize((0 to 100).map(x => (x.toLong, x.toLong)), 5) - val b = VertexSetRDD(a).mapValues(x => -x).cache() // Allow joining b with a derived RDD of b + val b = VertexRDD(a).mapValues(x => -x).cache() // Allow joining b with a derived RDD of b assert(b.count === 101) assert(b.leftJoin(a){ (id, a, bOpt) => a + bOpt.get }.map(x=> x._2).reduce(_+_) === 0) val c = b.aggregateUsingIndex[Long, (Long, Long)](a, (x, y) => x) From 583a389e3f58a44172e96d962b5b0bca85ef7eb0 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 30 Nov 2013 17:00:54 -0800 Subject: [PATCH 243/531] Removed PartitionStrategy from GraphImpl. --- .../apache/spark/graph/impl/GraphImpl.scala | 41 ++++++++++--------- 1 file changed, 21 insertions(+), 20 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index b89336fcc1ab8..04e03e0b1abac 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -27,8 +27,7 @@ import org.apache.spark.util.collection.{BitSet, OpenHashSet, PrimitiveKeyOpenHa class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( @transient val vertices: VertexRDD[VD], @transient val edges: EdgeRDD[ED], - @transient val vertexPlacement: VertexPlacement, - @transient val partitionStrategy: PartitionStrategy) + @transient val vertexPlacement: VertexPlacement) extends Graph[VD, ED] { //def this() = this(null, null, null, null) @@ -36,9 +35,8 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( def this( vertices: RDD[VertexPartition[VD]], edges: RDD[(Pid, EdgePartition[ED])], - vertexPlacement: VertexPlacement, - partitionStrategy: PartitionStrategy) = { - this(new VertexRDD(vertices), new EdgeRDD(edges), vertexPlacement, partitionStrategy) + vertexPlacement: VertexPlacement) = { + this(new VertexRDD(vertices), new EdgeRDD(edges), vertexPlacement) } @transient val vTableReplicated: VTableReplicated[VD] = @@ -137,13 +135,13 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( } // end of printLineage override def reverse: Graph[VD, ED] = - new GraphImpl(vertices, edges.mapEdgePartitions(_.reverse), vertexPlacement, partitionStrategy) + new GraphImpl(vertices, edges.mapEdgePartitions(_.reverse), vertexPlacement) override def mapVertices[VD2: ClassManifest](f: (Vid, VD) => VD2): Graph[VD2, ED] = - new GraphImpl(vertices.mapVertexPartitions(_.map(f)), edges, vertexPlacement, partitionStrategy) + new GraphImpl(vertices.mapVertexPartitions(_.map(f)), edges, vertexPlacement) override def mapEdges[ED2: ClassManifest](f: Edge[ED] => ED2): Graph[VD, ED2] = - new GraphImpl(vertices, edges.mapEdgePartitions(_.map(f)), vertexPlacement, partitionStrategy) + new GraphImpl(vertices, edges.mapEdgePartitions(_.map(f)), vertexPlacement) override def mapTriplets[ED2: ClassManifest](f: EdgeTriplet[VD, ED] => ED2): Graph[VD, ED2] = { // Use an explicit manifest in PrimitiveKeyOpenHashMap init so we don't pull in the implicit @@ -163,7 +161,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( } Iterator((pid, newEdgePartition)) } - new GraphImpl(vertices, new EdgeRDD(newETable), vertexPlacement, partitionStrategy) + new GraphImpl(vertices, new EdgeRDD(newETable), vertexPlacement) } override def subgraph( @@ -174,24 +172,27 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( // this graph val newVTable = vertices.mapVertexPartitions(_.filter(vpred).reindex()) - val newEdges = triplets.filter { et => - vpred(et.srcId, et.srcAttr) && vpred(et.dstId, et.dstAttr) && epred(et) - }.map(et => Edge(et.srcId, et.dstId, et.attr)) + val edManifest = classManifest[ED] - // Restrict the set of edges to those that satisfy the vertex and the edge predicate. - // TODO don't use createETable here as it is inefficient. - val newETable = createETable(newEdges, partitionStrategy) + val newETable = new EdgeRDD[ED](triplets.filter { et => + vpred(et.srcId, et.srcAttr) && vpred(et.dstId, et.dstAttr) && epred(et) + }.mapPartitionsWithIndex( { (pid, iter) => + val builder = new EdgePartitionBuilder[ED]()(edManifest) + iter.foreach { et => builder.add(et.srcId, et.dstId, et.attr) } + val edgePartition = builder.toEdgePartition + Iterator((pid, edgePartition)) + }, preservesPartitioning = true)).cache() // Construct the VertexPlacement map val newVertexPlacement = new VertexPlacement(newETable, newVTable) - new GraphImpl(newVTable, newETable, newVertexPlacement, partitionStrategy) + new GraphImpl(newVTable, newETable, newVertexPlacement) } // end of subgraph override def groupEdges(merge: (ED, ED) => ED): Graph[VD, ED] = { ClosureCleaner.clean(merge) val newETable = edges.mapEdgePartitions(_.groupEdges(merge)) - new GraphImpl(vertices, newETable, vertexPlacement, partitionStrategy) + new GraphImpl(vertices, newETable, vertexPlacement) } ////////////////////////////////////////////////////////////////////////////////////////////////// @@ -266,7 +267,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( (updates: RDD[(Vid, U)])(updateF: (Vid, VD, Option[U]) => VD2): Graph[VD2, ED] = { ClosureCleaner.clean(updateF) val newVTable = vertices.leftJoin(updates)(updateF) - new GraphImpl(newVTable, edges, vertexPlacement, partitionStrategy) + new GraphImpl(newVTable, edges, vertexPlacement) } } // end of class GraphImpl @@ -293,7 +294,7 @@ object GraphImpl { val vtable = VertexRDD(shuffled.mapValues(x => defaultValue)) val vertexPlacement = new VertexPlacement(etable, vtable) - new GraphImpl(vtable, etable, vertexPlacement, partitionStrategy) + new GraphImpl(vtable, etable, vertexPlacement) } def apply[VD: ClassManifest, ED: ClassManifest]( @@ -322,7 +323,7 @@ object GraphImpl { val vtable = VertexRDD(vids, vPartitioned, defaultVertexAttr) val vertexPlacement = new VertexPlacement(etable, vtable) - new GraphImpl(vtable, etable, vertexPlacement, partitionStrategy) + new GraphImpl(vtable, etable, vertexPlacement) } /** From e36fe55a031d2c01c9d7c5d85965951c681a0c74 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 30 Nov 2013 18:07:36 -0800 Subject: [PATCH 244/531] Memoize preferred locations in ZippedPartitionsBaseRDD so preferred location computation doesn't lead to exponential explosion. --- .../spark/rdd/ZippedPartitionsRDD.scala | 27 ++++++++----------- 1 file changed, 11 insertions(+), 16 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala index faeb316664b00..a97d2a01c8b01 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala @@ -22,7 +22,8 @@ import java.io.{ObjectOutputStream, IOException} private[spark] class ZippedPartitionsPartition( idx: Int, - @transient rdds: Seq[RDD[_]]) + @transient rdds: Seq[RDD[_]], + @transient val preferredLocations: Seq[String]) extends Partition { override val index: Int = idx @@ -47,27 +48,21 @@ abstract class ZippedPartitionsBaseRDD[V: ClassManifest]( if (preservesPartitioning) firstParent[Any].partitioner else None override def getPartitions: Array[Partition] = { - val sizes = rdds.map(x => x.partitions.size) - if (!sizes.forall(x => x == sizes(0))) { + val numParts = rdds.head.partitions.size + if (!rdds.forall(rdd => rdd.partitions.size == numParts)) { throw new IllegalArgumentException("Can't zip RDDs with unequal numbers of partitions") } - val array = new Array[Partition](sizes(0)) - for (i <- 0 until sizes(0)) { - array(i) = new ZippedPartitionsPartition(i, rdds) + Array.tabulate[Partition](numParts) { i => + val prefs = rdds.map(rdd => rdd.preferredLocations(rdd.partitions(i))) + // Check whether there are any hosts that match all RDDs; otherwise return the union + val exactMatchLocations = prefs.reduce((x, y) => x.intersect(y)) + val locs = if (!exactMatchLocations.isEmpty) exactMatchLocations else prefs.flatten.distinct + new ZippedPartitionsPartition(i, rdds, locs) } - array } override def getPreferredLocations(s: Partition): Seq[String] = { - val parts = s.asInstanceOf[ZippedPartitionsPartition].partitions - val prefs = rdds.zip(parts).map { case (rdd, p) => rdd.preferredLocations(p) } - // Check whether there are any hosts that match all RDDs; otherwise return the union - val exactMatchLocations = prefs.reduce((x, y) => x.intersect(y)) - if (!exactMatchLocations.isEmpty) { - exactMatchLocations - } else { - prefs.flatten.distinct - } + s.asInstanceOf[ZippedPartitionsPartition].preferredLocations } override def clearDependencies() { From 55edbb4209d86e6f695b5cd7c7826082174ba3ba Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 1 Dec 2013 20:17:26 -0800 Subject: [PATCH 245/531] Created an algorithms package and put all algorithms there. --- .../org/apache/spark/graph/Analytics.scala | 312 +----------------- .../algorithms/ConnectedComponents.scala | 40 +++ .../spark/graph/algorithms/PageRank.scala | 154 +++++++++ .../graph/algorithms/TriangleCount.scala | 74 +++++ .../apache/spark/graph/AnalyticsSuite.scala | 27 +- 5 files changed, 288 insertions(+), 319 deletions(-) create mode 100644 graph/src/main/scala/org/apache/spark/graph/algorithms/ConnectedComponents.scala create mode 100644 graph/src/main/scala/org/apache/spark/graph/algorithms/PageRank.scala create mode 100644 graph/src/main/scala/org/apache/spark/graph/algorithms/TriangleCount.scala diff --git a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala index db1bb8e2bec8c..e7451ee5e47aa 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala @@ -1,6 +1,8 @@ package org.apache.spark.graph import org.apache.spark._ +import org.apache.spark.graph.algorithms._ + /** * The Analytics object contains a collection of basic graph analytics @@ -12,272 +14,6 @@ import org.apache.spark._ */ object Analytics extends Logging { - /** - * Run PageRank for a fixed number of iterations returning a graph - * 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) - * - * @param graph the graph on which to compute PageRank - * @param numIter the number of iterations of PageRank to run - * @param resetProb the random reset probability (alpha) - * - * @return the graph containing with each vertex containing the - * PageRank and each edge containing the normalized weight. - * - */ - def pagerank[VD: Manifest, ED: Manifest]( - 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. - */ - val pagerankGraph: Graph[Double, Double] = graph - // Associate the degree with each vertex - .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 - .mapVertices( (id, attr) => 1.0 ) - - // Display statistics about pagerank - println(pagerankGraph.statistics) - - // Define the three functions needed to implement PageRank in the GraphX - // version of Pregel - def vertexProgram(id: Vid, attr: Double, msgSum: Double): Double = - resetProb + (1.0 - resetProb) * msgSum - def sendMessage(edge: EdgeTriplet[Double, Double]) = - Iterator((edge.dstId, edge.srcAttr * edge.attr)) - def messageCombiner(a: Double, b: Double): Double = a + b - // The initial message received by all vertices in PageRank - val initialMessage = 0.0 - - // Execute pregel for a fixed number of iterations. - Pregel(pagerankGraph, initialMessage, numIter)( - vertexProgram, sendMessage, messageCombiner) - } - - - /** - * 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) - * - * @param graph the graph on which to compute PageRank - * @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 containing the normalized weight. - */ - def deltaPagerank[VD: Manifest, ED: Manifest]( - graph: Graph[VD, ED], tol: Double, 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. - */ - val pagerankGraph: Graph[(Double, Double), Double] = graph - // Associate the degree with each vertex - .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 (initalPR, delta = 0) - .mapVertices( (id, attr) => (0.0, 0.0) ) - - // Display statistics about pagerank - println(pagerankGraph.statistics) - - // Define the three functions needed to implement PageRank in the GraphX - // version of Pregel - def vertexProgram(id: Vid, attr: (Double, Double), msgSum: Double): (Double, Double) = { - val (oldPR, lastDelta) = attr - val newPR = oldPR + (1.0 - resetProb) * msgSum - (newPR, newPR - oldPR) - } - def sendMessage(edge: EdgeTriplet[(Double, Double), Double]) = { - if (edge.srcAttr._2 > tol) { - Iterator((edge.dstId, edge.srcAttr._2 * edge.attr)) - } else { - Iterator.empty - } - } - def messageCombiner(a: Double, b: Double): Double = a + b - // The initial message received by all vertices in PageRank - val initialMessage = resetProb / (1.0 - resetProb) - - // Execute a dynamic version of Pregel. - Pregel(pagerankGraph, initialMessage)( - vertexProgram, sendMessage, messageCombiner) - .mapVertices( (vid, attr) => attr._1 ) - } // end of deltaPageRank - - - /** - * Compute the connected component membership of each vertex and - * return an RDD with the vertex value containing the lowest vertex - * id in the connected component containing that vertex. - * - * @tparam VD the vertex attribute type (discarded in the - * computation) - * @tparam ED the edge attribute type (preserved in the computation) - * - * @param graph the graph for which to compute the connected - * components - * - * @return a graph with vertex attributes containing the smallest - * vertex in each connected component - */ - def connectedComponents[VD: Manifest, ED: Manifest](graph: Graph[VD, ED]): - Graph[Vid, ED] = { - val ccGraph = graph.mapVertices { case (vid, _) => vid } - - def sendMessage(edge: EdgeTriplet[Vid, 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)( - (id, attr, msg) => math.min(attr, msg), - sendMessage, - (a,b) => math.min(a,b) - ) - } // end of connectedComponents - - - /** - * 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` - * @tparam VD - * @tparam ED - * @return - */ - def triangleCount[VD: ClassManifest, ED: ClassManifest](rawGraph: Graph[VD,ED]): - Graph[Int, ED] = { - // Remove redundant edges - val graph = rawGraph.groupEdges( (a,b) => a ).cache - - // Construct set representations of the neighborhoods - val nbrSets: VertexRDD[VertexSet] = - graph.collectNeighborIds(EdgeDirection.Both).mapValues { (vid, nbrs) => - val set = new VertexSet(4) - var i = 0 - while (i < nbrs.size) { - // prevent self cycle - if(nbrs(i) != vid) { - set.add(nbrs(i)) - } - i += 1 - } - set - } - // join the sets with the graph - val setGraph: Graph[VertexSet, ED] = graph.outerJoinVertices(nbrSets) { - (vid, _, optSet) => optSet.getOrElse(null) - } - // Edge function computes intersection of smaller vertex with larger vertex - def edgeFunc(et: EdgeTriplet[VertexSet, ED]): Iterator[(Vid, Int)] = { - assert(et.srcAttr != null) - assert(et.dstAttr != null) - val (smallSet, largeSet) = if (et.srcAttr.size < et.dstAttr.size) { - (et.srcAttr, et.dstAttr) - } else { - (et.dstAttr, et.srcAttr) - } - 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 } - } - Iterator((et.srcId, counter), (et.dstId, counter)) - } - // compute the intersection along edges - val counters: VertexRDD[Int] = setGraph.mapReduceTriplets(edgeFunc, _ + _) - // Merge counters with the graph and divide by two since each triangle is counted twice - graph.outerJoinVertices(counters) { - (vid, _, optCounter: Option[Int]) => - val dblCount = optCounter.getOrElse(0) - // double count should be even (divisible by two) - assert((dblCount & 1) == 0) - dblCount / 2 - } - - } // end of TriangleCount - - - - def main(args: Array[String]) = { val host = args(0) val taskType = args(1) @@ -361,8 +97,8 @@ object Analytics extends Logging { logInfo("GRAPHX: Number of edges " + graph.edges.count) //val pr = Analytics.pagerank(graph, numIter) - val pr = if(isDynamic) Analytics.deltaPagerank(graph, tol, numIter) - else Analytics.pagerank(graph, numIter) + val pr = if(isDynamic) PageRank.runUntillConvergence(graph, tol, numIter) + else PageRank.run(graph, numIter) logInfo("GRAPHX: Total rank: " + pr.vertices.map{ case (id,r) => r }.reduce(_+_) ) if (!outFname.isEmpty) { println("Saving pageranks of pages to " + outFname) @@ -405,7 +141,7 @@ object Analytics extends Logging { val sc = new SparkContext(host, "ConnectedComponents(" + fname + ")") val graph = GraphLoader.edgeListFile(sc, fname, minEdgePartitions = numEPart, partitionStrategy=partitionStrategy).cache() - val cc = Analytics.connectedComponents(graph) + val cc = ConnectedComponents.run(graph) println("Components: " + cc.vertices.map{ case (vid,data) => data}.distinct()) sc.stop() } @@ -427,7 +163,7 @@ object Analytics extends Logging { val sc = new SparkContext(host, "TriangleCount(" + fname + ")") val graph = GraphLoader.edgeListFile(sc, fname, canonicalOrientation = true, minEdgePartitions = numEPart, partitionStrategy=partitionStrategy).cache() - val triangles = Analytics.triangleCount(graph) + val triangles = TriangleCount.run(graph) println("Triangles: " + triangles.vertices.map { case (vid,data) => data.toLong }.reduce(_+_) / 3) @@ -536,42 +272,6 @@ object Analytics extends Logging { } } - // /** - // * Compute the PageRank of a graph returning the pagerank of each vertex as an RDD - // */ - // def dynamicPagerank[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], - // tol: Double, maxIter: Int = 10) = { - // // Compute the out degree of each vertex - // val pagerankGraph = graph.updateVertices[Int, (Int, Double, Double)](graph.outDegrees, - // (vertex, degIter) => (degIter.sum, 1.0, 1.0) - // ) - - // // Run PageRank - // GraphLab.iterateGAS(pagerankGraph)( - // (me_id, edge) => edge.src.data._2 / edge.src.data._1, // gather - // (a: Double, b: Double) => a + b, - // (vertex, a: Option[Double]) => - // (vertex.data._1, (0.15 + 0.85 * a.getOrElse(0.0)), vertex.data._2), // apply - // (me_id, edge) => math.abs(edge.src.data._2 - edge.dst.data._1) > tol, // scatter - // maxIter).mapVertices { case Vertex(vid, data) => Vertex(vid, data._2) } - // } - - // /** - // * Compute the connected component membership of each vertex - // * and return an RDD with the vertex value containing the - // * lowest vertex id in the connected component containing - // * that vertex. - // */ - // def connectedComponents[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], numIter: Int) = { - // val ccGraph = graph.mapVertices { case Vertex(vid, _) => Vertex(vid, vid) } - // GraphLab.iterateGA[Int, ED, Int](ccGraph)( - // (me_id, edge) => edge.otherVertex(me_id).data, // gather - // (a: Int, b: Int) => math.min(a, b), // merge - // (v, a: Option[Int]) => math.min(v.data, a.getOrElse(Integer.MAX_VALUE)), // apply - // numIter, - // gatherDirection = EdgeDirection.Both) - // } - // /** // * Compute the shortest path to a set of markers // */ diff --git a/graph/src/main/scala/org/apache/spark/graph/algorithms/ConnectedComponents.scala b/graph/src/main/scala/org/apache/spark/graph/algorithms/ConnectedComponents.scala new file mode 100644 index 0000000000000..50c9f15f413f4 --- /dev/null +++ b/graph/src/main/scala/org/apache/spark/graph/algorithms/ConnectedComponents.scala @@ -0,0 +1,40 @@ +package org.apache.spark.graph.algorithms + +import org.apache.spark.graph._ + + +object ConnectedComponents { + /** + * Compute the connected component membership of each vertex and + * return an RDD with the vertex value containing the lowest vertex + * id in the connected component containing that vertex. + * + * @tparam VD the vertex attribute type (discarded in the + * computation) + * @tparam ED the edge attribute type (preserved in the computation) + * + * @param graph the graph for which to compute the connected + * components + * + * @return a graph with vertex attributes containing the smallest + * vertex in each connected component + */ + def run[VD: Manifest, ED: Manifest](graph: Graph[VD, ED]): Graph[Vid, ED] = { + val ccGraph = graph.mapVertices { case (vid, _) => vid } + + def sendMessage(edge: EdgeTriplet[Vid, 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)( + vprog = (id, attr, msg) => math.min(attr, msg), + sendMsg = sendMessage, + mergeMsg = (a, b) => math.min(a, b)) + } // end of connectedComponents +} diff --git a/graph/src/main/scala/org/apache/spark/graph/algorithms/PageRank.scala b/graph/src/main/scala/org/apache/spark/graph/algorithms/PageRank.scala new file mode 100644 index 0000000000000..d190910c55cb5 --- /dev/null +++ b/graph/src/main/scala/org/apache/spark/graph/algorithms/PageRank.scala @@ -0,0 +1,154 @@ +package org.apache.spark.graph.algorithms + +import org.apache.spark.graph._ + + +object PageRank { + + /** + * Run PageRank for a fixed number of iterations returning a graph + * 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) + * + * @param graph the graph on which to compute PageRank + * @param numIter the number of iterations of PageRank to run + * @param resetProb the random reset probability (alpha) + * + * @return the graph containing with each vertex containing the PageRank and each edge + * containing the normalized weight. + * + */ + def run[VD: Manifest, ED: Manifest]( + 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. + */ + val pagerankGraph: Graph[Double, Double] = graph + // Associate the degree with each vertex + .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 + .mapVertices( (id, attr) => 1.0 ) + + // Display statistics about pagerank + println(pagerankGraph.statistics) + + // Define the three functions needed to implement PageRank in the GraphX + // version of Pregel + def vertexProgram(id: Vid, attr: Double, msgSum: Double): Double = + resetProb + (1.0 - resetProb) * msgSum + def sendMessage(edge: EdgeTriplet[Double, Double]) = + Iterator((edge.dstId, edge.srcAttr * edge.attr)) + def messageCombiner(a: Double, b: Double): Double = a + b + // The initial message received by all vertices in PageRank + val initialMessage = 0.0 + + // Execute pregel for a fixed number of iterations. + Pregel(pagerankGraph, initialMessage, numIter)( + vertexProgram, sendMessage, messageCombiner) + } + + /** + * 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) + * + * @param graph the graph on which to compute PageRank + * @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 + * containing the normalized weight. + */ + def runUntillConvergence[VD: Manifest, ED: Manifest]( + graph: Graph[VD, ED], tol: Double, 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. + val pagerankGraph: Graph[(Double, Double), Double] = graph + // Associate the degree with each vertex + .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 (initalPR, delta = 0) + .mapVertices( (id, attr) => (0.0, 0.0) ) + + // Display statistics about pagerank + println(pagerankGraph.statistics) + + // Define the three functions needed to implement PageRank in the GraphX + // version of Pregel + def vertexProgram(id: Vid, attr: (Double, Double), msgSum: Double): (Double, Double) = { + val (oldPR, lastDelta) = attr + val newPR = oldPR + (1.0 - resetProb) * msgSum + (newPR, newPR - oldPR) + } + + def sendMessage(edge: EdgeTriplet[(Double, Double), Double]) = { + if (edge.srcAttr._2 > tol) { + Iterator((edge.dstId, edge.srcAttr._2 * edge.attr)) + } else { + Iterator.empty + } + } + + def messageCombiner(a: Double, b: Double): Double = a + b + + // The initial message received by all vertices in PageRank + val initialMessage = resetProb / (1.0 - resetProb) + + // Execute a dynamic version of Pregel. + Pregel(pagerankGraph, initialMessage)(vertexProgram, sendMessage, messageCombiner) + .mapVertices((vid, attr) => attr._1) + } // end of deltaPageRank +} diff --git a/graph/src/main/scala/org/apache/spark/graph/algorithms/TriangleCount.scala b/graph/src/main/scala/org/apache/spark/graph/algorithms/TriangleCount.scala new file mode 100644 index 0000000000000..963986d20dcde --- /dev/null +++ b/graph/src/main/scala/org/apache/spark/graph/algorithms/TriangleCount.scala @@ -0,0 +1,74 @@ +package org.apache.spark.graph.algorithms + +import org.apache.spark.graph._ + + +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` + * @return + */ + def run[VD: ClassManifest, ED: ClassManifest](graph: Graph[VD,ED]): Graph[Int, ED] = { + // Remove redundant edges + val g = graph.groupEdges((a, b) => a).cache + + // Construct set representations of the neighborhoods + val nbrSets: VertexRDD[VertexSet] = + g.collectNeighborIds(EdgeDirection.Both).mapValues { (vid, nbrs) => + val set = new VertexSet(4) + var i = 0 + while (i < nbrs.size) { + // prevent self cycle + if(nbrs(i) != vid) { + set.add(nbrs(i)) + } + i += 1 + } + set + } + // join the sets with the graph + val setGraph: Graph[VertexSet, ED] = g.outerJoinVertices(nbrSets) { + (vid, _, optSet) => optSet.getOrElse(null) + } + // Edge function computes intersection of smaller vertex with larger vertex + def edgeFunc(et: EdgeTriplet[VertexSet, ED]): Iterator[(Vid, Int)] = { + assert(et.srcAttr != null) + assert(et.dstAttr != null) + val (smallSet, largeSet) = if (et.srcAttr.size < et.dstAttr.size) { + (et.srcAttr, et.dstAttr) + } else { + (et.dstAttr, et.srcAttr) + } + 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 } + } + Iterator((et.srcId, counter), (et.dstId, counter)) + } + // compute the intersection along edges + val counters: VertexRDD[Int] = setGraph.mapReduceTriplets(edgeFunc, _ + _) + // Merge counters with the graph and divide by two since each triangle is counted twice + g.outerJoinVertices(counters) { + (vid, _, optCounter: Option[Int]) => + val dblCount = optCounter.getOrElse(0) + // double count should be even (divisible by two) + assert((dblCount & 1) == 0) + dblCount / 2 + } + + } // end of TriangleCount + +} diff --git a/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala b/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala index b74835216aa4d..c9cb65eeda3fb 100644 --- a/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala @@ -4,6 +4,7 @@ import org.scalatest.FunSuite import org.apache.spark.SparkContext import org.apache.spark.SparkContext._ +import org.apache.spark.graph.algorithms._ import org.apache.spark.rdd._ import org.apache.spark.graph.LocalSparkContext._ @@ -56,8 +57,8 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { val nVertices = 100 val starGraph = GraphGenerators.starGraph(sc, nVertices).cache() val resetProb = 0.15 - val prGraph1 = Analytics.pagerank(starGraph, 1, resetProb) - val prGraph2 = Analytics.pagerank(starGraph, 2, resetProb) + val prGraph1 = PageRank.run(starGraph, 1, resetProb) + val prGraph2 = PageRank.run(starGraph, 2, resetProb) val notMatching = prGraph1.vertices.zipJoin(prGraph2.vertices) { (vid, pr1, pr2) => if (pr1 != pr2) { 1 } else { 0 } @@ -71,7 +72,7 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { } assert(errors.sum === 0) - val prGraph3 = Analytics.deltaPagerank(starGraph, 0, resetProb) + val prGraph3 = PageRank.runUntillConvergence(starGraph, 0, resetProb) val errors2 = prGraph2.vertices.leftJoin(prGraph3.vertices){ (vid, pr1, pr2Opt) => pr2Opt match { case Some(pr2) if(pr1 == pr2) => 0 @@ -88,8 +89,8 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { withSpark(new SparkContext("local", "test")) { sc => val gridGraph = GraphGenerators.gridGraph(sc, 10, 10).cache() val resetProb = 0.15 - val prGraph1 = Analytics.pagerank(gridGraph, 50, resetProb).cache() - val prGraph2 = Analytics.deltaPagerank(gridGraph, 0.0001, resetProb).cache() + val prGraph1 = PageRank.run(gridGraph, 50, resetProb).cache() + val prGraph2 = PageRank.runUntillConvergence(gridGraph, 0.0001, resetProb).cache() val error = prGraph1.vertices.zipJoin(prGraph2.vertices) { case (id, a, b) => (a - b) * (a - b) } .map { case (id, error) => error }.sum prGraph1.vertices.zipJoin(prGraph2.vertices) { (id, a, b) => (a, b, a-b) }.foreach(println(_)) @@ -110,7 +111,7 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { test("Grid Connected Components") { withSpark(new SparkContext("local", "test")) { sc => val gridGraph = GraphGenerators.gridGraph(sc, 10, 10).cache() - val ccGraph = Analytics.connectedComponents(gridGraph).cache() + val ccGraph = ConnectedComponents.run(gridGraph).cache() val maxCCid = ccGraph.vertices.map { case (vid, ccId) => ccId }.sum assert(maxCCid === 0) } @@ -120,7 +121,7 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { test("Reverse Grid Connected Components") { withSpark(new SparkContext("local", "test")) { sc => val gridGraph = GraphGenerators.gridGraph(sc, 10, 10).reverse.cache() - val ccGraph = Analytics.connectedComponents(gridGraph).cache() + val ccGraph = ConnectedComponents.run(gridGraph).cache() val maxCCid = ccGraph.vertices.map { case (vid, ccId) => ccId }.sum assert(maxCCid === 0) } @@ -133,7 +134,7 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { val chain2 = (10 until 20).map(x => (x, x+1) ) val rawEdges = sc.parallelize(chain1 ++ chain2, 3).map { case (s,d) => (s.toLong, d.toLong) } val twoChains = Graph.fromEdgeTuples(rawEdges, 1.0).cache() - val ccGraph = Analytics.connectedComponents(twoChains).cache() + val ccGraph = ConnectedComponents.run(twoChains).cache() val vertices = ccGraph.vertices.collect() for ( (id, cc) <- vertices ) { if(id < 10) { assert(cc === 0) } @@ -157,7 +158,7 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { val chain2 = (10 until 20).map(x => (x, x+1) ) val rawEdges = sc.parallelize(chain1 ++ chain2, 3).map { case (s,d) => (s.toLong, d.toLong) } val twoChains = Graph.fromEdgeTuples(rawEdges, true).reverse.cache() - val ccGraph = Analytics.connectedComponents(twoChains).cache() + val ccGraph = ConnectedComponents.run(twoChains).cache() val vertices = ccGraph.vertices.collect for ( (id, cc) <- vertices ) { if (id < 10) { @@ -182,7 +183,7 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { withSpark(new SparkContext("local", "test")) { sc => val rawEdges = sc.parallelize(Array( 0L->1L, 1L->2L, 2L->0L ), 2) val graph = Graph.fromEdgeTuples(rawEdges, true).cache() - val triangleCount = Analytics.triangleCount(graph) + val triangleCount = TriangleCount.run(graph) val verts = triangleCount.vertices verts.collect.foreach { case (vid, count) => assert(count === 1) } } @@ -194,7 +195,7 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { Array(0L -> -1L, -1L -> -2L, -2L -> 0L) val rawEdges = sc.parallelize(triangles, 2) val graph = Graph.fromEdgeTuples(rawEdges, true).cache() - val triangleCount = Analytics.triangleCount(graph) + val triangleCount = TriangleCount.run(graph) val verts = triangleCount.vertices verts.collect().foreach { case (vid, count) => if (vid == 0) { @@ -214,7 +215,7 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { val revTriangles = triangles.map { case (a,b) => (b,a) } val rawEdges = sc.parallelize(triangles ++ revTriangles, 2) val graph = Graph.fromEdgeTuples(rawEdges, true).cache() - val triangleCount = Analytics.triangleCount(graph) + val triangleCount = TriangleCount.run(graph) val verts = triangleCount.vertices verts.collect().foreach { case (vid, count) => if (vid == 0) { @@ -231,7 +232,7 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { val rawEdges = sc.parallelize(Array(0L -> 1L, 1L -> 2L, 2L -> 0L) ++ Array(0L -> 1L, 1L -> 2L, 2L -> 0L), 2) val graph = Graph.fromEdgeTuples(rawEdges, true).cache() - val triangleCount = Analytics.triangleCount(graph) + val triangleCount = TriangleCount.run(graph) val verts = triangleCount.vertices verts.collect.foreach { case (vid, count) => assert(count === 1) } } From 8701cb55e6590bd9069f09910a54968ba5941804 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 1 Dec 2013 21:55:50 -0800 Subject: [PATCH 246/531] Use specialized shuffler for aggregation. --- .../org/apache/spark/graph/Analytics.scala | 12 +++--- .../spark/graph/GraphKryoRegistrator.scala | 1 - .../org/apache/spark/graph/GraphLoader.scala | 3 -- .../scala/org/apache/spark/graph/Pregel.scala | 2 - .../org/apache/spark/graph/VertexRDD.scala | 10 ++--- .../algorithms/ConnectedComponents.scala | 15 +++---- .../graph/impl/EdgeTripletIterator.scala | 19 --------- .../apache/spark/graph/impl/GraphImpl.scala | 24 +++++------ .../spark/graph/impl/MessageToPartition.scala | 42 ++++++------------- .../apache/spark/graph/impl/Serializers.scala | 24 +++++------ .../spark/graph/impl/VTableReplicated.scala | 2 +- .../spark/graph/impl/VertexPlacement.scala | 5 +-- 12 files changed, 55 insertions(+), 104 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala index e7451ee5e47aa..755809b4b96e9 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala @@ -92,19 +92,21 @@ object Analytics extends Logging { minEdgePartitions = numEPart, partitionStrategy=partitionStrategy).cache() val startTime = System.currentTimeMillis - logInfo("GRAPHX: starting tasks") - logInfo("GRAPHX: Number of vertices " + graph.vertices.count) - logInfo("GRAPHX: Number of edges " + graph.edges.count) + println("GRAPHX: starting tasks") + println("GRAPHX: Number of vertices " + graph.vertices.count) + println("GRAPHX: Number of edges " + graph.edges.count) //val pr = Analytics.pagerank(graph, numIter) val pr = if(isDynamic) PageRank.runUntillConvergence(graph, tol, numIter) else PageRank.run(graph, numIter) - logInfo("GRAPHX: Total rank: " + pr.vertices.map{ case (id,r) => r }.reduce(_+_) ) + println("GRAPHX: Total rank: " + pr.vertices.map{ case (id,r) => r }.reduce(_+_) ) if (!outFname.isEmpty) { println("Saving pageranks of pages to " + outFname) pr.vertices.map{case (id, r) => id + "\t" + r}.saveAsTextFile(outFname) } - logInfo("GRAPHX: Runtime: " + ((System.currentTimeMillis - startTime)/1000.0) + " seconds") + println("GRAPHX: Runtime: " + ((System.currentTimeMillis - startTime)/1000.0) + " seconds") + + Thread.sleep(100000) sc.stop() } diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphKryoRegistrator.scala b/graph/src/main/scala/org/apache/spark/graph/GraphKryoRegistrator.scala index d04e82a0a333a..a02b33686beed 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphKryoRegistrator.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphKryoRegistrator.scala @@ -13,7 +13,6 @@ class GraphKryoRegistrator extends KryoRegistrator { kryo.register(classOf[Edge[Object]]) kryo.register(classOf[MessageToPartition[Object]]) kryo.register(classOf[VertexBroadcastMsg[Object]]) - kryo.register(classOf[AggregationMsg[Object]]) kryo.register(classOf[(Vid, Object)]) kryo.register(classOf[EdgePartition[Object]]) kryo.register(classOf[BitSet]) diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala b/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala index 7623b2b596156..29d14452de833 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala @@ -1,9 +1,6 @@ package org.apache.spark.graph -import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext -import org.apache.spark.SparkContext._ -import org.apache.spark.graph.impl.GraphImpl object GraphLoader { diff --git a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala index 55b3464b56989..5e3f71983ffe0 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala @@ -1,7 +1,5 @@ package org.apache.spark.graph -import org.apache.spark.rdd.RDD - /** * This object implements a Pregel-like bulk-synchronous diff --git a/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala b/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala index 5f9672e07b540..bdfa146710775 100644 --- a/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala @@ -21,10 +21,9 @@ import org.apache.spark._ import org.apache.spark.SparkContext._ import org.apache.spark.rdd._ import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.collection.{BitSet, OpenHashSet, PrimitiveKeyOpenHashMap} +import org.apache.spark.graph.impl.MsgRDDFunctions import org.apache.spark.graph.impl.VertexPartition -import org.apache.spark.util.ClosureCleaner /** @@ -269,11 +268,10 @@ class VertexRDD[@specialized VD: ClassManifest]( } } - def aggregateUsingIndex[VD2: ClassManifest, VidVDPair <: Product2[Vid, VD2] : ClassManifest]( - messages: RDD[VidVDPair], reduceFunc: (VD2, VD2) => VD2): VertexRDD[VD2] = + def aggregateUsingIndex[VD2: ClassManifest]( + messages: RDD[(Vid, VD2)], reduceFunc: (VD2, VD2) => VD2): VertexRDD[VD2] = { - // TODO: use specialized shuffle serializer. - val shuffled = new ShuffledRDD[Vid, VD2, VidVDPair](messages, this.partitioner.get) + val shuffled = MsgRDDFunctions.partitionForAggregation(messages, this.partitioner.get) val parts = partitionsRDD.zipPartitions(shuffled, true) { (thisIter, msgIter) => val vertextPartition: VertexPartition[VD] = thisIter.next() Iterator(vertextPartition.aggregateUsingIndex(msgIter, reduceFunc)) diff --git a/graph/src/main/scala/org/apache/spark/graph/algorithms/ConnectedComponents.scala b/graph/src/main/scala/org/apache/spark/graph/algorithms/ConnectedComponents.scala index 50c9f15f413f4..7cd947d2ba7df 100644 --- a/graph/src/main/scala/org/apache/spark/graph/algorithms/ConnectedComponents.scala +++ b/graph/src/main/scala/org/apache/spark/graph/algorithms/ConnectedComponents.scala @@ -5,19 +5,16 @@ import org.apache.spark.graph._ object ConnectedComponents { /** - * Compute the connected component membership of each vertex and - * return an RDD with the vertex value containing the lowest vertex - * id in the connected component containing that vertex. + * Compute the connected component membership of each vertex and return an RDD with the vertex + * value containing the lowest vertex id in the connected component containing that vertex. * - * @tparam VD the vertex attribute type (discarded in the - * computation) + * @tparam VD the vertex attribute type (discarded in the computation) * @tparam ED the edge attribute type (preserved in the computation) * - * @param graph the graph for which to compute the connected - * components + * @param graph the graph for which to compute the connected components * - * @return a graph with vertex attributes containing the smallest - * vertex in each connected component + * @return a graph with vertex attributes containing the smallest vertex in each + * connected component */ def run[VD: Manifest, ED: Manifest](graph: Graph[VD, ED]): Graph[Vid, ED] = { val ccGraph = graph.mapVertices { case (vid, _) => vid } diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/EdgeTripletIterator.scala b/graph/src/main/scala/org/apache/spark/graph/impl/EdgeTripletIterator.scala index f23f68cf22ca8..c9e1e081534cb 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/EdgeTripletIterator.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/EdgeTripletIterator.scala @@ -1,7 +1,5 @@ package org.apache.spark.graph.impl -import scala.collection.mutable - import org.apache.spark.graph._ import org.apache.spark.util.collection.PrimitiveKeyOpenHashMap @@ -40,21 +38,4 @@ class EdgeTripletIterator[VD: ClassManifest, ED: ClassManifest]( pos += 1 triplet } - - // TODO: Why do we need this? - override def toList: List[EdgeTriplet[VD, ED]] = { - val lb = new mutable.ListBuffer[EdgeTriplet[VD,ED]] - val currentEdge = new EdgeTriplet[VD, ED] - for (i <- 0 until edgePartition.size) { - currentEdge.srcId = edgePartition.srcIds(i) - // assert(vmap.containsKey(e.src.id)) - currentEdge.srcAttr = vmap(currentEdge.srcId) - currentEdge.dstId = edgePartition.dstIds(i) - // assert(vmap.containsKey(e.dst.id)) - currentEdge.dstAttr = vmap(currentEdge.dstId) - currentEdge.attr = edgePartition.data(i) - lb += currentEdge - } - lb.toList - } } diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index 04e03e0b1abac..9e9da3b80e4a9 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -1,7 +1,7 @@ package org.apache.spark.graph.impl -import org.apache.spark.SparkContext._ import org.apache.spark.{HashPartitioner, Partitioner} +import org.apache.spark.SparkContext._ import org.apache.spark.graph._ import org.apache.spark.graph.impl.GraphImpl._ import org.apache.spark.graph.impl.MsgRDDFunctions._ @@ -213,7 +213,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( // in the relevant position in an edge. val mapUsesSrcAttr = accessesVertexAttr[VD, ED](mapFunc, "srcAttr") val mapUsesDstAttr = accessesVertexAttr[VD, ED](mapFunc, "dstAttr") - val vs: RDD[(Pid, (VertexIdToIndexMap, Array[VD]))] = vTableReplicated.get(mapUsesSrcAttr, mapUsesDstAttr) + val vs = vTableReplicated.get(mapUsesSrcAttr, mapUsesDstAttr) // Map and combine. val preAgg = edges.zipEdgePartitions(vs) { (edgePartition, vTableReplicatedIter) => @@ -254,9 +254,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( } } // construct an iterator of tuples. Iterator[(Vid, A)] - msgBS.iterator.map { ind => - new AggregationMsg[A](vidToIndex.getValue(ind), msgArray(ind)) - } + msgBS.iterator.map { ind => (vidToIndex.getValue(ind), msgArray(ind)) } } // do the final reduction reusing the index map @@ -336,16 +334,16 @@ object GraphImpl { */ protected def createETable[ED: ClassManifest]( edges: RDD[Edge[ED]], - partitionStrategy: PartitionStrategy): EdgeRDD[ED] = { - // Get the number of partitions - val numPartitions = edges.partitions.size + partitionStrategy: PartitionStrategy): EdgeRDD[ED] = { + // Get the number of partitions + val numPartitions = edges.partitions.size - val eTable = edges.map { e => - val part: Pid = partitionStrategy.getPartition(e.srcId, e.dstId, numPartitions) + val eTable = edges.map { e => + val part: Pid = partitionStrategy.getPartition(e.srcId, e.dstId, numPartitions) - // Should we be using 3-tuple or an optimized class - new MessageToPartition(part, (e.srcId, e.dstId, e.attr)) - } + // Should we be using 3-tuple or an optimized class + new MessageToPartition(part, (e.srcId, e.dstId, e.attr)) + } .partitionBy(new HashPartitioner(numPartitions)) .mapPartitionsWithIndex( { (pid, iter) => val builder = new EdgePartitionBuilder[ED] diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/MessageToPartition.scala b/graph/src/main/scala/org/apache/spark/graph/impl/MessageToPartition.scala index d0a5adb85cd8a..c2e452cc72ab5 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/MessageToPartition.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/MessageToPartition.scala @@ -19,17 +19,6 @@ class VertexBroadcastMsg[@specialized(Int, Long, Double, Boolean) T]( } -class AggregationMsg[@specialized(Int, Long, Double, Boolean) T](var vid: Vid, var data: T) - extends Product2[Vid, T] { - - override def _1 = vid - - override def _2 = data - - override def canEqual(that: Any): Boolean = that.isInstanceOf[AggregationMsg[_]] -} - - /** * A message used to send a specific value to a partition. * @param partition index of the target partition. @@ -65,23 +54,6 @@ class VertexBroadcastMsgRDDFunctions[T: ClassManifest](self: RDD[VertexBroadcast } -class AggregationMessageRDDFunctions[T: ClassManifest](self: RDD[AggregationMsg[T]]) { - def partitionBy(partitioner: Partitioner): RDD[AggregationMsg[T]] = { - val rdd = new ShuffledRDD[Vid, T, AggregationMsg[T]](self, partitioner) - - // Set a custom serializer if the data is of int or double type. - if (classManifest[T] == ClassManifest.Int) { - rdd.setSerializer(classOf[IntAggMsgSerializer].getName) - } else if (classManifest[T] == ClassManifest.Long) { - rdd.setSerializer(classOf[LongAggMsgSerializer].getName) - } else if (classManifest[T] == ClassManifest.Double) { - rdd.setSerializer(classOf[DoubleAggMsgSerializer].getName) - } - rdd - } -} - - class MsgRDDFunctions[T: ClassManifest](self: RDD[MessageToPartition[T]]) { /** @@ -103,7 +75,17 @@ object MsgRDDFunctions { new VertexBroadcastMsgRDDFunctions(rdd) } - implicit def rdd2aggMessageRDDFunctions[T: ClassManifest](rdd: RDD[AggregationMsg[T]]) = { - new AggregationMessageRDDFunctions(rdd) + def partitionForAggregation[T: ClassManifest](msgs: RDD[(Vid, T)], partitioner: Partitioner) = { + val rdd = new ShuffledRDD[Vid, T, (Vid, T)](msgs, partitioner) + + // Set a custom serializer if the data is of int or double type. + if (classManifest[T] == ClassManifest.Int) { + rdd.setSerializer(classOf[IntAggMsgSerializer].getName) + } else if (classManifest[T] == ClassManifest.Long) { + rdd.setSerializer(classOf[LongAggMsgSerializer].getName) + } else if (classManifest[T] == ClassManifest.Double) { + rdd.setSerializer(classOf[DoubleAggMsgSerializer].getName) + } + rdd } } diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala b/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala index 9143820e13b4f..1cecb084e08bc 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala @@ -101,9 +101,9 @@ class IntAggMsgSerializer extends Serializer { override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { def writeObject[T](t: T) = { - val msg = t.asInstanceOf[AggregationMsg[Int]] - writeLong(msg.vid) - writeUnsignedVarInt(msg.data) + val msg = t.asInstanceOf[(Vid, Int)] + writeLong(msg._1) + writeUnsignedVarInt(msg._2) this } } @@ -112,7 +112,7 @@ class IntAggMsgSerializer extends Serializer { override def readObject[T](): T = { val a = readLong() val b = readUnsignedVarInt() - new AggregationMsg[Int](a, b).asInstanceOf[T] + (a, b).asInstanceOf[T] } } } @@ -124,9 +124,9 @@ class LongAggMsgSerializer extends Serializer { override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { def writeObject[T](t: T) = { - val msg = t.asInstanceOf[AggregationMsg[Long]] - writeVarLong(msg.vid, optimizePositive = false) - writeVarLong(msg.data, optimizePositive = true) + val msg = t.asInstanceOf[(Vid, Long)] + writeVarLong(msg._1, optimizePositive = false) + writeVarLong(msg._2, optimizePositive = true) this } } @@ -135,7 +135,7 @@ class LongAggMsgSerializer extends Serializer { override def readObject[T](): T = { val a = readVarLong(optimizePositive = false) val b = readVarLong(optimizePositive = true) - new AggregationMsg[Long](a, b).asInstanceOf[T] + (a, b).asInstanceOf[T] } } } @@ -148,9 +148,9 @@ class DoubleAggMsgSerializer extends Serializer { override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { def writeObject[T](t: T) = { - val msg = t.asInstanceOf[AggregationMsg[Double]] - writeVarLong(msg.vid, optimizePositive = false) - writeDouble(msg.data) + val msg = t.asInstanceOf[(Vid, Double)] + writeVarLong(msg._1, optimizePositive = false) + writeDouble(msg._2) this } } @@ -159,7 +159,7 @@ class DoubleAggMsgSerializer extends Serializer { def readObject[T](): T = { val a = readVarLong(optimizePositive = false) val b = readDouble() - new AggregationMsg[Double](a, b).asInstanceOf[T] + (a, b).asInstanceOf[T] } } } diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala b/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala index 148f016aeae93..c58dee8cfeef6 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala @@ -2,7 +2,7 @@ package org.apache.spark.graph.impl import org.apache.spark.SparkContext._ import org.apache.spark.rdd.RDD -import org.apache.spark.util.collection.{OpenHashSet, PrimitiveKeyOpenHashMap} +import org.apache.spark.util.collection.OpenHashSet import org.apache.spark.graph._ diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPlacement.scala b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPlacement.scala index b7aa9b4b49943..2f70e92da4f60 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPlacement.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPlacement.scala @@ -1,10 +1,9 @@ package org.apache.spark.graph.impl import org.apache.spark.SparkContext._ +import org.apache.spark.graph._ import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel - -import org.apache.spark.graph._ import org.apache.spark.util.collection.PrimitiveVector /** @@ -41,7 +40,7 @@ class VertexPlacement(eTable: EdgeRDD[_], vTable: VertexRDD[_]) { val numEdges = edgePartition.size val vSet = new VertexSet if (includeSrcAttr) { // Add src vertices to the set. - var i = 0 + var i = 0 while (i < numEdges) { vSet.add(edgePartition.srcIds(i)) i += 1 From c888b611daa95634f309ae72c6b062635a8259b1 Mon Sep 17 00:00:00 2001 From: Akihiro Matsukawa Date: Wed, 4 Dec 2013 16:48:32 -0800 Subject: [PATCH 247/531] add mask operation to Graph --- .../scala/org/apache/spark/graph/Graph.scala | 8 ++++ .../apache/spark/graph/impl/GraphImpl.scala | 36 ++++++++++++++ .../org/apache/spark/graph/GraphSuite.scala | 47 +++++++++++++++++++ 3 files changed, 91 insertions(+) diff --git a/graph/src/main/scala/org/apache/spark/graph/Graph.scala b/graph/src/main/scala/org/apache/spark/graph/Graph.scala index 42ede2d420a0b..144ca1b7883f9 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Graph.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Graph.scala @@ -191,6 +191,14 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { def subgraph(epred: EdgeTriplet[VD,ED] => Boolean = (x => true), vpred: (Vid, VD) => Boolean = ((v,d) => true) ): Graph[VD, ED] + /** + * Subgraph of this graph with only vertices and edges from the other graph. + * @param other the graph to project this graph onto + * @return a graph with vertices and edges that exists in both the current graph and other, + * with vertex and edge data from the current graph. + */ + def mask[VD2: ClassManifest, ED2: ClassManifest](other: Graph[VD2, ED2]): Graph[VD, ED] + /** * This function merges multiple edges between two vertices into a * single Edge. See diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index 337d560c2659b..96a36331f5f80 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -208,6 +208,9 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( new GraphImpl(newVTable, newETable, newVertexPlacement, partitioner) } // end of subgraph + override def mask[VD2: ClassManifest, ED2: ClassManifest] ( + other: Graph[VD2, ED2]) : Graph[VD, ED] = GraphImpl.mask(this, other) + override def groupEdges(merge: (ED, ED) => ED): Graph[VD, ED] = { ClosureCleaner.clean(merge) val newETable = @@ -347,6 +350,39 @@ object GraphImpl { } } + def mask[VD: ClassManifest, ED: ClassManifest, VD2: ClassManifest, ED2: ClassManifest] ( + thisGraph: Graph[VD, ED], otherGraph: Graph[VD2, ED2]) : Graph[VD, ED] = { + // basically vertices.join(other.vertices) + // written this way to take advantage of fast join in VertexSetRDDs + val newVTable = VertexSetRDD( + thisGraph.vertices.leftJoin(otherGraph.vertices)((vid, v, w) => if (w.isEmpty) None else Some(v)) + .filter{case (vid, opt) => !opt.isEmpty} + .map{case (vid, opt) => (vid, opt.get)} + ) + + // TODO(amatsukawa): safer way to downcast? case matching perhaps? + val thisImpl = thisGraph.asInstanceOf[GraphImpl[VD, ED]] + val otherImpl = otherGraph.asInstanceOf[GraphImpl[VD2, ED2]] + val newETable = thisImpl.eTable.zipPartitions(otherImpl.eTable) { + // extract two edge partitions, keep all edges in in this partition that is + // also in the other partition + (thisIter, otherIter) => + val (_, otherEPart) = otherIter.next() + val otherEdges = otherEPart.iterator.map(e => (e.srcId, e.dstId)).toSet + val (pid, thisEPart) = thisIter.next() + val newEPartBuilder = new EdgePartitionBuilder[ED] + thisEPart.foreach { e => + if (otherEdges.contains((e.srcId, e.dstId))) + newEPartBuilder.add(e.srcId, e.dstId, e.attr) + } + Iterator((pid, newEPartBuilder.toEdgePartition)) + }.partitionBy(thisImpl.eTable.partitioner.get) + + val newVertexPlacement = new VertexPlacement(newETable, newVTable) + new GraphImpl(newVTable, newETable, newVertexPlacement, thisImpl.partitioner) + } + + protected def mapTriplets[VD: ClassManifest, ED: ClassManifest, ED2: ClassManifest]( g: GraphImpl[VD, ED], f: EdgeTriplet[VD, ED] => ED2): Graph[VD, ED2] = { diff --git a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala index da7b2bdd997cd..e70773118f18f 100644 --- a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala @@ -116,6 +116,53 @@ class GraphSuite extends FunSuite with LocalSparkContext { } } + test("projectGraph") { + withSpark(new SparkContext("local", "test")) { sc => + val n = 5 + val vertices = sc.parallelize((0 to n).map(x => (x:Vid, x))) + val edges = sc.parallelize((1 to n).map(x => Edge(0, x, x))) + val graph: Graph[Int, Int] = Graph(vertices, edges) + + val subgraph = graph.subgraph( + e => e.dstId != 4L, + (vid, vdata) => vid != 3L + ).mapVertices((vid, vdata) => -1).mapEdges(e => -1) + + val projectedGraph = graph.mask(subgraph) + + val v = projectedGraph.vertices.collect().toSet + assert(v === Set((0,0), (1,1), (2,2), (4,4), (5,5))) + + // the map is necessary because of object-reuse in the edge iterator + val e = projectedGraph.edges.map(e => Edge(e.srcId, e.dstId, e.attr)).collect().toSet + assert(e === Set(Edge(0,1,1), Edge(0,2,2), Edge(0,5,5))) + + } + } + + test ("filterGraph") { + withSpark(new SparkContext("local", "test")) { sc => + val n = 5 + val vertices = sc.parallelize((0 to n).map(x => (x:Vid, x))) + val edges = sc.parallelize((1 to n).map(x => Edge(0, x, x))) + val graph: Graph[Int, Int] = Graph(vertices, edges) + val filteredGraph = graph.filter( + graph => { + val degrees: VertexSetRDD[Int] = graph.outDegrees + graph.outerJoinVertices(degrees) {(vid, data, deg) => deg.getOrElse(0)} + }, + vpred = (vid: Vid, deg:Int) => deg > 0 + ) + + val v = filteredGraph.vertices.collect().toSet + assert(v === Set((0,0))) + + // the map is necessary because of object-reuse in the edge iterator + val e = filteredGraph.edges.map(e => Edge(e.srcId, e.dstId, e.attr)).collect().toSet + assert(e.isEmpty) + } + } + test("VertexSetRDD") { withSpark(new SparkContext("local", "test")) { sc => val a = sc.parallelize((0 to 100).map(x => (x.toLong, x.toLong)), 5) From cb20175f97f2cf95af228d2bac8d21fa92122b2f Mon Sep 17 00:00:00 2001 From: Akihiro Matsukawa Date: Wed, 4 Dec 2013 16:48:51 -0800 Subject: [PATCH 248/531] add filter operation to GraphOps --- .../org/apache/spark/graph/GraphOps.scala | 31 +++++++++++++++++++ 1 file changed, 31 insertions(+) diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala b/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala index b7e28186c6527..a49eddc1dfb7b 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala @@ -244,4 +244,35 @@ class GraphOps[VD: ClassManifest, ED: ClassManifest](graph: Graph[VD, ED]) { graph.outerJoinVertices(table)(uf) } + /** + * 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 + * @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 + * + * @example This function can be used to filter the graph based on some property, without + * changing the vertex and edge values in your program. For example, we could remove the vertices + * in a graph with 0 outdegree + * + * {{{ + * graph.filter( + * graph => { + * val degrees: VertexSetRDD[Int] = graph.outDegrees + * graph.outerJoinVertices(degrees) {(vid, data, deg) => deg.getOrElse(0)} + * }, + * vpred = (vid: Vid, deg:Int) => deg > 0 + * ) + * }}} + * + */ + def filter[VD2: ClassManifest, ED2: ClassManifest]( + preprocess: Graph[VD, ED] => Graph[VD2, ED2], + epred: (EdgeTriplet[VD2, ED2]) => Boolean = (x: EdgeTriplet[VD2, ED2]) => true, + vpred: (Vid, VD2) => Boolean = (v:Vid, d:VD2) => true): Graph[VD, ED] = { + graph.mask(preprocess(graph).subgraph(epred, vpred)) + } } // end of GraphOps From 92e96f727ee944e7ed17b84aa8a57106907a9881 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Wed, 4 Dec 2013 17:29:52 -0800 Subject: [PATCH 249/531] Fix compile errors in GraphSuite and SerializerSuite --- .../org/apache/spark/graph/GraphSuite.scala | 2 +- .../apache/spark/graph/SerializerSuite.scala | 46 +++++++------------ 2 files changed, 17 insertions(+), 31 deletions(-) diff --git a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala index cf8884bd047cf..1d23911ff90e3 100644 --- a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala @@ -124,7 +124,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { val b = VertexRDD(a).mapValues(x => -x).cache() // Allow joining b with a derived RDD of b assert(b.count === 101) assert(b.leftJoin(a){ (id, a, bOpt) => a + bOpt.get }.map(x=> x._2).reduce(_+_) === 0) - val c = b.aggregateUsingIndex[Long, (Long, Long)](a, (x, y) => x) + val c = b.aggregateUsingIndex[Long](a, (x, y) => x) assert(b.leftJoin(c){ (id, b, cOpt) => b + cOpt.get }.map(x=> x._2).reduce(_+_) === 0) val d = c.filter(q => ((q._2 % 2) == 0)) val e = a.filter(q => ((q._2 % 2) == 0)) diff --git a/graph/src/test/scala/org/apache/spark/graph/SerializerSuite.scala b/graph/src/test/scala/org/apache/spark/graph/SerializerSuite.scala index 6295f866b8c48..dde6a021ef8de 100644 --- a/graph/src/test/scala/org/apache/spark/graph/SerializerSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/SerializerSuite.scala @@ -79,7 +79,7 @@ class SerializerSuite extends FunSuite with LocalSparkContext { } test("IntAggMsgSerializer") { - val outMsg = new AggregationMsg[Int](4, 5) + val outMsg = (4, 5) val bout = new ByteArrayOutputStream val outStrm = new IntAggMsgSerializer().newInstance().serializeStream(bout) outStrm.writeObject(outMsg) @@ -87,12 +87,10 @@ class SerializerSuite extends FunSuite with LocalSparkContext { bout.flush() val bin = new ByteArrayInputStream(bout.toByteArray) val inStrm = new IntAggMsgSerializer().newInstance().deserializeStream(bin) - val inMsg1: AggregationMsg[Int] = inStrm.readObject() - val inMsg2: AggregationMsg[Int] = inStrm.readObject() - assert(outMsg.vid === inMsg1.vid) - assert(outMsg.vid === inMsg2.vid) - assert(outMsg.data === inMsg1.data) - assert(outMsg.data === inMsg2.data) + val inMsg1: (Vid, Int) = inStrm.readObject() + val inMsg2: (Vid, Int) = inStrm.readObject() + assert(outMsg === inMsg1) + assert(outMsg === inMsg2) intercept[EOFException] { inStrm.readObject() @@ -100,7 +98,7 @@ class SerializerSuite extends FunSuite with LocalSparkContext { } test("LongAggMsgSerializer") { - val outMsg = new AggregationMsg[Long](4, 1L << 32) + val outMsg = (4, 1L << 32) val bout = new ByteArrayOutputStream val outStrm = new LongAggMsgSerializer().newInstance().serializeStream(bout) outStrm.writeObject(outMsg) @@ -108,12 +106,10 @@ class SerializerSuite extends FunSuite with LocalSparkContext { bout.flush() val bin = new ByteArrayInputStream(bout.toByteArray) val inStrm = new LongAggMsgSerializer().newInstance().deserializeStream(bin) - val inMsg1: AggregationMsg[Long] = inStrm.readObject() - val inMsg2: AggregationMsg[Long] = inStrm.readObject() - assert(outMsg.vid === inMsg1.vid) - assert(outMsg.vid === inMsg2.vid) - assert(outMsg.data === inMsg1.data) - assert(outMsg.data === inMsg2.data) + val inMsg1: (Vid, Long) = inStrm.readObject() + val inMsg2: (Vid, Long) = inStrm.readObject() + assert(outMsg === inMsg1) + assert(outMsg === inMsg2) intercept[EOFException] { inStrm.readObject() @@ -121,7 +117,7 @@ class SerializerSuite extends FunSuite with LocalSparkContext { } test("DoubleAggMsgSerializer") { - val outMsg = new AggregationMsg[Double](4, 5.0) + val outMsg = (4, 5.0) val bout = new ByteArrayOutputStream val outStrm = new DoubleAggMsgSerializer().newInstance().serializeStream(bout) outStrm.writeObject(outMsg) @@ -129,12 +125,10 @@ class SerializerSuite extends FunSuite with LocalSparkContext { bout.flush() val bin = new ByteArrayInputStream(bout.toByteArray) val inStrm = new DoubleAggMsgSerializer().newInstance().deserializeStream(bin) - val inMsg1: AggregationMsg[Double] = inStrm.readObject() - val inMsg2: AggregationMsg[Double] = inStrm.readObject() - assert(outMsg.vid === inMsg1.vid) - assert(outMsg.vid === inMsg2.vid) - assert(outMsg.data === inMsg1.data) - assert(outMsg.data === inMsg2.data) + val inMsg1: (Vid, Double) = inStrm.readObject() + val inMsg2: (Vid, Double) = inStrm.readObject() + assert(outMsg === inMsg1) + assert(outMsg === inMsg2) intercept[EOFException] { inStrm.readObject() @@ -149,12 +143,4 @@ class SerializerSuite extends FunSuite with LocalSparkContext { bmsgs.partitionBy(new HashPartitioner(3)).collect() } } - - test("TestShuffleAggregationMsg") { - withSpark(new SparkContext("local[2]", "test")) { sc => - val bmsgs = sc.parallelize(0 until 100, 10).map(pid => new AggregationMsg[Int](pid, pid)) - bmsgs.partitionBy(new HashPartitioner(3)).collect() - } - } - -} \ No newline at end of file +} From 2e583d2de4e6740fb99111d02e9f5ba293e9b7e7 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Wed, 4 Dec 2013 17:34:14 -0800 Subject: [PATCH 250/531] Declare Vids explicitly to avoid ClassCastException --- .../test/scala/org/apache/spark/graph/SerializerSuite.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/graph/src/test/scala/org/apache/spark/graph/SerializerSuite.scala b/graph/src/test/scala/org/apache/spark/graph/SerializerSuite.scala index dde6a021ef8de..0ec1a2a4d5487 100644 --- a/graph/src/test/scala/org/apache/spark/graph/SerializerSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/SerializerSuite.scala @@ -79,7 +79,7 @@ class SerializerSuite extends FunSuite with LocalSparkContext { } test("IntAggMsgSerializer") { - val outMsg = (4, 5) + val outMsg = (4: Vid, 5) val bout = new ByteArrayOutputStream val outStrm = new IntAggMsgSerializer().newInstance().serializeStream(bout) outStrm.writeObject(outMsg) @@ -98,7 +98,7 @@ class SerializerSuite extends FunSuite with LocalSparkContext { } test("LongAggMsgSerializer") { - val outMsg = (4, 1L << 32) + val outMsg = (4: Vid, 1L << 32) val bout = new ByteArrayOutputStream val outStrm = new LongAggMsgSerializer().newInstance().serializeStream(bout) outStrm.writeObject(outMsg) @@ -117,7 +117,7 @@ class SerializerSuite extends FunSuite with LocalSparkContext { } test("DoubleAggMsgSerializer") { - val outMsg = (4, 5.0) + val outMsg = (4: Vid, 5.0) val bout = new ByteArrayOutputStream val outStrm = new DoubleAggMsgSerializer().newInstance().serializeStream(bout) outStrm.writeObject(outMsg) From 38c6f5f6122ba32e1ef5d8b8a48ec99e6446d7e1 Mon Sep 17 00:00:00 2001 From: Akihiro Matsukawa Date: Wed, 4 Dec 2013 18:17:14 -0800 Subject: [PATCH 251/531] add a predicate to GraphLab to indicate active vertices at start --- graph/src/main/scala/org/apache/spark/graph/GraphLab.scala | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala b/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala index bf1f4168da54b..799c0fc901dc0 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala @@ -24,6 +24,8 @@ object GraphLab { * @param scatterFunc Executed after the apply function the scatter function takes * a triplet and signals whether the neighboring vertex program * must be recomputed. + * @param startVertices predicate to determine which vertices to start the computation on. + * these will be the active vertices in the first iteration. * @param numIter The maximum number of iterations to run. * @param gatherDirection The direction of edges to consider during the gather phase * @param scatterDirection The direction of edges to consider during the scatter phase @@ -40,12 +42,13 @@ object GraphLab { (gatherFunc: (Vid, EdgeTriplet[VD, ED]) => A, mergeFunc: (A, A) => A, applyFunc: (Vid, VD, Option[A]) => VD, - scatterFunc: (Vid, EdgeTriplet[VD, ED]) => Boolean): Graph[VD, ED] = { + scatterFunc: (Vid, EdgeTriplet[VD, ED]) => Boolean, + startVertices: (Vid, VD) => Boolean = (vid: Vid, data: VD) => true): Graph[VD, ED] = { // Add an active attribute to all vertices to track convergence. var activeGraph: Graph[(Boolean, VD), ED] = graph.mapVertices { - case (id, data) => (true, data) + case (id, data) => (startVertices(id, data), data) }.cache() // The gather function wrapper strips the active attribute and From c49a6a7954e9e1abf2377c5e6121819aa27f6e68 Mon Sep 17 00:00:00 2001 From: Akihiro Matsukawa Date: Wed, 4 Dec 2013 18:18:28 -0800 Subject: [PATCH 252/531] vertex should be inactive if no vertices scatter to it --- graph/src/main/scala/org/apache/spark/graph/GraphLab.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala b/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala index 799c0fc901dc0..aa35f9a746444 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala @@ -89,9 +89,9 @@ object GraphLab { } // Used to set the active status of vertices for the next round - def applyActive(vid: Vid, data: (Boolean, VD), newActive: Boolean): (Boolean, VD) = { + def applyActive(vid: Vid, data: (Boolean, VD), newActiveOpt: Option[Boolean]): (Boolean, VD) = { val (prevActive, vData) = data - (newActive, vData) + (newActiveOpt.getOrElse(false), vData) } // Main Loop --------------------------------------------------------------------- @@ -113,7 +113,7 @@ object GraphLab { val scattered: RDD[(Vid, Boolean)] = activeGraph.aggregateNeighbors(scatter, _ || _, scatterDirection.reverse) - activeGraph = activeGraph.joinVertices(scattered)(applyActive).cache() + activeGraph = activeGraph.outerJoinVertices(scattered)(applyActive).cache() // Calculate the number of active vertices numActive = activeGraph.vertices.map{ From a3bb98b88a3a65fe89ec45b8eb4015c4f196d44f Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Wed, 4 Dec 2013 18:20:10 -0800 Subject: [PATCH 253/531] Always write Vids using variable encoding Also, autoformat Serializers.scala. --- .../org/apache/spark/graph/impl/Serializers.scala | 14 ++++---------- 1 file changed, 4 insertions(+), 10 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala b/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala index 1cecb084e08bc..d726d2352d359 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala @@ -6,7 +6,6 @@ import java.nio.ByteBuffer import org.apache.spark.graph._ import org.apache.spark.serializer._ - class VidMsgSerializer extends Serializer { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { @@ -26,7 +25,6 @@ class VidMsgSerializer extends Serializer { } } - /** A special shuffle serializer for VertexBroadcastMessage[Int]. */ class IntVertexBroadcastMsgSerializer extends Serializer { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { @@ -34,7 +32,7 @@ class IntVertexBroadcastMsgSerializer extends Serializer { override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { def writeObject[T](t: T) = { val msg = t.asInstanceOf[VertexBroadcastMsg[Int]] - writeLong(msg.vid) + writeVarLong(msg._1, optimizePositive = false) writeInt(msg.data) this } @@ -55,7 +53,7 @@ class LongVertexBroadcastMsgSerializer extends Serializer { override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { def writeObject[T](t: T) = { val msg = t.asInstanceOf[VertexBroadcastMsg[Long]] - writeLong(msg.vid) + writeVarLong(msg._1, optimizePositive = false) writeLong(msg.data) this } @@ -78,7 +76,7 @@ class DoubleVertexBroadcastMsgSerializer extends Serializer { override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { def writeObject[T](t: T) = { val msg = t.asInstanceOf[VertexBroadcastMsg[Double]] - writeLong(msg.vid) + writeVarLong(msg._1, optimizePositive = false) writeDouble(msg.data) this } @@ -94,7 +92,6 @@ class DoubleVertexBroadcastMsgSerializer extends Serializer { } } - /** A special shuffle serializer for AggregationMessage[Int]. */ class IntAggMsgSerializer extends Serializer { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { @@ -102,7 +99,7 @@ class IntAggMsgSerializer extends Serializer { override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { def writeObject[T](t: T) = { val msg = t.asInstanceOf[(Vid, Int)] - writeLong(msg._1) + writeVarLong(msg._1, optimizePositive = false) writeUnsignedVarInt(msg._2) this } @@ -141,7 +138,6 @@ class LongAggMsgSerializer extends Serializer { } } - /** A special shuffle serializer for AggregationMessage[Double]. */ class DoubleAggMsgSerializer extends Serializer { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { @@ -282,7 +278,6 @@ sealed abstract class ShuffleSerializationStream(s: OutputStream) extends Serial override def close(): Unit = s.close() } - sealed abstract class ShuffleDeserializationStream(s: InputStream) extends DeserializationStream { // The implementation should override this one. def readObject[T](): T @@ -350,7 +345,6 @@ sealed abstract class ShuffleDeserializationStream(s: InputStream) extends Deser override def close(): Unit = s.close() } - sealed trait ShuffleSerializerInstance extends SerializerInstance { override def serialize[T](t: T): ByteBuffer = throw new UnsupportedOperationException From 6a7b396e5dbf06d4be277692b132b7f8a58c3b93 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Thu, 5 Dec 2013 12:35:03 -0800 Subject: [PATCH 254/531] Finish work on #85 --- .../apache/spark/graph/impl/Serializers.scala | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala b/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala index d726d2352d359..9e393d624b0b7 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala @@ -32,7 +32,7 @@ class IntVertexBroadcastMsgSerializer extends Serializer { override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { def writeObject[T](t: T) = { val msg = t.asInstanceOf[VertexBroadcastMsg[Int]] - writeVarLong(msg._1, optimizePositive = false) + writeVarLong(msg.vid, optimizePositive = false) writeInt(msg.data) this } @@ -40,7 +40,9 @@ class IntVertexBroadcastMsgSerializer extends Serializer { override def deserializeStream(s: InputStream) = new ShuffleDeserializationStream(s) { override def readObject[T](): T = { - new VertexBroadcastMsg[Int](0, readLong(), readInt()).asInstanceOf[T] + val a = readVarLong(optimizePositive = false) + val b = readInt() + new VertexBroadcastMsg[Int](0, a, b).asInstanceOf[T] } } } @@ -53,7 +55,7 @@ class LongVertexBroadcastMsgSerializer extends Serializer { override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { def writeObject[T](t: T) = { val msg = t.asInstanceOf[VertexBroadcastMsg[Long]] - writeVarLong(msg._1, optimizePositive = false) + writeVarLong(msg.vid, optimizePositive = false) writeLong(msg.data) this } @@ -61,7 +63,7 @@ class LongVertexBroadcastMsgSerializer extends Serializer { override def deserializeStream(s: InputStream) = new ShuffleDeserializationStream(s) { override def readObject[T](): T = { - val a = readLong() + val a = readVarLong(optimizePositive = false) val b = readLong() new VertexBroadcastMsg[Long](0, a, b).asInstanceOf[T] } @@ -76,7 +78,7 @@ class DoubleVertexBroadcastMsgSerializer extends Serializer { override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { def writeObject[T](t: T) = { val msg = t.asInstanceOf[VertexBroadcastMsg[Double]] - writeVarLong(msg._1, optimizePositive = false) + writeVarLong(msg.vid, optimizePositive = false) writeDouble(msg.data) this } @@ -84,7 +86,7 @@ class DoubleVertexBroadcastMsgSerializer extends Serializer { override def deserializeStream(s: InputStream) = new ShuffleDeserializationStream(s) { def readObject[T](): T = { - val a = readLong() + val a = readVarLong(optimizePositive = false) val b = readDouble() new VertexBroadcastMsg[Double](0, a, b).asInstanceOf[T] } @@ -107,7 +109,7 @@ class IntAggMsgSerializer extends Serializer { override def deserializeStream(s: InputStream) = new ShuffleDeserializationStream(s) { override def readObject[T](): T = { - val a = readLong() + val a = readVarLong(optimizePositive = false) val b = readUnsignedVarInt() (a, b).asInstanceOf[T] } From 3fc4534d19a7abf7dcb6c9ebe17e5e29f1febbc5 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 5 Dec 2013 14:55:26 -0800 Subject: [PATCH 255/531] wip delta join. --- .../apache/spark/util/collection/BitSet.scala | 4 + .../scala/org/apache/spark/graph/Graph.scala | 3 + .../scala/org/apache/spark/graph/Pregel.scala | 20 ++- .../org/apache/spark/graph/VertexRDD.scala | 7 + .../apache/spark/graph/impl/GraphImpl.scala | 14 +- .../spark/graph/impl/VTableReplicated.scala | 125 +++++++++++------- .../spark/graph/impl/VertexPartition.scala | 32 ++++- project/SparkBuild.scala | 3 +- 8 files changed, 150 insertions(+), 58 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala b/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala index 1f794379f74f3..0e12779152bda 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala @@ -102,6 +102,10 @@ class BitSet(numBits: Int) { words(index >> 6) |= bitmask // div by 64 and mask } + def unset(index: Int) { + val bitmask = 1L << (index & 0x3f) // mod 64 and shift + words(index >> 6) &= ~bitmask // div by 64 and mask + } /** * Return the value of the bit with the specified index. The value is true if the bit with diff --git a/graph/src/main/scala/org/apache/spark/graph/Graph.scala b/graph/src/main/scala/org/apache/spark/graph/Graph.scala index d81275c5ba042..b80c4e32adebd 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Graph.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Graph.scala @@ -278,6 +278,9 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { (mapFunc: (Vid, VD, Option[U]) => VD2) : Graph[VD2, ED] + def deltaJoin[VD2: ClassManifest] + (updates: VertexRDD[VD2])(updateF: (Vid, VD, VD2) => VD): Graph[VD, ED] + // 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) diff --git a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala index 5e3f71983ffe0..447a97b1ad51c 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala @@ -96,17 +96,29 @@ object Pregel { : Graph[VD, ED] = { // Receive the first set of messages - var g = graph.mapVertices( (vid, vdata) => vprog(vid, vdata, initialMsg)).cache + var g = graph.mapVertices( (vid, vdata) => vprog(vid, vdata, initialMsg)).cache() var i = 0 - while (i < numIter) { + while (i < numIter - 1) { // compute the messages - val messages = g.mapReduceTriplets(sendMsg, mergeMsg) + val messages = g.mapReduceTriplets(sendMsg, mergeMsg) // broadcast & aggregation // receive the messages - g = g.joinVertices(messages)(vprog).cache + g = g.deltaJoin(messages)(vprog).cache() // updating the graph // count the iteration i += 1 } + + // compute the messages + val messages = g.mapReduceTriplets(sendMsg, mergeMsg) + // receive the messages + g = g.outerJoinVertices(messages) { (vid, vd, msgOption) => + if (msgOption.isDefined) { + vprog(vid, vd, msgOption.get) + } else { + vd + } + }.cache() + // Return the final graph g } // end of apply diff --git a/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala b/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala index bdfa146710775..a6f52a6e22542 100644 --- a/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala @@ -268,6 +268,13 @@ class VertexRDD[@specialized VD: ClassManifest]( } } + def deltaJoin[VD2: ClassManifest](other: VertexRDD[VD2])(f: (Vid, VD, VD2) => VD): VertexRDD[VD] = + { + this.zipVertexPartitions(other) { (thisPart, otherPart) => + thisPart.deltaJoin(otherPart)(f) + } + } + def aggregateUsingIndex[VD2: ClassManifest]( messages: RDD[(Vid, VD2)], reduceFunc: (VD2, VD2) => VD2): VertexRDD[VD2] = { diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index 9e9da3b80e4a9..73018afa4c3b9 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -27,7 +27,8 @@ import org.apache.spark.util.collection.{BitSet, OpenHashSet, PrimitiveKeyOpenHa class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( @transient val vertices: VertexRDD[VD], @transient val edges: EdgeRDD[ED], - @transient val vertexPlacement: VertexPlacement) + @transient val vertexPlacement: VertexPlacement, + @transient val prevVTableReplicated: Option[VTableReplicated[VD]] = None) extends Graph[VD, ED] { //def this() = this(null, null, null, null) @@ -39,8 +40,8 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( this(new VertexRDD(vertices), new EdgeRDD(edges), vertexPlacement) } - @transient val vTableReplicated: VTableReplicated[VD] = - new VTableReplicated(vertices, edges, vertexPlacement) + @transient private val vTableReplicated: VTableReplicated[VD] = + new VTableReplicated(vertices, edges, vertexPlacement, prevVTableReplicated) /** Return a RDD of edges. */ // @transient override val edges: RDD[Edge[ED]] = @@ -267,6 +268,13 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( val newVTable = vertices.leftJoin(updates)(updateF) new GraphImpl(newVTable, edges, vertexPlacement) } + + override def deltaJoin[VD2: ClassManifest] + (updates: VertexRDD[VD2])(updateF: (Vid, VD, VD2) => VD): Graph[VD, ED] = + { + val newVTable = vertices.deltaJoin(updates)(updateF) + new GraphImpl(newVTable, edges, vertexPlacement, Some(vTableReplicated)) + } } // end of class GraphImpl diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala b/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala index c58dee8cfeef6..be7cf516ea5cf 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala @@ -13,7 +13,8 @@ private[impl] class VTableReplicated[VD: ClassManifest]( vTable: VertexRDD[VD], eTable: EdgeRDD[_], - vertexPlacement: VertexPlacement) { + vertexPlacement: VertexPlacement, + prevVTableReplicated: Option[VTableReplicated[VD]] = None) { val bothAttrs: RDD[(Pid, (VertexIdToIndexMap, Array[VD]))] = createVTableReplicated(vTable, eTable, vertexPlacement, true, true) @@ -36,7 +37,7 @@ class VTableReplicated[VD: ClassManifest]( } } - private def createVTableReplicated[VD: ClassManifest]( + private def createVTableReplicated( vTable: VertexRDD[VD], eTable: EdgeRDD[_], vertexPlacement: VertexPlacement, @@ -44,58 +45,90 @@ class VTableReplicated[VD: ClassManifest]( includeDstAttr: Boolean): RDD[(Pid, (VertexIdToIndexMap, Array[VD]))] = { val placement = vertexPlacement.get(includeSrcAttr, includeDstAttr) + val vdManifest = classManifest[VD] // Send each edge partition the vertex attributes it wants, as specified in // vertexPlacement - val msgsByPartition = placement.zipPartitions(vTable.partitionsRDD) { - (pid2vidIter, vertexPartIter) => - val pid2vid: Array[Array[Vid]] = pid2vidIter.next() - val vertexPart: VertexPartition[VD] = vertexPartIter.next() - val output = new Array[(Pid, VertexAttributeBlock[VD])](pid2vid.size) - for (pid <- 0 until pid2vid.size) { - val block = new VertexAttributeBlock( - pid2vid(pid), pid2vid(pid).map(vid => vertexPart(vid))) - output(pid) = (pid, block) - } - output.iterator - }.partitionBy(eTable.partitioner.get).cache() + val msgsByPartition = placement.zipPartitions(vTable.partitionsRDD)(VTableReplicated.buildBuffer(_, _)(vdManifest)) + .partitionBy(eTable.partitioner.get).cache() // TODO: Consider using a specialized shuffler. - // Within each edge partition, create a local map from vid to an index into - // the attribute array. Each map contains a superset of the vertices that it - // will receive, because it stores vids from both the source and destination - // of edges. It must always include both source and destination vids because - // some operations, such as GraphImpl.mapReduceTriplets, rely on this. - val localVidMap = eTable.partitionsRDD.mapPartitions(_.map { - case (pid, epart) => - val vidToIndex = new VertexIdToIndexMap - epart.foreach { e => - vidToIndex.add(e.srcId) - vidToIndex.add(e.dstId) - } - (pid, vidToIndex) - }, preservesPartitioning = true).cache() - - // Within each edge partition, place the vertex attributes received from - // msgsByPartition into the correct locations specified in localVidMap - localVidMap.zipPartitions(msgsByPartition) { - (mapIter, msgsIter) => - val (pid, vidToIndex) = mapIter.next() - assert(!mapIter.hasNext) - // Populate the vertex array using the vidToIndex map - val vertexArray = new Array[VD](vidToIndex.capacity) - for ((_, block) <- msgsIter) { - for (i <- 0 until block.vids.size) { - val vid = block.vids(i) - val attr = block.attrs(i) - val ind = vidToIndex.getPos(vid) & OpenHashSet.POSITION_MASK - vertexArray(ind) = attr + prevVTableReplicated match { + case Some(vTableReplicated) => + val prev: RDD[(Pid, (VertexIdToIndexMap, Array[VD]))] = + vTableReplicated.get(includeSrcAttr, includeDstAttr) + + prev.zipPartitions(msgsByPartition) { (vTableIter, msgsIter) => + val (pid, (vidToIndex, oldVertexArray)) = vTableIter.next() + + val vertexArray = vdManifest.newArray(oldVertexArray.length) + System.arraycopy(oldVertexArray, 0, vertexArray, 0, vertexArray.length) + + for ((_, block) <- msgsIter) { + for (i <- 0 until block.vids.size) { + val vid = block.vids(i) + val attr = block.attrs(i) + val ind = vidToIndex.getPos(vid) & OpenHashSet.POSITION_MASK + vertexArray(ind) = attr + } + } + + Iterator((pid, (vidToIndex, vertexArray))) + }.cache() + + case None => + // Within each edge partition, create a local map from vid to an index into + // the attribute array. Each map contains a superset of the vertices that it + // will receive, because it stores vids from both the source and destination + // of edges. It must always include both source and destination vids because + // some operations, such as GraphImpl.mapReduceTriplets, rely on this. + val localVidMap = eTable.partitionsRDD.mapPartitions(_.map { + case (pid, epart) => + val vidToIndex = new VertexIdToIndexMap + epart.foreach { e => + vidToIndex.add(e.srcId) + vidToIndex.add(e.dstId) + } + (pid, vidToIndex) + }, preservesPartitioning = true).cache() + + // Within each edge partition, place the vertex attributes received from + // msgsByPartition into the correct locations specified in localVidMap + localVidMap.zipPartitions(msgsByPartition) { (mapIter, msgsIter) => + val (pid, vidToIndex) = mapIter.next() + assert(!mapIter.hasNext) + // Populate the vertex array using the vidToIndex map + val vertexArray = vdManifest.newArray(vidToIndex.capacity) + for ((_, block) <- msgsIter) { + for (i <- 0 until block.vids.size) { + val vid = block.vids(i) + val attr = block.attrs(i) + val ind = vidToIndex.getPos(vid) & OpenHashSet.POSITION_MASK + vertexArray(ind) = attr + } } - } - Iterator((pid, (vidToIndex, vertexArray))) - }.cache() + Iterator((pid, (vidToIndex, vertexArray))) + }.cache() + } } } + +object VTableReplicated { + + def buildBuffer[VD: ClassManifest](pid2vidIter: Iterator[Array[Array[Vid]]], vertexPartIter: Iterator[VertexPartition[VD]]) = { + val pid2vid: Array[Array[Vid]] = pid2vidIter.next() + val vertexPart: VertexPartition[VD] = vertexPartIter.next() + val output = new Array[(Pid, VertexAttributeBlock[VD])](pid2vid.size) + //val output = mmm.newArray(pid2vid.size) + for (pid <- 0 until pid2vid.size) { + val block = new VertexAttributeBlock( + pid2vid(pid), pid2vid(pid).map(vid => vertexPart(vid)).toArray) + output(pid) = (pid, block) + } + output.iterator + } +} + class VertexAttributeBlock[VD: ClassManifest](val vids: Array[Vid], val attrs: Array[VD]) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala index 56f0a544822d5..f677b86cccf2a 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala @@ -30,9 +30,9 @@ private[graph] object VertexPartition { private[graph] class VertexPartition[@specialized(Long, Int, Double) VD: ClassManifest]( - private val index: VertexIdToIndexMap, - private val values: Array[VD], - private val mask: BitSet) + val index: VertexIdToIndexMap, + val values: Array[VD], + val mask: BitSet) extends Logging { val capacity: Int = index.capacity @@ -92,7 +92,8 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassManifest]( /** Inner join another VertexPartition. */ def join[VD2: ClassManifest, VD3: ClassManifest] (other: VertexPartition[VD2]) - (f: (Vid, VD, VD2) => VD3): VertexPartition[VD3] = { + (f: (Vid, VD, VD2) => VD3): VertexPartition[VD3] = + { if (index != other.index) { logWarning("Joining two VertexPartitions with different indexes is slow.") join(createUsingIndex(other.iterator))(f) @@ -109,6 +110,29 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassManifest]( } } + /** Inner join another VertexPartition, only keeping values that change. */ + def deltaJoin[VD2: ClassManifest] + (other: VertexPartition[VD2]) + (f: (Vid, VD, VD2) => VD): VertexPartition[VD] = + { + assert(index == other.index) + + val newValues = new Array[VD](capacity) + val newMask = mask & other.mask + + var i = newMask.nextSetBit(0) + while (i >= 0) { + newValues(i) = f(index.getValue(i), values(i), other.values(i)) + // Only set the mask if the value changes (we are using precise comparison here). + // TODO: Use delta comparison for double type. + if (newValues(i) == values(i)) { + newMask.unset(i) + } + i = mask.nextSetBit(i + 1) + } + new VertexPartition[VD](index, newValues, newMask) + } + /** Left outer join another VertexPartition. */ def leftJoin[VD2: ClassManifest, VD3: ClassManifest] (other: VertexPartition[VD2]) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 47906ada74b84..148e573f0728c 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -51,7 +51,7 @@ object SparkBuild extends Build { lazy val bagel = Project("bagel", file("bagel"), settings = bagelSettings) dependsOn(core) - lazy val graph = Project("graph", file("graph"), settings = graphSettings) dependsOn(core) + lazy val graph = Project("graph", file("graph"), settings = graphSettings) dependsOn(core) lazy val streaming = Project("streaming", file("streaming"), settings = streamingSettings) dependsOn(core) @@ -91,6 +91,7 @@ object SparkBuild extends Build { scalacOptions := Seq("-Xmax-classfile-name", "120", "-unchecked", "-deprecation", "-target:" + SCALAC_JVM_VERSION), javacOptions := Seq("-target", JAVAC_JVM_VERSION, "-source", JAVAC_JVM_VERSION), + javaOptions += "-Dsun.io.serialization.extendedDebugInfo=true", unmanagedJars in Compile <<= baseDirectory map { base => (base / "lib" ** "*.jar").classpath }, retrieveManaged := true, retrievePattern := "[type]s/[artifact](-[revision])(-[classifier]).[ext]", From 15374fed97588efe23596dd6c8fc96c218bfa140 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Thu, 5 Dec 2013 15:47:51 -0800 Subject: [PATCH 256/531] In static Pregel, replicate only changed vertices --- .../scala/org/apache/spark/graph/Graph.scala | 3 +- .../scala/org/apache/spark/graph/Pregel.scala | 7 +++- .../org/apache/spark/graph/VertexRDD.scala | 13 +++---- .../apache/spark/graph/impl/GraphImpl.scala | 28 +++++--------- .../spark/graph/impl/VertexPartition.scala | 38 ++++++++----------- 5 files changed, 38 insertions(+), 51 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/Graph.scala b/graph/src/main/scala/org/apache/spark/graph/Graph.scala index b80c4e32adebd..86502182fb774 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Graph.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Graph.scala @@ -278,8 +278,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { (mapFunc: (Vid, VD, Option[U]) => VD2) : Graph[VD2, ED] - def deltaJoin[VD2: ClassManifest] - (updates: VertexRDD[VD2])(updateF: (Vid, VD, VD2) => VD): Graph[VD, ED] + def deltaJoinVertices(newVerts: VertexRDD[VD], changedVerts: VertexRDD[VD]): Graph[VD, ED] // 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. diff --git a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala index 447a97b1ad51c..bf32d34cbd28d 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala @@ -103,7 +103,12 @@ object Pregel { // compute the messages val messages = g.mapReduceTriplets(sendMsg, mergeMsg) // broadcast & aggregation // receive the messages - g = g.deltaJoin(messages)(vprog).cache() // updating the graph + val newVerts = g.vertices.zipJoin(messages)(vprog).cache() // updating the vertices + val changedVerts = g.vertices.diff(newVerts) + println("Replicating %d changed vertices instead of %d total vertices".format( + changedVerts.count, newVerts.count)) + // replicate the changed vertices + g = graph.deltaJoinVertices(newVerts, changedVerts) // count the iteration i += 1 } diff --git a/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala b/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala index a6f52a6e22542..5afe2df0ca75a 100644 --- a/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala @@ -177,6 +177,12 @@ class VertexRDD[@specialized VD: ClassManifest]( def mapValues[VD2: ClassManifest](f: (Vid, VD) => VD2): VertexRDD[VD2] = this.mapVertexPartitions(_.map(f)) + def diff(other: VertexRDD[VD]): VertexRDD[VD] = { + this.zipVertexPartitions(other) { (thisPart, otherPart) => + thisPart.diff(otherPart) + } + } + /** * Inner join this VertexSet with another VertexSet which has the * same Index. This function will fail if both VertexSets do not @@ -268,13 +274,6 @@ class VertexRDD[@specialized VD: ClassManifest]( } } - def deltaJoin[VD2: ClassManifest](other: VertexRDD[VD2])(f: (Vid, VD, VD2) => VD): VertexRDD[VD] = - { - this.zipVertexPartitions(other) { (thisPart, otherPart) => - thisPart.deltaJoin(otherPart)(f) - } - } - def aggregateUsingIndex[VD2: ClassManifest]( messages: RDD[(Vid, VD2)], reduceFunc: (VD2, VD2) => VD2): VertexRDD[VD2] = { diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index 73018afa4c3b9..0ceb70eecc277 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -28,25 +28,16 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( @transient val vertices: VertexRDD[VD], @transient val edges: EdgeRDD[ED], @transient val vertexPlacement: VertexPlacement, - @transient val prevVTableReplicated: Option[VTableReplicated[VD]] = None) + @transient val vTableReplicated: VTableReplicated[VD]) extends Graph[VD, ED] { - //def this() = this(null, null, null, null) - def this( - vertices: RDD[VertexPartition[VD]], - edges: RDD[(Pid, EdgePartition[ED])], + vertices: VertexRDD[VD], + edges: EdgeRDD[ED], vertexPlacement: VertexPlacement) = { - this(new VertexRDD(vertices), new EdgeRDD(edges), vertexPlacement) + this(vertices, edges, vertexPlacement, new VTableReplicated(vertices, edges, vertexPlacement)) } - @transient private val vTableReplicated: VTableReplicated[VD] = - new VTableReplicated(vertices, edges, vertexPlacement, prevVTableReplicated) - - /** Return a RDD of edges. */ -// @transient override val edges: RDD[Edge[ED]] = -// edges.mapPartitions(_.next()._2.iterator, true) - /** Return a RDD that brings edges with its source and destination vertices together. */ @transient override val triplets: RDD[EdgeTriplet[VD, ED]] = { val vdManifest = classManifest[VD] @@ -269,11 +260,12 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( new GraphImpl(newVTable, edges, vertexPlacement) } - override def deltaJoin[VD2: ClassManifest] - (updates: VertexRDD[VD2])(updateF: (Vid, VD, VD2) => VD): Graph[VD, ED] = - { - val newVTable = vertices.deltaJoin(updates)(updateF) - new GraphImpl(newVTable, edges, vertexPlacement, Some(vTableReplicated)) + override def deltaJoinVertices( + newVerts: VertexRDD[VD], + changedVerts: VertexRDD[VD]): Graph[VD, ED] = { + val newVTableReplicated = new VTableReplicated( + changedVerts, edges, vertexPlacement, Some(vTableReplicated)) + new GraphImpl(newVerts, edges, vertexPlacement, newVTableReplicated) } } // end of class GraphImpl diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala index f677b86cccf2a..ce93899ed1b36 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala @@ -89,6 +89,21 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassManifest]( new VertexPartition(index, values, newMask) } + def diff(other: VertexPartition[VD]): VertexPartition[VD] = { + assert(index == other.index) + + val newMask = mask & other.mask + + var i = newMask.nextSetBit(0) + while (i >= 0) { + if (values(i) == other.values(i)) { + newMask.unset(i) + } + i = mask.nextSetBit(i + 1) + } + new VertexPartition[VD](index, other.values, newMask) + } + /** Inner join another VertexPartition. */ def join[VD2: ClassManifest, VD3: ClassManifest] (other: VertexPartition[VD2]) @@ -110,29 +125,6 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassManifest]( } } - /** Inner join another VertexPartition, only keeping values that change. */ - def deltaJoin[VD2: ClassManifest] - (other: VertexPartition[VD2]) - (f: (Vid, VD, VD2) => VD): VertexPartition[VD] = - { - assert(index == other.index) - - val newValues = new Array[VD](capacity) - val newMask = mask & other.mask - - var i = newMask.nextSetBit(0) - while (i >= 0) { - newValues(i) = f(index.getValue(i), values(i), other.values(i)) - // Only set the mask if the value changes (we are using precise comparison here). - // TODO: Use delta comparison for double type. - if (newValues(i) == values(i)) { - newMask.unset(i) - } - i = mask.nextSetBit(i + 1) - } - new VertexPartition[VD](index, newValues, newMask) - } - /** Left outer join another VertexPartition. */ def leftJoin[VD2: ClassManifest, VD3: ClassManifest] (other: VertexPartition[VD2]) From 4d4c1c931e10dd8a5dfcf4c39d6b37d832b382d8 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Thu, 5 Dec 2013 16:15:56 -0800 Subject: [PATCH 257/531] Test Graph.deltaJoinVertices and VertexRDD.diff --- .../org/apache/spark/graph/GraphSuite.scala | 30 +++++++++++++++---- 1 file changed, 25 insertions(+), 5 deletions(-) diff --git a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala index 1d23911ff90e3..d5aa36c04f0ba 100644 --- a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala @@ -120,9 +120,10 @@ class GraphSuite extends FunSuite with LocalSparkContext { test("VertexSetRDD") { withSpark(new SparkContext("local", "test")) { sc => - val a = sc.parallelize((0 to 100).map(x => (x.toLong, x.toLong)), 5) + val n = 100 + val a = sc.parallelize((0 to n).map(x => (x.toLong, x.toLong)), 5) val b = VertexRDD(a).mapValues(x => -x).cache() // Allow joining b with a derived RDD of b - assert(b.count === 101) + assert(b.count === n + 1) assert(b.leftJoin(a){ (id, a, bOpt) => a + bOpt.get }.map(x=> x._2).reduce(_+_) === 0) val c = b.aggregateUsingIndex[Long](a, (x, y) => x) assert(b.leftJoin(c){ (id, b, cOpt) => b + cOpt.get }.map(x=> x._2).reduce(_+_) === 0) @@ -130,7 +131,8 @@ class GraphSuite extends FunSuite with LocalSparkContext { val e = a.filter(q => ((q._2 % 2) == 0)) assert(d.count === e.count) assert(b.zipJoin(c)((id, b, c) => b + c).map(x => x._2).reduce(_+_) === 0) - + val f = b.mapValues(x => if (x % 2 == 0) -x else x) + assert(b.diff(f).collect().toSet === (2 to n by 2).map(x => (x.toLong, x.toLong)).toSet) } } @@ -143,10 +145,28 @@ class GraphSuite extends FunSuite with LocalSparkContext { val subgraph = star.subgraph(vpred = (vid, attr) => vid % 2 == 0) // We should have 5 vertices. - assert(subgraph.vertices.collect().toSet === (0 to n / 2).map(x => (x * 2, "v")).toSet) + assert(subgraph.vertices.collect().toSet === (0 to n by 2).map(x => (x, "v")).toSet) // And 4 edges. - assert(subgraph.edges.map(_.copy()).collect().toSet === (1 to n / 2).map(x => Edge(0, x * 2, 1)).toSet) + assert(subgraph.edges.map(_.copy()).collect().toSet === (2 to n by 2).map(x => Edge(0, x, 1)).toSet) + } + } + + test("deltaJoinVertices") { + withSpark(new SparkContext("local", "test")) { sc => + // Create a star graph of 10 vertices + val n = 10 + val star = Graph.fromEdgeTuples(sc.parallelize((1 to n).map(x => (0: Vid, x: Vid))), "v1") + + // Modify only vertices whose vids are even + val newVerts = star.vertices.mapValues((vid, attr) => if (vid % 2 == 0) "v2" else attr) + val changedVerts = star.vertices.diff(newVerts) + + // Apply the modification to the graph + val changedStar = star.deltaJoinVertices(newVerts, changedVerts) + + // The graph's vertices should be correct + assert(changedStar.vertices.collect().toSet === newVerts.collect().toSet) } } From 3d57be6151371bb54dfbe3673196e4c73156531a Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Thu, 5 Dec 2013 16:19:26 -0800 Subject: [PATCH 258/531] Revert changes to project/SparkBuild.scala Changes were made in 3fc4534d19a7abf7dcb6c9ebe17e5e29f1febbc5 --- project/SparkBuild.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 148e573f0728c..47906ada74b84 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -51,7 +51,7 @@ object SparkBuild extends Build { lazy val bagel = Project("bagel", file("bagel"), settings = bagelSettings) dependsOn(core) - lazy val graph = Project("graph", file("graph"), settings = graphSettings) dependsOn(core) + lazy val graph = Project("graph", file("graph"), settings = graphSettings) dependsOn(core) lazy val streaming = Project("streaming", file("streaming"), settings = streamingSettings) dependsOn(core) @@ -91,7 +91,6 @@ object SparkBuild extends Build { scalacOptions := Seq("-Xmax-classfile-name", "120", "-unchecked", "-deprecation", "-target:" + SCALAC_JVM_VERSION), javacOptions := Seq("-target", JAVAC_JVM_VERSION, "-source", JAVAC_JVM_VERSION), - javaOptions += "-Dsun.io.serialization.extendedDebugInfo=true", unmanagedJars in Compile <<= baseDirectory map { base => (base / "lib" ** "*.jar").classpath }, retrieveManaged := true, retrievePattern := "[type]s/[artifact](-[revision])(-[classifier]).[ext]", From 4f80dd22bd4068ca82a6b96436bc0fdd97d8e9be Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 5 Dec 2013 16:19:37 -0800 Subject: [PATCH 259/531] Fixed a bug that variable encoding doesn't work for ints that use all 64 bits. --- .../apache/spark/graph/impl/Serializers.scala | 9 ++-- .../apache/spark/graph/SerializerSuite.scala | 41 +++++++++++++++++-- 2 files changed, 42 insertions(+), 8 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala b/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala index 9e393d624b0b7..68b38de2b8686 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala @@ -167,7 +167,7 @@ class DoubleAggMsgSerializer extends Serializer { // Helper classes to shorten the implementation of those special serializers. //////////////////////////////////////////////////////////////////////////////// -sealed abstract class ShuffleSerializationStream(s: OutputStream) extends SerializationStream { +abstract class ShuffleSerializationStream(s: OutputStream) extends SerializationStream { // The implementation should override this one. def writeObject[T](t: T): SerializationStream @@ -280,7 +280,7 @@ sealed abstract class ShuffleSerializationStream(s: OutputStream) extends Serial override def close(): Unit = s.close() } -sealed abstract class ShuffleDeserializationStream(s: InputStream) extends DeserializationStream { +abstract class ShuffleDeserializationStream(s: InputStream) extends DeserializationStream { // The implementation should override this one. def readObject[T](): T @@ -311,17 +311,16 @@ sealed abstract class ShuffleDeserializationStream(s: InputStream) extends Deser def readVarLong(optimizePositive: Boolean): Long = { // TODO: unroll the while loop. var value: Long = 0L - var i: Int = 0 def readOrThrow(): Int = { val in = s.read() if (in < 0) throw new java.io.EOFException in & 0xFF } + var i: Int = 0 var b: Int = readOrThrow() - while ((b & 0x80) != 0) { + while (i < 56 && (b & 0x80) != 0) { value |= (b & 0x7F).toLong << i i += 7 - if (i > 63) throw new IllegalArgumentException("Variable length quantity is too long") b = readOrThrow() } val ret = value | (b.toLong << i) diff --git a/graph/src/test/scala/org/apache/spark/graph/SerializerSuite.scala b/graph/src/test/scala/org/apache/spark/graph/SerializerSuite.scala index 0ec1a2a4d5487..6b86f9b25d00c 100644 --- a/graph/src/test/scala/org/apache/spark/graph/SerializerSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/SerializerSuite.scala @@ -1,13 +1,16 @@ package org.apache.spark.graph +import java.io.{EOFException, ByteArrayInputStream, ByteArrayOutputStream} + +import scala.util.Random + import org.scalatest.FunSuite -import org.apache.spark.SparkContext +import org.apache.spark._ import org.apache.spark.graph.LocalSparkContext._ -import java.io.{EOFException, ByteArrayInputStream, ByteArrayOutputStream} import org.apache.spark.graph.impl._ import org.apache.spark.graph.impl.MsgRDDFunctions._ -import org.apache.spark._ +import org.apache.spark.serializer.SerializationStream class SerializerSuite extends FunSuite with LocalSparkContext { @@ -143,4 +146,36 @@ class SerializerSuite extends FunSuite with LocalSparkContext { bmsgs.partitionBy(new HashPartitioner(3)).collect() } } + + test("variable long encoding") { + def testVarLongEncoding(v: Long, optimizePositive: Boolean) { + val bout = new ByteArrayOutputStream + val stream = new ShuffleSerializationStream(bout) { + def writeObject[T](t: T): SerializationStream = { + writeVarLong(t.asInstanceOf[Long], optimizePositive = optimizePositive) + this + } + } + stream.writeObject(v) + + val bin = new ByteArrayInputStream(bout.toByteArray) + val dstream = new ShuffleDeserializationStream(bin) { + def readObject[T](): T = { + readVarLong(optimizePositive).asInstanceOf[T] + } + } + val read = dstream.readObject[Long]() + assert(read === v) + } + + // Test all variable encoding code path (each branch uses 7 bits, i.e. 1L << 7 difference) + val d = Random.nextLong() % 128 + Seq[Long](0, 1L << 0 + d, 1L << 7 + d, 1L << 14 + d, 1L << 21 + d, 1L << 28 + d, 1L << 35 + d, + 1L << 42 + d, 1L << 49 + d, 1L << 56 + d, 1L << 63 + d).foreach { number => + testVarLongEncoding(number, optimizePositive = false) + testVarLongEncoding(number, optimizePositive = true) + testVarLongEncoding(-number, optimizePositive = false) + testVarLongEncoding(-number, optimizePositive = true) + } + } } From 7457abe282ee797c8f000b64cf9fdb2be6eb5be1 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Thu, 5 Dec 2013 16:59:49 -0800 Subject: [PATCH 260/531] Re-roll last iteration of static Pregel loop --- .../main/scala/org/apache/spark/graph/Pregel.scala | 13 +------------ 1 file changed, 1 insertion(+), 12 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala index bf32d34cbd28d..d38ed7da4af6a 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala @@ -99,7 +99,7 @@ object Pregel { var g = graph.mapVertices( (vid, vdata) => vprog(vid, vdata, initialMsg)).cache() var i = 0 - while (i < numIter - 1) { + while (i < numIter) { // compute the messages val messages = g.mapReduceTriplets(sendMsg, mergeMsg) // broadcast & aggregation // receive the messages @@ -113,17 +113,6 @@ object Pregel { i += 1 } - // compute the messages - val messages = g.mapReduceTriplets(sendMsg, mergeMsg) - // receive the messages - g = g.outerJoinVertices(messages) { (vid, vd, msgOption) => - if (msgOption.isDefined) { - vprog(vid, vd, msgOption.get) - } else { - vd - } - }.cache() - // Return the final graph g } // end of apply From 39b0256668c4f7806725fec751c477ea8c76cd84 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Thu, 5 Dec 2013 18:55:59 -0800 Subject: [PATCH 261/531] Use mask for dynamic Pregel --- .../org/apache/spark/graph/EdgeTriplet.scala | 17 +++--- .../org/apache/spark/graph/GraphLab.scala | 9 ---- .../scala/org/apache/spark/graph/Pregel.scala | 35 +++++------- .../apache/spark/graph/impl/GraphImpl.scala | 53 ++++++------------- .../spark/graph/impl/VTableReplicated.scala | 39 ++++++-------- .../spark/graph/impl/VertexPartition.scala | 15 ++++++ 6 files changed, 71 insertions(+), 97 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/EdgeTriplet.scala b/graph/src/main/scala/org/apache/spark/graph/EdgeTriplet.scala index b8df94c419420..1f92233df9586 100644 --- a/graph/src/main/scala/org/apache/spark/graph/EdgeTriplet.scala +++ b/graph/src/main/scala/org/apache/spark/graph/EdgeTriplet.scala @@ -1,33 +1,38 @@ package org.apache.spark.graph +import org.apache.spark.graph.impl.VertexPartition + /** * An edge triplet represents two vertices and edge along with their * attributes. * * @tparam VD the type of the vertex attribute. * @tparam ED the type of the edge attribute - * + * * @todo specialize edge triplet for basic types, though when I last * tried specializing I got a warning about inherenting from a type * that is not a trait. */ -class EdgeTriplet[VD, ED] extends Edge[ED] { +class EdgeTriplet[VD, ED](vPart: VertexPartition[VD] = null) extends Edge[ED] { // class EdgeTriplet[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) VD: ClassManifest, // @specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED: ClassManifest] extends Edge[ED] { - + /** * The source vertex attribute */ - var srcAttr: VD = _ //nullValue[VD] + var srcAttr: VD = _ //nullValue[VD] /** * The destination vertex attribute */ - var dstAttr: VD = _ //nullValue[VD] + var dstAttr: VD = _ //nullValue[VD] + + def srcMask: Boolean = vPart.isDefined(srcId) + def dstMask: Boolean = vPart.isDefined(dstId) /** - * Set the edge properties of this triplet. + * Set the edge properties of this triplet. */ protected[spark] def set(other: Edge[ED]): EdgeTriplet[VD,ED] = { srcId = other.srcId diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala b/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala index bf1f4168da54b..856a9aca372b4 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala @@ -124,12 +124,3 @@ object GraphLab { activeGraph.mapVertices{case (vid, data) => data._2 } } } - - - - - - - - - diff --git a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala index d38ed7da4af6a..ffbb6fe3cad72 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala @@ -169,36 +169,29 @@ object Pregel { mergeMsg: (A, A) => A) : Graph[VD, ED] = { - def vprogFun(id: Vid, attr: (VD, Boolean), msgOpt: Option[A]): (VD, Boolean) = { - msgOpt match { - case Some(msg) => (vprog(id, attr._1, msg), true) - case None => (attr._1, false) - } - } - - def sendMsgFun(edge: EdgeTriplet[(VD,Boolean), ED]): Iterator[(Vid, A)] = { - if(edge.srcAttr._2) { - val et = new EdgeTriplet[VD, ED] - et.srcId = edge.srcId - et.srcAttr = edge.srcAttr._1 - et.dstId = edge.dstId - et.dstAttr = edge.dstAttr._1 - et.attr = edge.attr - sendMsg(et) + def sendMsgFun(edge: EdgeTriplet[VD, ED]): Iterator[(Vid, A)] = { + if (edge.srcMask) { + sendMsg(edge) } else { Iterator.empty } } - var g = graph.mapVertices( (vid, vdata) => (vprog(vid, vdata, initialMsg), true) ) + var g = graph.mapVertices( (vid, vdata) => vprog(vid, vdata, initialMsg) ) // compute the messages - var messages = g.mapReduceTriplets(sendMsgFun, mergeMsg).cache + var messages = g.mapReduceTriplets(sendMsgFun, mergeMsg).cache() var activeMessages = messages.count // Loop var i = 0 while (activeMessages > 0) { // receive the messages - g = g.outerJoinVertices(messages)(vprogFun) + val newVerts = g.vertices.zipJoin(messages)(vprog).cache() // updating the vertices + val changedVerts = g.vertices.diff(newVerts) + println("Replicating %d changed vertices instead of %d total vertices".format( + changedVerts.count, newVerts.count)) + // replicate the changed vertices + g = graph.deltaJoinVertices(newVerts, changedVerts) + val oldMessages = messages // compute the messages messages = g.mapReduceTriplets(sendMsgFun, mergeMsg).cache @@ -208,8 +201,8 @@ object Pregel { // count the iteration i += 1 } - // Return the final graph - g.mapVertices((id, attr) => attr._1) + + g } // end of apply } // end of class Pregel diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index 0ceb70eecc277..6e9566e06071b 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -23,6 +23,9 @@ import org.apache.spark.util.collection.{BitSet, OpenHashSet, PrimitiveKeyOpenHa * destinations. `vertexPlacement` specifies where each vertex will be * replicated. `vTableReplicated` stores the replicated vertex attributes, which * are co-partitioned with the relevant edges. + * + * mask in vertices means filter + * mask in vTableReplicated means active */ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( @transient val vertices: VertexRDD[VD], @@ -44,8 +47,8 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( val edManifest = classManifest[ED] edges.zipEdgePartitions(vTableReplicated.bothAttrs) { (edgePartition, vTableReplicatedIter) => - val (_, (vidToIndex, vertexArray)) = vTableReplicatedIter.next() - new EdgeTripletIterator(vidToIndex, vertexArray, edgePartition)(vdManifest, edManifest) + val (_, vPart) = vTableReplicatedIter.next() + new EdgeTripletIterator(vPart.index, vPart.values, edgePartition)(vdManifest, edManifest) } } @@ -141,14 +144,12 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( val vdManifest = classManifest[VD] val newETable = edges.zipEdgePartitions(vTableReplicated.bothAttrs) { (edgePartition, vTableReplicatedIter) => - val (pid, (vidToIndex, vertexArray)) = vTableReplicatedIter.next() + val (pid, vPart) = vTableReplicatedIter.next() val et = new EdgeTriplet[VD, ED] - val vmap = new PrimitiveKeyOpenHashMap[Vid, VD]( - vidToIndex, vertexArray)(classManifest[Vid], vdManifest) val newEdgePartition = edgePartition.map { e => et.set(e) - et.srcAttr = vmap(e.srcId) - et.dstAttr = vmap(e.dstId) + et.srcAttr = vPart(e.srcId) + et.dstAttr = vPart(e.dstId) f(et) } Iterator((pid, newEdgePartition)) @@ -209,44 +210,22 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( // Map and combine. val preAgg = edges.zipEdgePartitions(vs) { (edgePartition, vTableReplicatedIter) => - val (_, (vidToIndex, vertexArray)) = vTableReplicatedIter.next() - assert(vidToIndex.capacity == vertexArray.size) - val vmap = new PrimitiveKeyOpenHashMap[Vid, VD](vidToIndex, vertexArray)( - classManifest[Vid], vdManifest) + val (_, vertexPartition) = vTableReplicatedIter.next() - // Note: This doesn't allow users to send messages to arbitrary vertices. - val msgArray = new Array[A](vertexArray.size) - val msgBS = new BitSet(vertexArray.size) // Iterate over the partition - val et = new EdgeTriplet[VD, ED] - - edgePartition.foreach { e => + val et = new EdgeTriplet[VD, ED](vertexPartition) + val filteredEdges = edgePartition.iterator.flatMap { e => et.set(e) if (mapUsesSrcAttr) { - et.srcAttr = vmap(e.srcId) + et.srcAttr = vertexPartition(e.srcId) } if (mapUsesDstAttr) { - et.dstAttr = vmap(e.dstId) - } - // TODO(rxin): rewrite the foreach using a simple while loop to speed things up. - // Also given we are only allowing zero, one, or two messages, we can completely unroll - // the for loop. - mapFunc(et).foreach { case (vid, msg) => - // verify that the vid is valid - assert(vid == et.srcId || vid == et.dstId) - // Get the index of the key - val ind = vidToIndex.getPos(vid) & OpenHashSet.POSITION_MASK - // Populate the aggregator map - if (msgBS.get(ind)) { - msgArray(ind) = reduceFunc(msgArray(ind), msg) - } else { - msgArray(ind) = msg - msgBS.set(ind) - } + et.dstAttr = vertexPartition(e.dstId) } + mapFunc(et) } - // construct an iterator of tuples. Iterator[(Vid, A)] - msgBS.iterator.map { ind => (vidToIndex.getValue(ind), msgArray(ind)) } + // Note: This doesn't allow users to send messages to arbitrary vertices. + vertexPartition.aggregateUsingIndex(filteredEdges, reduceFunc).iterator } // do the final reduction reusing the index map diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala b/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala index be7cf516ea5cf..3e3769b9afc79 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala @@ -16,19 +16,19 @@ class VTableReplicated[VD: ClassManifest]( vertexPlacement: VertexPlacement, prevVTableReplicated: Option[VTableReplicated[VD]] = None) { - val bothAttrs: RDD[(Pid, (VertexIdToIndexMap, Array[VD]))] = + val bothAttrs: RDD[(Pid, VertexPartition[VD])] = createVTableReplicated(vTable, eTable, vertexPlacement, true, true) - val srcAttrOnly: RDD[(Pid, (VertexIdToIndexMap, Array[VD]))] = + val srcAttrOnly: RDD[(Pid, VertexPartition[VD])] = createVTableReplicated(vTable, eTable, vertexPlacement, true, false) - val dstAttrOnly: RDD[(Pid, (VertexIdToIndexMap, Array[VD]))] = + val dstAttrOnly: RDD[(Pid, VertexPartition[VD])] = createVTableReplicated(vTable, eTable, vertexPlacement, false, true) - val noAttrs: RDD[(Pid, (VertexIdToIndexMap, Array[VD]))] = + val noAttrs: RDD[(Pid, VertexPartition[VD])] = createVTableReplicated(vTable, eTable, vertexPlacement, false, false) - def get(includeSrc: Boolean, includeDst: Boolean): RDD[(Pid, (VertexIdToIndexMap, Array[VD]))] = { + def get(includeSrc: Boolean, includeDst: Boolean): RDD[(Pid, VertexPartition[VD])] = { (includeSrc, includeDst) match { case (true, true) => bothAttrs case (true, false) => srcAttrOnly @@ -42,7 +42,7 @@ class VTableReplicated[VD: ClassManifest]( eTable: EdgeRDD[_], vertexPlacement: VertexPlacement, includeSrcAttr: Boolean, - includeDstAttr: Boolean): RDD[(Pid, (VertexIdToIndexMap, Array[VD]))] = { + includeDstAttr: Boolean): RDD[(Pid, VertexPartition[VD])] = { val placement = vertexPlacement.get(includeSrcAttr, includeDstAttr) val vdManifest = classManifest[VD] @@ -55,25 +55,14 @@ class VTableReplicated[VD: ClassManifest]( prevVTableReplicated match { case Some(vTableReplicated) => - val prev: RDD[(Pid, (VertexIdToIndexMap, Array[VD]))] = + val prev: RDD[(Pid, VertexPartition[VD])] = vTableReplicated.get(includeSrcAttr, includeDstAttr) prev.zipPartitions(msgsByPartition) { (vTableIter, msgsIter) => - val (pid, (vidToIndex, oldVertexArray)) = vTableIter.next() - - val vertexArray = vdManifest.newArray(oldVertexArray.length) - System.arraycopy(oldVertexArray, 0, vertexArray, 0, vertexArray.length) - - for ((_, block) <- msgsIter) { - for (i <- 0 until block.vids.size) { - val vid = block.vids(i) - val attr = block.attrs(i) - val ind = vidToIndex.getPos(vid) & OpenHashSet.POSITION_MASK - vertexArray(ind) = attr - } - } - - Iterator((pid, (vidToIndex, vertexArray))) + val (pid, vertexPartition) = vTableIter.next() + val (_, block) = msgsIter.next() + val newVPart = vertexPartition.updateUsingIndex(block.iterator)(vdManifest) + Iterator((pid, newVPart)) }.cache() case None => @@ -107,7 +96,7 @@ class VTableReplicated[VD: ClassManifest]( vertexArray(ind) = attr } } - Iterator((pid, (vidToIndex, vertexArray))) + Iterator((pid, new VertexPartition(vidToIndex, vertexArray, vidToIndex.getBitSet)(vdManifest))) }.cache() } } @@ -131,4 +120,6 @@ object VTableReplicated { } } -class VertexAttributeBlock[VD: ClassManifest](val vids: Array[Vid], val attrs: Array[VD]) +class VertexAttributeBlock[VD: ClassManifest](val vids: Array[Vid], val attrs: Array[VD]) { + def iterator: Iterator[(Vid, VD)] = (0 until vids.size).iterator.map { i => (vids(i), attrs(i)) } +} diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala index ce93899ed1b36..c922350345aaf 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala @@ -42,6 +42,8 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassManifest]( /** Return the vertex attribute for the given vertex ID. */ def apply(vid: Vid): VD = values(index.getPos(vid)) + def isDefined(vid: Vid): Boolean = mask.get(index.getPos(vid)) + /** * Pass each vertex attribute along with the vertex id through a map * function and retain the original RDD's partitioning and index. @@ -167,6 +169,19 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassManifest]( new VertexPartition[VD2](index, newValues, newMask) } + def updateUsingIndex[VD2: ClassManifest](iter: Iterator[Product2[Vid, VD2]]) + : VertexPartition[VD2] = { + val newMask = new BitSet(capacity) + val newValues = new Array[VD2](capacity) + System.arraycopy(values, 0, newValues, 0, newValues.length) + iter.foreach { case (vid, vdata) => + val pos = index.getPos(vid) + newMask.set(pos) + newValues(pos) = vdata + } + new VertexPartition[VD2](index, newValues, newMask) + } + def aggregateUsingIndex[VD2: ClassManifest]( iter: Iterator[Product2[Vid, VD2]], reduceFunc: (VD2, VD2) => VD2): VertexPartition[VD2] = { From 67bfa7fd6579823377778b151418ba451f74f2ba Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Thu, 5 Dec 2013 19:16:00 -0800 Subject: [PATCH 262/531] Test prevVTableReplicated code path --- .../test/scala/org/apache/spark/graph/GraphSuite.scala | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala index d5aa36c04f0ba..fa4ebf3c888fb 100644 --- a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala @@ -156,7 +156,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { withSpark(new SparkContext("local", "test")) { sc => // Create a star graph of 10 vertices val n = 10 - val star = Graph.fromEdgeTuples(sc.parallelize((1 to n).map(x => (0: Vid, x: Vid))), "v1") + val star = Graph.fromEdgeTuples(sc.parallelize((1 to n).map(x => (0: Vid, x: Vid))), "v1").cache() // Modify only vertices whose vids are even val newVerts = star.vertices.mapValues((vid, attr) => if (vid % 2 == 0) "v2" else attr) @@ -167,7 +167,12 @@ class GraphSuite extends FunSuite with LocalSparkContext { // The graph's vertices should be correct assert(changedStar.vertices.collect().toSet === newVerts.collect().toSet) + + // Send the leaf attributes to the center + val sums = changedStar.mapReduceTriplets( + edge => Iterator((edge.srcId, Set(edge.dstAttr))), + (a: Set[String], b: Set[String]) => a ++ b) + assert(sums.collect().toSet === Set((0, Set("v1", "v2")))) } } - } From b707861ba05ab744acb3b49cb36b3b71a6df85fc Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Thu, 5 Dec 2013 20:01:32 -0800 Subject: [PATCH 263/531] Simplify GraphImpl.deltaJoinVertices --- .../scala/org/apache/spark/graph/Graph.scala | 2 +- .../scala/org/apache/spark/graph/Pregel.scala | 14 ++++------- .../org/apache/spark/graph/VertexRDD.scala | 8 +++++++ .../apache/spark/graph/impl/GraphImpl.scala | 10 +++++--- .../spark/graph/impl/VertexPartition.scala | 24 +++++++++++++++++++ 5 files changed, 44 insertions(+), 14 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/Graph.scala b/graph/src/main/scala/org/apache/spark/graph/Graph.scala index 86502182fb774..a0907c319aac6 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Graph.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Graph.scala @@ -278,7 +278,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { (mapFunc: (Vid, VD, Option[U]) => VD2) : Graph[VD2, ED] - def deltaJoinVertices(newVerts: VertexRDD[VD], changedVerts: VertexRDD[VD]): Graph[VD, ED] + def deltaJoinVertices(changedVerts: VertexRDD[VD]): Graph[VD, ED] // 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. diff --git a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala index ffbb6fe3cad72..1e59a39485b8f 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala @@ -103,12 +103,9 @@ object Pregel { // compute the messages val messages = g.mapReduceTriplets(sendMsg, mergeMsg) // broadcast & aggregation // receive the messages - val newVerts = g.vertices.zipJoin(messages)(vprog).cache() // updating the vertices - val changedVerts = g.vertices.diff(newVerts) - println("Replicating %d changed vertices instead of %d total vertices".format( - changedVerts.count, newVerts.count)) + val changedVerts = g.vertices.deltaJoin(messages)(vprog).cache() // updating the vertices // replicate the changed vertices - g = graph.deltaJoinVertices(newVerts, changedVerts) + g = graph.deltaJoinVertices(changedVerts) // count the iteration i += 1 } @@ -185,12 +182,9 @@ object Pregel { var i = 0 while (activeMessages > 0) { // receive the messages - val newVerts = g.vertices.zipJoin(messages)(vprog).cache() // updating the vertices - val changedVerts = g.vertices.diff(newVerts) - println("Replicating %d changed vertices instead of %d total vertices".format( - changedVerts.count, newVerts.count)) + val changedVerts = g.vertices.deltaJoin(messages)(vprog).cache() // updating the vertices // replicate the changed vertices - g = graph.deltaJoinVertices(newVerts, changedVerts) + g = graph.deltaJoinVertices(changedVerts) val oldMessages = messages // compute the messages diff --git a/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala b/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala index 5afe2df0ca75a..1b8ab89ebe465 100644 --- a/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala @@ -207,6 +207,14 @@ class VertexRDD[@specialized VD: ClassManifest]( } } + def deltaJoin[VD2: ClassManifest] + (other: VertexRDD[VD2])(f: (Vid, VD, VD2) => VD): VertexRDD[VD] = + { + this.zipVertexPartitions(other) { (thisPart, otherPart) => + thisPart.deltaJoin(otherPart)(f) + } + } + /** * Left join this VertexSet with another VertexSet which has the * same Index. This function will fail if both VertexSets do not diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index 6e9566e06071b..4300812990cff 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -239,9 +239,13 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( new GraphImpl(newVTable, edges, vertexPlacement) } - override def deltaJoinVertices( - newVerts: VertexRDD[VD], - changedVerts: VertexRDD[VD]): Graph[VD, ED] = { + override def deltaJoinVertices(changedVerts: VertexRDD[VD]): Graph[VD, ED] = { + val newVerts = vertices.leftZipJoin(changedVerts) { (vid, oldAttr, newAttrOpt) => + newAttrOpt match { + case Some(newAttr) => newAttr + case None => oldAttr + } + } val newVTableReplicated = new VTableReplicated( changedVerts, edges, vertexPlacement, Some(vTableReplicated)) new GraphImpl(newVerts, edges, vertexPlacement, newVTableReplicated) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala index c922350345aaf..0af445fa7d127 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala @@ -127,6 +127,30 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassManifest]( } } + /** Inner join another VertexPartition. */ + def deltaJoin[VD2: ClassManifest, VD3: ClassManifest] + (other: VertexPartition[VD2]) + (f: (Vid, VD, VD2) => VD3): VertexPartition[VD3] = + { + if (index != other.index) { + logWarning("Joining two VertexPartitions with different indexes is slow.") + join(createUsingIndex(other.iterator))(f) + } else { + val newValues = new Array[VD3](capacity) + val newMask = mask & other.mask + + var i = newMask.nextSetBit(0) + while (i >= 0) { + newValues(i) = f(index.getValue(i), values(i), other.values(i)) + if (newValues(i) == values(i)) { + newMask.unset(i) + } + i = mask.nextSetBit(i + 1) + } + new VertexPartition(index, newValues, newMask) + } + } + /** Left outer join another VertexPartition. */ def leftJoin[VD2: ClassManifest, VD3: ClassManifest] (other: VertexPartition[VD2]) From 15168d6c4dab8f5debf6d3935035fbdb923b157b Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 5 Dec 2013 23:25:53 -0800 Subject: [PATCH 264/531] Fixed a bug in VTableReplicated that we are always broadcasting all the vertices. --- .../scala/org/apache/spark/graph/Pregel.scala | 10 ++++---- .../spark/graph/impl/VTableReplicated.scala | 25 +++++++++++++------ .../org/apache/spark/graph/GraphSuite.scala | 14 ++++++++--- 3 files changed, 32 insertions(+), 17 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala index 1e59a39485b8f..8e9ca89efcf36 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala @@ -105,7 +105,7 @@ object Pregel { // receive the messages val changedVerts = g.vertices.deltaJoin(messages)(vprog).cache() // updating the vertices // replicate the changed vertices - g = graph.deltaJoinVertices(changedVerts) + g = g.deltaJoinVertices(changedVerts) // count the iteration i += 1 } @@ -177,19 +177,19 @@ object Pregel { var g = graph.mapVertices( (vid, vdata) => vprog(vid, vdata, initialMsg) ) // compute the messages var messages = g.mapReduceTriplets(sendMsgFun, mergeMsg).cache() - var activeMessages = messages.count + var activeMessages = messages.count() // Loop var i = 0 while (activeMessages > 0) { // receive the messages val changedVerts = g.vertices.deltaJoin(messages)(vprog).cache() // updating the vertices // replicate the changed vertices - g = graph.deltaJoinVertices(changedVerts) + g = g.deltaJoinVertices(changedVerts) val oldMessages = messages // compute the messages - messages = g.mapReduceTriplets(sendMsgFun, mergeMsg).cache - activeMessages = messages.count + messages = g.mapReduceTriplets(sendMsgFun, mergeMsg).cache() + activeMessages = messages.count() // after counting we can unpersist the old messages oldMessages.unpersist(blocking=false) // count the iteration diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala b/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala index 3e3769b9afc79..0c50ad09c71ef 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala @@ -2,7 +2,7 @@ package org.apache.spark.graph.impl import org.apache.spark.SparkContext._ import org.apache.spark.rdd.RDD -import org.apache.spark.util.collection.OpenHashSet +import org.apache.spark.util.collection.{PrimitiveVector, OpenHashSet} import org.apache.spark.graph._ @@ -109,14 +109,23 @@ object VTableReplicated { def buildBuffer[VD: ClassManifest](pid2vidIter: Iterator[Array[Array[Vid]]], vertexPartIter: Iterator[VertexPartition[VD]]) = { val pid2vid: Array[Array[Vid]] = pid2vidIter.next() val vertexPart: VertexPartition[VD] = vertexPartIter.next() - val output = new Array[(Pid, VertexAttributeBlock[VD])](pid2vid.size) - //val output = mmm.newArray(pid2vid.size) - for (pid <- 0 until pid2vid.size) { - val block = new VertexAttributeBlock( - pid2vid(pid), pid2vid(pid).map(vid => vertexPart(vid)).toArray) - output(pid) = (pid, block) + + Iterator.tabulate(pid2vid.size) { pid => + val vidsCandidate = pid2vid(pid) + val size = vidsCandidate.length + val vids = new PrimitiveVector[Vid](pid2vid(pid).size) + val attrs = new PrimitiveVector[VD](pid2vid(pid).size) + var i = 0 + while (i < size) { + val vid = vidsCandidate(i) + if (vertexPart.isDefined(vid)) { + vids += vid + attrs += vertexPart(vid) + } + i += 1 + } + (pid, new VertexAttributeBlock(vids.trim().array, attrs.trim().array)) } - output.iterator } } diff --git a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala index fa4ebf3c888fb..514d20b76c0bc 100644 --- a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala @@ -159,14 +159,20 @@ class GraphSuite extends FunSuite with LocalSparkContext { val star = Graph.fromEdgeTuples(sc.parallelize((1 to n).map(x => (0: Vid, x: Vid))), "v1").cache() // Modify only vertices whose vids are even - val newVerts = star.vertices.mapValues((vid, attr) => if (vid % 2 == 0) "v2" else attr) - val changedVerts = star.vertices.diff(newVerts) + val changedVerts = star.vertices.filter(_._1 % 2 == 0).mapValues((vid, attr) => "v2") // Apply the modification to the graph - val changedStar = star.deltaJoinVertices(newVerts, changedVerts) + val changedStar = star.deltaJoinVertices(changedVerts) + + val newVertices = star.vertices.leftZipJoin(changedVerts) { (vid, oldVd, newVdOpt) => + newVdOpt match { + case Some(newVd) => newVd + case None => oldVd + } + } // The graph's vertices should be correct - assert(changedStar.vertices.collect().toSet === newVerts.collect().toSet) + assert(changedStar.vertices.collect().toSet === newVertices.collect().toSet) // Send the leaf attributes to the center val sums = changedStar.mapReduceTriplets( From 3b0ee53eda906eef109e6a4332e62e09e432aeb8 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 5 Dec 2013 23:30:56 -0800 Subject: [PATCH 265/531] Minor update. --- .../main/scala/org/apache/spark/graph/impl/GraphImpl.scala | 4 ---- .../test/scala/org/apache/spark/graph/AnalyticsSuite.scala | 4 ++-- 2 files changed, 2 insertions(+), 6 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index 4300812990cff..771c460345655 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -9,7 +9,6 @@ import org.apache.spark.graph.util.BytecodeUtils import org.apache.spark.rdd.{ShuffledRDD, RDD} import org.apache.spark.storage.StorageLevel import org.apache.spark.util.ClosureCleaner -import org.apache.spark.util.collection.{BitSet, OpenHashSet, PrimitiveKeyOpenHashMap} /** @@ -199,9 +198,6 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( ClosureCleaner.clean(mapFunc) ClosureCleaner.clean(reduceFunc) - // Use explicit manifest in PrimitiveKeyOpenHashMap so we don't have to serialize GraphImpl. - val vdManifest = classManifest[VD] - // 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") diff --git a/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala b/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala index c9cb65eeda3fb..e719ca08722f9 100644 --- a/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala @@ -93,7 +93,7 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { val prGraph2 = PageRank.runUntillConvergence(gridGraph, 0.0001, resetProb).cache() val error = prGraph1.vertices.zipJoin(prGraph2.vertices) { case (id, a, b) => (a - b) * (a - b) } .map { case (id, error) => error }.sum - prGraph1.vertices.zipJoin(prGraph2.vertices) { (id, a, b) => (a, b, a-b) }.foreach(println(_)) + //prGraph1.vertices.zipJoin(prGraph2.vertices) { (id, a, b) => (a, b, a-b) }.foreach(println(_)) println(error) assert(error < 1.0e-5) val pr3: RDD[(Vid, Double)] = sc.parallelize(GridPageRank(10,10, 50, resetProb)) @@ -101,7 +101,7 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { val b: Double = bOpt.get (a - b) * (a - b) }.map { case (id, error) => error }.sum - prGraph1.vertices.leftJoin(pr3) { (id, a, b) => (a, b) }.foreach( println(_) ) + //prGraph1.vertices.leftJoin(pr3) { (id, a, b) => (a, b) }.foreach( println(_) ) println(error2) assert(error2 < 1.0e-5) } From 41721b1494fe33c184374925bada3981296dee69 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 6 Dec 2013 00:51:12 -0800 Subject: [PATCH 266/531] Fixed a bug in VTableReplicated that we only process the first block. --- .../scala/org/apache/spark/graph/impl/VTableReplicated.scala | 3 +-- .../scala/org/apache/spark/graph/impl/VertexPartition.scala | 5 ++++- .../test/scala/org/apache/spark/graph/AnalyticsSuite.scala | 3 +-- 3 files changed, 6 insertions(+), 5 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala b/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala index 0c50ad09c71ef..8ee631bdbae4f 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala @@ -60,8 +60,7 @@ class VTableReplicated[VD: ClassManifest]( prev.zipPartitions(msgsByPartition) { (vTableIter, msgsIter) => val (pid, vertexPartition) = vTableIter.next() - val (_, block) = msgsIter.next() - val newVPart = vertexPartition.updateUsingIndex(block.iterator)(vdManifest) + val newVPart = vertexPartition.updateUsingIndex(msgsIter.flatMap(_._2.iterator))(vdManifest) Iterator((pid, newVPart)) }.cache() diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala index 0af445fa7d127..fe005c872328f 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala @@ -42,7 +42,10 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassManifest]( /** Return the vertex attribute for the given vertex ID. */ def apply(vid: Vid): VD = values(index.getPos(vid)) - def isDefined(vid: Vid): Boolean = mask.get(index.getPos(vid)) + def isDefined(vid: Vid): Boolean = { + val pos = index.getPos(vid) + pos >= 0 && mask.get(pos) + } /** * Pass each vertex attribute along with the vertex id through a map diff --git a/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala b/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala index e719ca08722f9..a3ac7470a54fe 100644 --- a/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala @@ -64,7 +64,7 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { if (pr1 != pr2) { 1 } else { 0 } }.map { case (vid, test) => test }.sum assert(notMatching === 0) - prGraph2.vertices.foreach(println(_)) + //prGraph2.vertices.foreach(println(_)) val errors = prGraph2.vertices.map { case (vid, pr) => val correct = (vid > 0 && pr == resetProb) || (vid == 0 && math.abs(pr - (resetProb + (1.0 - resetProb) * (resetProb * (nVertices - 1)) )) < 1.0E-5) @@ -141,7 +141,6 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { else { assert(cc === 10) } } val ccMap = vertices.toMap - println(ccMap) for (id <- 0 until 20) { if (id < 10) { assert(ccMap(id) === 0) From 84d0e1a3344fb6afbc8c3562018a4b8767002051 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 6 Dec 2013 21:16:59 -0800 Subject: [PATCH 267/531] Set RDD names for easy debugging --- graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala | 2 ++ graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala | 2 ++ .../scala/org/apache/spark/graph/impl/VTableReplicated.scala | 4 ++-- .../scala/org/apache/spark/graph/impl/VertexPlacement.scala | 2 +- 4 files changed, 7 insertions(+), 3 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala b/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala index e9174df53d826..24844262bc1a9 100644 --- a/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala @@ -11,6 +11,8 @@ class EdgeRDD[@specialized ED: ClassManifest]( val partitionsRDD: RDD[(Pid, EdgePartition[ED])]) extends RDD[Edge[ED]](partitionsRDD.context, List(new OneToOneDependency(partitionsRDD))) { + partitionsRDD.setName("EdgeRDD") + override val partitioner = partitionsRDD.partitioner override protected def getPartitions: Array[Partition] = partitionsRDD.partitions diff --git a/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala b/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala index 1b8ab89ebe465..33e0b892fb191 100644 --- a/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala @@ -56,6 +56,8 @@ class VertexRDD[@specialized VD: ClassManifest]( require(partitionsRDD.partitioner.isDefined) + partitionsRDD.setName("VertexRDD") + /** * Construct a new VertexRDD that is indexed by only the keys in the RDD. * The resulting VertexSet will be based on a different index and can diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala b/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala index 8ee631bdbae4f..6cbeb50186452 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala @@ -62,7 +62,7 @@ class VTableReplicated[VD: ClassManifest]( val (pid, vertexPartition) = vTableIter.next() val newVPart = vertexPartition.updateUsingIndex(msgsIter.flatMap(_._2.iterator))(vdManifest) Iterator((pid, newVPart)) - }.cache() + }.cache().setName("VTableReplicated delta %s %s".format(includeSrcAttr, includeDstAttr)) case None => // Within each edge partition, create a local map from vid to an index into @@ -96,7 +96,7 @@ class VTableReplicated[VD: ClassManifest]( } } Iterator((pid, new VertexPartition(vidToIndex, vertexArray, vidToIndex.getBitSet)(vdManifest))) - }.cache() + }.cache().setName("VTableReplicated %s %s".format(includeSrcAttr, includeDstAttr)) } } diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPlacement.scala b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPlacement.scala index 2f70e92da4f60..24fdf0db45cc1 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPlacement.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPlacement.scala @@ -64,6 +64,6 @@ class VertexPlacement(eTable: EdgeRDD[_], vTable: VertexRDD[_]) { } Iterator(pid2vid.map(_.trim().array)) - } + }.cache().setName("VertexPlacement %s %s".format(includeSrcAttr, includeDstAttr)) } } From 12483d4ae641d4f3d9153e763fbd8aab0931ef9b Mon Sep 17 00:00:00 2001 From: Dan Crankshaw Date: Wed, 11 Dec 2013 20:56:21 -0800 Subject: [PATCH 268/531] Added BoundedPriorityQueue kryo registrator. Fixes top issue. --- .../scala/org/apache/spark/graph/GraphKryoRegistrator.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphKryoRegistrator.scala b/graph/src/main/scala/org/apache/spark/graph/GraphKryoRegistrator.scala index a02b33686beed..b8c1b5b0f032d 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphKryoRegistrator.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphKryoRegistrator.scala @@ -5,6 +5,7 @@ import com.esotericsoftware.kryo.Kryo import org.apache.spark.graph.impl._ import org.apache.spark.serializer.KryoRegistrator import org.apache.spark.util.collection.BitSet +import org.apache.spark.util.BoundedPriorityQueue class GraphKryoRegistrator extends KryoRegistrator { @@ -19,6 +20,7 @@ class GraphKryoRegistrator extends KryoRegistrator { kryo.register(classOf[VertexIdToIndexMap]) kryo.register(classOf[VertexAttributeBlock[Object]]) kryo.register(classOf[PartitionStrategy]) + kryo.register(classOf[BoundedPriorityQueue[Object]]) // This avoids a large number of hash table lookups. kryo.setReferences(false) From e8ba51d6440574957a7e19ccecd139ac236cf091 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Thu, 12 Dec 2013 15:06:03 -0800 Subject: [PATCH 269/531] Add standalone PageRank using only GraphX operators --- .../spark/graph/algorithms/PageRank.scala | 52 +++++++++++++++++-- 1 file changed, 49 insertions(+), 3 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/algorithms/PageRank.scala b/graph/src/main/scala/org/apache/spark/graph/algorithms/PageRank.scala index d190910c55cb5..bb92e7c767ad2 100644 --- a/graph/src/main/scala/org/apache/spark/graph/algorithms/PageRank.scala +++ b/graph/src/main/scala/org/apache/spark/graph/algorithms/PageRank.scala @@ -1,9 +1,10 @@ package org.apache.spark.graph.algorithms +import org.apache.spark.Logging import org.apache.spark.graph._ -object PageRank { +object PageRank extends Logging { /** * Run PageRank for a fixed number of iterations returning a graph @@ -60,7 +61,7 @@ object PageRank { .mapVertices( (id, attr) => 1.0 ) // Display statistics about pagerank - println(pagerankGraph.statistics) + logInfo(pagerankGraph.statistics.toString) // Define the three functions needed to implement PageRank in the GraphX // version of Pregel @@ -124,7 +125,7 @@ object PageRank { .mapVertices( (id, attr) => (0.0, 0.0) ) // Display statistics about pagerank - println(pagerankGraph.statistics) + logInfo(pagerankGraph.statistics.toString) // Define the three functions needed to implement PageRank in the GraphX // version of Pregel @@ -151,4 +152,49 @@ object PageRank { Pregel(pagerankGraph, initialMessage)(vertexProgram, sendMessage, messageCombiner) .mapVertices((vid, attr) => attr._1) } // end of deltaPageRank + + def runStandalone[VD: Manifest, ED: Manifest]( + graph: Graph[VD, ED], tol: Double, resetProb: Double = 0.15): VertexRDD[Double] = { + + // Initialize the ranks + var ranks: VertexRDD[Double] = graph.vertices.mapValues((vid, attr) => resetProb).cache() + + // Initialize the delta graph where each vertex stores its delta and each edge knows its weight + var deltaGraph: Graph[Double, Double] = + graph.outerJoinVertices(graph.outDegrees)((vid, vdata, deg) => deg.getOrElse(0)) + .mapTriplets(e => 1.0 / e.srcAttr) + .mapVertices((vid, degree) => resetProb).cache() + var numDeltas: Long = ranks.count() + + var i = 0 + val weight = (1.0 - resetProb) + while (numDeltas > 0) { + // Compute new deltas + val deltas = deltaGraph + .mapReduceTriplets[Double]( + et => { + if (et.srcMask) Iterator((et.dstId, et.srcAttr * et.attr * weight)) + else Iterator.empty + }, + _ + _) + .filter { case (vid, delta) => delta > tol } + .cache() + numDeltas = deltas.count() + logInfo("Standalone PageRank: iter %d has %d deltas".format(i, numDeltas)) + + // Apply deltas. Sets the mask for each vertex to false if it does not appear in deltas. + deltaGraph = deltaGraph.deltaJoinVertices(deltas).cache() + + // Update ranks + ranks = ranks.leftZipJoin(deltas) { (vid, oldRank, deltaOpt) => + oldRank + deltaOpt.getOrElse(0.0) + } + ranks.foreach(x => {}) // force the iteration for ease of debugging + + i += 1 + } + + ranks + } + } From a0fb477726f20c2c7eed0eed19008c3642a76da6 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Thu, 12 Dec 2013 15:42:55 -0800 Subject: [PATCH 270/531] Test standalone PageRank --- .../apache/spark/graph/AnalyticsSuite.scala | 89 ++++++++++++------- 1 file changed, 55 insertions(+), 34 deletions(-) diff --git a/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala b/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala index a3ac7470a54fe..2e6b57a8ec267 100644 --- a/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala @@ -51,35 +51,38 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { System.setProperty("spark.serializer", "org.apache.spark.serializer.KryoSerializer") System.setProperty("spark.kryo.registrator", "org.apache.spark.graph.GraphKryoRegistrator") + def compareRanks(a: VertexRDD[Double], b: VertexRDD[Double]): Double = { + a.leftJoin(b) { case (id, a, bOpt) => (a - bOpt.getOrElse(0.0)) * (a - bOpt.getOrElse(0.0)) } + .map { case (id, error) => error }.sum + } test("Star PageRank") { withSpark(new SparkContext("local", "test")) { sc => val nVertices = 100 val starGraph = GraphGenerators.starGraph(sc, nVertices).cache() val resetProb = 0.15 - val prGraph1 = PageRank.run(starGraph, 1, resetProb) - val prGraph2 = PageRank.run(starGraph, 2, resetProb) + val errorTol = 1.0e-5 + + val staticRanks1 = PageRank.run(starGraph, numIter = 1, resetProb).vertices.cache() + val staticRanks2 = PageRank.run(starGraph, numIter = 2, resetProb).vertices.cache() - val notMatching = prGraph1.vertices.zipJoin(prGraph2.vertices) { (vid, pr1, pr2) => - if (pr1 != pr2) { 1 } else { 0 } + // Static PageRank should only take 2 iterations to converge + val notMatching = staticRanks1.zipJoin(staticRanks2) { (vid, pr1, pr2) => + if (pr1 != pr2) 1 else 0 }.map { case (vid, test) => test }.sum assert(notMatching === 0) - //prGraph2.vertices.foreach(println(_)) - val errors = prGraph2.vertices.map { case (vid, pr) => + + val staticErrors = staticRanks2.map { case (vid, pr) => val correct = (vid > 0 && pr == resetProb) || - (vid == 0 && math.abs(pr - (resetProb + (1.0 - resetProb) * (resetProb * (nVertices - 1)) )) < 1.0E-5) - if ( !correct ) { 1 } else { 0 } + (vid == 0 && math.abs(pr - (resetProb + (1.0 - resetProb) * (resetProb * (nVertices - 1)) )) < 1.0E-5) + if (!correct) 1 else 0 } - assert(errors.sum === 0) + assert(staticErrors.sum === 0) - val prGraph3 = PageRank.runUntillConvergence(starGraph, 0, resetProb) - val errors2 = prGraph2.vertices.leftJoin(prGraph3.vertices){ (vid, pr1, pr2Opt) => - pr2Opt match { - case Some(pr2) if(pr1 == pr2) => 0 - case _ => 1 - } - }.map { case (vid, test) => test }.sum - assert(errors2 === 0) + val dynamicRanks = PageRank.runUntillConvergence(starGraph, 0, resetProb).vertices.cache() + val standaloneRanks = PageRank.runStandalone(starGraph, 0, resetProb).cache() + assert(compareRanks(staticRanks2, dynamicRanks) < errorTol) + assert(compareRanks(staticRanks2, standaloneRanks) < errorTol) } } // end of test Star PageRank @@ -87,27 +90,46 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { test("Grid PageRank") { withSpark(new SparkContext("local", "test")) { sc => - val gridGraph = GraphGenerators.gridGraph(sc, 10, 10).cache() + val rows = 10 + val cols = 10 val resetProb = 0.15 - val prGraph1 = PageRank.run(gridGraph, 50, resetProb).cache() - val prGraph2 = PageRank.runUntillConvergence(gridGraph, 0.0001, resetProb).cache() - val error = prGraph1.vertices.zipJoin(prGraph2.vertices) { case (id, a, b) => (a - b) * (a - b) } - .map { case (id, error) => error }.sum - //prGraph1.vertices.zipJoin(prGraph2.vertices) { (id, a, b) => (a, b, a-b) }.foreach(println(_)) - println(error) - assert(error < 1.0e-5) - val pr3: RDD[(Vid, Double)] = sc.parallelize(GridPageRank(10,10, 50, resetProb)) - val error2 = prGraph1.vertices.leftJoin(pr3) { (id, a, bOpt) => - val b: Double = bOpt.get - (a - b) * (a - b) - }.map { case (id, error) => error }.sum - //prGraph1.vertices.leftJoin(pr3) { (id, a, b) => (a, b) }.foreach( println(_) ) - println(error2) - assert(error2 < 1.0e-5) + val tol = 0.0001 + val numIter = 50 + val errorTol = 1.0e-5 + val gridGraph = GraphGenerators.gridGraph(sc, rows, cols).cache() + + val staticRanks = PageRank.run(gridGraph, numIter, resetProb).vertices.cache() + val dynamicRanks = PageRank.runUntillConvergence(gridGraph, tol, resetProb).vertices.cache() + val standaloneRanks = PageRank.runStandalone(gridGraph, tol, resetProb).cache() + val referenceRanks = VertexRDD(sc.parallelize(GridPageRank(rows, cols, numIter, resetProb))) + + assert(compareRanks(staticRanks, referenceRanks) < errorTol) + assert(compareRanks(dynamicRanks, referenceRanks) < errorTol) + assert(compareRanks(standaloneRanks, referenceRanks) < errorTol) } } // end of Grid PageRank + test("Chain PageRank") { + withSpark(new SparkContext("local", "test")) { sc => + val chain1 = (0 until 9).map(x => (x, x+1) ) + val rawEdges = sc.parallelize(chain1, 1).map { case (s,d) => (s.toLong, d.toLong) } + val chain = Graph.fromEdgeTuples(rawEdges, 1.0).cache() + val resetProb = 0.15 + val tol = 0.0001 + val numIter = 10 + val errorTol = 1.0e-5 + + val staticRanks = PageRank.run(chain, numIter, resetProb).vertices.cache() + val dynamicRanks = PageRank.runUntillConvergence(chain, tol, resetProb).vertices.cache() + val standaloneRanks = PageRank.runStandalone(chain, tol, resetProb).cache() + + assert(compareRanks(staticRanks, dynamicRanks) < errorTol) + assert(compareRanks(dynamicRanks, standaloneRanks) < errorTol) + } + } + + test("Grid Connected Components") { withSpark(new SparkContext("local", "test")) { sc => val gridGraph = GraphGenerators.gridGraph(sc, 10, 10).cache() @@ -167,7 +189,6 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { } } val ccMap = vertices.toMap - println(ccMap) for ( id <- 0 until 20 ) { if (id < 10) { assert(ccMap(id) === 0) From 3f69cdc81b21f59024477b74571cfb683a0d3ca6 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Thu, 12 Dec 2013 15:43:12 -0800 Subject: [PATCH 271/531] Use standalone PageRank in Analytics --- .../org/apache/spark/graph/Analytics.scala | 31 +++++-------------- 1 file changed, 7 insertions(+), 24 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala index 755809b4b96e9..ac50e9a3883cc 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala @@ -54,8 +54,6 @@ object Analytics extends Logging { taskType match { case "pagerank" => { - var numIter = Int.MaxValue - var isDynamic = false var tol:Float = 0.001F var outFname = "" var numVPart = 4 @@ -63,8 +61,6 @@ object Analytics extends Logging { var partitionStrategy: PartitionStrategy = RandomVertexCut options.foreach{ - case ("numIter", v) => numIter = v.toInt - case ("dynamic", v) => isDynamic = v.toBoolean case ("tol", v) => tol = v.toFloat case ("output", v) => outFname = v case ("numVPart", v) => numVPart = v.toInt @@ -73,40 +69,27 @@ object Analytics extends Logging { case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) } - if(!isDynamic && numIter == Int.MaxValue) { - println("Set number of iterations!") - sys.exit(1) - } println("======================================") println("| PageRank |") - println("--------------------------------------") - println(" Using parameters:") - println(" \tDynamic: " + isDynamic) - if(isDynamic) println(" \t |-> Tolerance: " + tol) - println(" \tNumIter: " + numIter) println("======================================") val sc = new SparkContext(host, "PageRank(" + fname + ")") val graph = GraphLoader.edgeListFile(sc, fname, - minEdgePartitions = numEPart, partitionStrategy=partitionStrategy).cache() + minEdgePartitions = numEPart, partitionStrategy = partitionStrategy).cache() - val startTime = System.currentTimeMillis - println("GRAPHX: starting tasks") println("GRAPHX: Number of vertices " + graph.vertices.count) println("GRAPHX: Number of edges " + graph.edges.count) //val pr = Analytics.pagerank(graph, numIter) - val pr = if(isDynamic) PageRank.runUntillConvergence(graph, tol, numIter) - else PageRank.run(graph, numIter) - println("GRAPHX: Total rank: " + pr.vertices.map{ case (id,r) => r }.reduce(_+_) ) + val pr = PageRank.runStandalone(graph, tol) + + println("GRAPHX: Total rank: " + pr.map(_._2).reduce(_+_)) + if (!outFname.isEmpty) { - println("Saving pageranks of pages to " + outFname) - pr.vertices.map{case (id, r) => id + "\t" + r}.saveAsTextFile(outFname) + logWarning("Saving pageranks of pages to " + outFname) + pr.map{case (id, r) => id + "\t" + r}.saveAsTextFile(outFname) } - println("GRAPHX: Runtime: " + ((System.currentTimeMillis - startTime)/1000.0) + " seconds") - - Thread.sleep(100000) sc.stop() } From 5b0d6f0ad52f3d27eb9001a6f7d88c3281e93014 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Sat, 7 Dec 2013 18:19:30 -0800 Subject: [PATCH 272/531] Remove static Pregel; take maxIters in dynamic Pregel --- .../scala/org/apache/spark/graph/Pregel.scala | 85 ++----------------- 1 file changed, 5 insertions(+), 80 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala index 8e9ca89efcf36..9fb1d3fd8cbb0 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala @@ -11,10 +11,6 @@ package org.apache.spark.graph * execution while also exposing greater flexibility for graph based * computation. * - * This object present several variants of the bulk synchronous - * execution that differ only in the edge direction along which - * messages are sent and whether a fixed number of iterations is used. - * * @example We can use the Pregel abstraction to implement PageRank * {{{ * val pagerankGraph: Graph[Double, Double] = graph @@ -41,7 +37,6 @@ package org.apache.spark.graph */ object Pregel { - /** * Execute a Pregel-like iterative vertex-parallel abstraction. The * user-defined vertex-program `vprog` is executed in parallel on @@ -56,7 +51,8 @@ object Pregel { * on subsequent iterations if a vertex does not receive a message * then the vertex-program is not invoked. * - * This function iterates a fixed number (`numIter`) of iterations. + * 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 @@ -67,78 +63,7 @@ object Pregel { * @param initialMsg the message each vertex will receive at the on * the first iteration. * - * @param numIter the number of iterations to run this computation. - * - * @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 apply[VD: ClassManifest, ED: ClassManifest, A: ClassManifest] - (graph: Graph[VD, ED], initialMsg: A, numIter: Int)( - vprog: (Vid, VD, A) => VD, - sendMsg: EdgeTriplet[VD, ED] => Iterator[(Vid,A)], - mergeMsg: (A, A) => A) - : Graph[VD, ED] = { - - // Receive the first set of messages - var g = graph.mapVertices( (vid, vdata) => vprog(vid, vdata, initialMsg)).cache() - - var i = 0 - while (i < numIter) { - // compute the messages - val messages = g.mapReduceTriplets(sendMsg, mergeMsg) // broadcast & aggregation - // receive the messages - val changedVerts = g.vertices.deltaJoin(messages)(vprog).cache() // updating the vertices - // replicate the changed vertices - g = g.deltaJoinVertices(changedVerts) - // count the iteration - i += 1 - } - - // Return the final graph - g - } // end of apply - - - /** - * 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. - * - * @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 vprog the user-defined vertex program which runs on each * vertex and receives the inbound message and computes a new vertex @@ -160,7 +85,7 @@ object Pregel { * */ def apply[VD: ClassManifest, ED: ClassManifest, A: ClassManifest] - (graph: Graph[VD, ED], initialMsg: A)( + (graph: Graph[VD, ED], initialMsg: A, maxIterations: Int = Int.MaxValue)( vprog: (Vid, VD, A) => VD, sendMsg: EdgeTriplet[VD, ED] => Iterator[(Vid,A)], mergeMsg: (A, A) => A) @@ -180,7 +105,7 @@ object Pregel { var activeMessages = messages.count() // Loop var i = 0 - while (activeMessages > 0) { + while (activeMessages > 0 && i < maxIterations) { // receive the messages val changedVerts = g.vertices.deltaJoin(messages)(vprog).cache() // updating the vertices // replicate the changed vertices From 1e98840128f3cffbe6566b384e742b3a52cdaa9f Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 6 Dec 2013 22:32:47 -0800 Subject: [PATCH 273/531] Load edges in columnar format In GraphLoader.edgeListFile, load edges directly into EdgePartitions, avoiding repartitioning. --- .../org/apache/spark/graph/EdgeRDD.scala | 11 ++- .../org/apache/spark/graph/GraphLoader.scala | 55 ++++++++---- .../apache/spark/graph/impl/GraphImpl.scala | 88 ++++++++++++------- 3 files changed, 101 insertions(+), 53 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala b/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala index 24844262bc1a9..a34113b1eb5a5 100644 --- a/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala @@ -1,6 +1,7 @@ package org.apache.spark.graph +import org.apache.spark.Partitioner import org.apache.spark.{TaskContext, Partition, OneToOneDependency} import org.apache.spark.graph.impl.EdgePartition import org.apache.spark.rdd.RDD @@ -13,10 +14,16 @@ class EdgeRDD[@specialized ED: ClassManifest]( partitionsRDD.setName("EdgeRDD") - override val partitioner = partitionsRDD.partitioner - override protected def getPartitions: Array[Partition] = partitionsRDD.partitions + /** + * If partitionsRDD already has a partitioner, use it. Otherwise assume that the Pids in + * partitionsRDD correspond to the actual partitions and create a new partitioner that allows + * co-partitioning with partitionsRDD. + */ + override val partitioner = + partitionsRDD.partitioner.orElse(Some(Partitioner.defaultPartitioner(partitionsRDD))) + override def compute(split: Partition, context: TaskContext): Iterator[Edge[ED]] = { val edgePartition = partitionsRDD.compute(split, context).next()._2 edgePartition.iterator diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala b/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala index 29d14452de833..b00c7c4afef68 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala @@ -1,9 +1,12 @@ package org.apache.spark.graph -import org.apache.spark.SparkContext +import java.util.{Arrays => JArrays} +import org.apache.spark.{Logging, SparkContext} +import org.apache.spark.graph.impl.{EdgePartition, GraphImpl} +import org.apache.spark.util.collection.PrimitiveVector -object GraphLoader { +object GraphLoader extends Logging { /** * Load an edge list from file initializing the Graph @@ -77,24 +80,42 @@ object GraphLoader { minEdgePartitions: Int = 1, partitionStrategy: PartitionStrategy = RandomVertexCut): Graph[Int, Int] = { + val startTime = System.currentTimeMillis + // Parse the edge data table - val edges = sc.textFile(path, minEdgePartitions).mapPartitions( iter => - iter.filter(line => !line.isEmpty && line(0) != '#').map { line => - val lineArray = line.split("\\s+") - if(lineArray.length < 2) { - println("Invalid line: " + line) - assert(false) + val edges = sc.textFile(path, minEdgePartitions).mapPartitionsWithIndex { (index, iter) => + val srcIds = new PrimitiveVector[Long] + val dstIds = new PrimitiveVector[Long] + iter.foreach { line => + if (!line.isEmpty && line(0) != '#') { + val lineArray = line.split("\\s+") + if (lineArray.length < 2) { + logWarning("Invalid line: " + line) + } + val srcId = lineArray(0).toLong + val dstId = lineArray(1).toLong + if (canonicalOrientation && dstId > srcId) { + srcIds += dstId + dstIds += srcId + } else { + srcIds += srcId + dstIds += dstId + } } - val source = lineArray(0).trim.toLong - val target = lineArray(1).trim.toLong - if (canonicalOrientation && target > source) { - Edge(target, source, 1) - } else { - Edge(source, target, 1) - } - }) + } + val srcIdArray = srcIds.trim().array + val dstIdArray = dstIds.trim().array + val data = new Array[Int](srcIdArray.length) + JArrays.fill(data, 1) + + Iterator((index, new EdgePartition[Int](srcIdArray, dstIdArray, data))) + }.cache() + edges.count() + + logInfo("It took %d ms to load the edges".format(System.currentTimeMillis - startTime)) + val defaultVertexAttr = 1 - Graph.fromEdges(edges, defaultVertexAttr, partitionStrategy) + GraphImpl.fromEdgePartitions(edges, defaultVertexAttr, partitionStrategy) } // end of edgeListFile } diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index 771c460345655..1e17fd5a67fc4 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -253,25 +253,11 @@ object GraphImpl { def apply[VD: ClassManifest, ED: ClassManifest]( edges: RDD[Edge[ED]], - defaultValue: VD, + defaultVertexAttr: VD, partitionStrategy: PartitionStrategy): GraphImpl[VD, ED] = { val etable = createETable(edges, partitionStrategy).cache() - - // Get the set of all vids - val vids = etable.flatMap { e => - Iterator((e.srcId, 0), (e.dstId, 0)) - } - - // Shuffle the vids and create the VertexRDD. - // TODO: Consider doing map side distinct before shuffle. - val shuffled = new ShuffledRDD[Vid, Int, (Vid, Int)]( - vids, new HashPartitioner(edges.partitions.size)) - shuffled.setSerializer(classOf[VidMsgSerializer].getName) - val vtable = VertexRDD(shuffled.mapValues(x => defaultValue)) - - val vertexPlacement = new VertexPlacement(etable, vtable) - new GraphImpl(vtable, etable, vertexPlacement) + fromEdgeRDD(etable, defaultVertexAttr) } def apply[VD: ClassManifest, ED: ClassManifest]( @@ -303,6 +289,14 @@ object GraphImpl { new GraphImpl(vtable, etable, vertexPlacement) } + def fromEdgePartitions[VD: ClassManifest, ED: ClassManifest]( + edges: RDD[(Pid, EdgePartition[ED])], + defaultVertexAttr: VD, + partitionStrategy: PartitionStrategy): GraphImpl[VD, ED] = { + val etable = createETableFromEdgePartitions(edges, partitionStrategy) + fromEdgeRDD(etable, defaultVertexAttr) + } + /** * Create the edge table RDD, which is much more efficient for Java heap storage than the * normal edges data structure (RDD[(Vid, Vid, ED)]). @@ -313,29 +307,55 @@ object GraphImpl { */ protected def createETable[ED: ClassManifest]( edges: RDD[Edge[ED]], - partitionStrategy: PartitionStrategy): EdgeRDD[ED] = { - // Get the number of partitions - val numPartitions = edges.partitions.size + partitionStrategy: PartitionStrategy): EdgeRDD[ED] = { + // Get the number of partitions + val numPartitions = edges.partitions.size - val eTable = edges.map { e => - val part: Pid = partitionStrategy.getPartition(e.srcId, e.dstId, numPartitions) + val eTable = edges.map { e => + val part: Pid = partitionStrategy.getPartition(e.srcId, e.dstId, numPartitions) - // Should we be using 3-tuple or an optimized class - new MessageToPartition(part, (e.srcId, e.dstId, e.attr)) - } - .partitionBy(new HashPartitioner(numPartitions)) - .mapPartitionsWithIndex( { (pid, iter) => - val builder = new EdgePartitionBuilder[ED] - iter.foreach { message => - val data = message.data - builder.add(data._1, data._2, data._3) - } - val edgePartition = builder.toEdgePartition - Iterator((pid, edgePartition)) - }, preservesPartitioning = true).cache() + // Should we be using 3-tuple or an optimized class + new MessageToPartition(part, (e.srcId, e.dstId, e.attr)) + } + .partitionBy(new HashPartitioner(numPartitions)) + .mapPartitionsWithIndex( { (pid, iter) => + val builder = new EdgePartitionBuilder[ED] + iter.foreach { message => + val data = message.data + builder.add(data._1, data._2, data._3) + } + val edgePartition = builder.toEdgePartition + Iterator((pid, edgePartition)) + }, preservesPartitioning = true).cache() new EdgeRDD(eTable) } + protected def createETableFromEdgePartitions[ED: ClassManifest]( + edges: RDD[(Pid, EdgePartition[ED])], + partitionStrategy: PartitionStrategy): EdgeRDD[ED] = { + // TODO(ankurdave): provide option to repartition edges using partitionStrategy + new EdgeRDD(edges) + } + + private def fromEdgeRDD[VD: ClassManifest, ED: ClassManifest]( + edges: EdgeRDD[ED], + defaultVertexAttr: VD): GraphImpl[VD, ED] = { + // Get the set of all vids + val vids = edges.flatMap { e => + Iterator((e.srcId, 0), (e.dstId, 0)) + } + + // Shuffle the vids and create the VertexRDD. + // TODO: Consider doing map side distinct before shuffle. + val shuffled = new ShuffledRDD[Vid, Int, (Vid, Int)]( + vids, new HashPartitioner(edges.partitions.size)) + shuffled.setSerializer(classOf[VidMsgSerializer].getName) + val vtable = VertexRDD(shuffled.mapValues(x => defaultVertexAttr)) + + val vertexPlacement = new VertexPlacement(edges, vtable) + new GraphImpl(vtable, edges, vertexPlacement) + } + private def accessesVertexAttr[VD, ED](closure: AnyRef, attrName: String): Boolean = { try { BytecodeUtils.invokedMethod(closure, classOf[EdgeTriplet[VD, ED]], attrName) From a8c7ebf0edc8161debc3f84b0008638321ddd568 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Sat, 7 Dec 2013 15:10:32 -0800 Subject: [PATCH 274/531] Don't partition edges by default; refactor Instead, expose Graph.partitionBy(PartitionStrategy). --- .../org/apache/spark/graph/Analytics.scala | 82 +++++----- .../org/apache/spark/graph/EdgeRDD.scala | 4 + .../scala/org/apache/spark/graph/Graph.scala | 52 +++--- .../org/apache/spark/graph/GraphLoader.scala | 34 ++-- .../graph/algorithms/TriangleCount.scala | 4 +- .../apache/spark/graph/impl/GraphImpl.scala | 148 +++++++++--------- .../apache/spark/graph/AnalyticsSuite.scala | 2 +- 7 files changed, 162 insertions(+), 164 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala index ac50e9a3883cc..2012dadb2f305 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala @@ -58,14 +58,14 @@ object Analytics extends Logging { var outFname = "" var numVPart = 4 var numEPart = 4 - var partitionStrategy: PartitionStrategy = RandomVertexCut + 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 = pickPartitioner(v) + case ("partStrategy", v) => partitionStrategy = Some(pickPartitioner(v)) case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) } @@ -75,8 +75,9 @@ object Analytics extends Logging { val sc = new SparkContext(host, "PageRank(" + fname + ")") - val graph = GraphLoader.edgeListFile(sc, fname, - minEdgePartitions = numEPart, partitionStrategy = partitionStrategy).cache() + val unpartitionedGraph = GraphLoader.edgeListFile(sc, fname, + minEdgePartitions = numEPart).cache() + val graph = partitionStrategy.foldLeft(unpartitionedGraph)(_.partitionBy(_)) println("GRAPHX: Number of vertices " + graph.vertices.count) println("GRAPHX: Number of edges " + graph.edges.count) @@ -96,44 +97,47 @@ object Analytics extends Logging { case "cc" => { - var numIter = Int.MaxValue - var numVPart = 4 - var numEPart = 4 - var isDynamic = false - var partitionStrategy: PartitionStrategy = RandomVertexCut - - options.foreach{ - case ("numIter", v) => numIter = v.toInt - case ("dynamic", v) => isDynamic = v.toBoolean - 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) - } - - 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 + ")") - val graph = GraphLoader.edgeListFile(sc, fname, - minEdgePartitions = numEPart, partitionStrategy=partitionStrategy).cache() - val cc = ConnectedComponents.run(graph) - println("Components: " + cc.vertices.map{ case (vid,data) => data}.distinct()) - sc.stop() - } + var numIter = Int.MaxValue + var numVPart = 4 + var numEPart = 4 + var isDynamic = false + var partitionStrategy: Option[PartitionStrategy] = None + + options.foreach{ + case ("numIter", v) => numIter = v.toInt + case ("dynamic", v) => isDynamic = v.toBoolean + case ("numEPart", v) => numEPart = v.toInt + case ("numVPart", v) => numVPart = v.toInt + case ("partStrategy", v) => partitionStrategy = Some(pickPartitioner(v)) + case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) + } + + 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 + ")") + val unpartitionedGraph = GraphLoader.edgeListFile(sc, fname, + minEdgePartitions = numEPart).cache() + val graph = partitionStrategy.foldLeft(unpartitionedGraph)(_.partitionBy(_)) + + val cc = ConnectedComponents.run(graph) + println("Components: " + cc.vertices.map{ case (vid,data) => data}.distinct()) + sc.stop() + } case "triangles" => { var numVPart = 4 var numEPart = 4 + // TriangleCount requires the graph to be partitioned var partitionStrategy: PartitionStrategy = RandomVertexCut options.foreach{ @@ -147,7 +151,7 @@ object Analytics extends Logging { println("--------------------------------------") val sc = new SparkContext(host, "TriangleCount(" + fname + ")") val graph = GraphLoader.edgeListFile(sc, fname, canonicalOrientation = true, - minEdgePartitions = numEPart, partitionStrategy=partitionStrategy).cache() + minEdgePartitions = numEPart).partitionBy(partitionStrategy).cache() val triangles = TriangleCount.run(graph) println("Triangles: " + triangles.vertices.map { case (vid,data) => data.toLong diff --git a/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala b/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala index a34113b1eb5a5..cbc3ca54c27e0 100644 --- a/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala @@ -64,4 +64,8 @@ class EdgeRDD[@specialized ED: ClassManifest]( } } + def collectVids(): RDD[Vid] = { + partitionsRDD.flatMap { case (_, p) => Array.concat(p.srcIds, p.dstIds) } + } + } diff --git a/graph/src/main/scala/org/apache/spark/graph/Graph.scala b/graph/src/main/scala/org/apache/spark/graph/Graph.scala index a0907c319aac6..d31d9dead0abb 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Graph.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Graph.scala @@ -85,6 +85,11 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { */ def cache(): Graph[VD, ED] + /** + * Repartition the edges in the graph according to partitionStrategy. + */ + def partitionBy(partitionStrategy: PartitionStrategy): Graph[VD, ED] + /** * Compute statistics describing the graph representation. */ @@ -193,18 +198,15 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { vpred: (Vid, VD) => Boolean = ((v,d) => true) ): Graph[VD, ED] /** - * This function merges multiple edges between two vertices into a - * single Edge. See - * [[org.apache.spark.graph.Graph.groupEdgeTriplets]] for more - * detail. + * This function 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 f the user supplied commutative associative function to merge edge attributes for + * duplicate edges. * - * @return Graph[VD,ED2] The resulting graph with a single Edge for - * each source, dest vertex pair. + * @return Graph[VD,ED2] The resulting graph with a single Edge for each source, dest vertex pair. */ def groupEdges(merge: (ED, ED) => ED): Graph[VD,ED] @@ -294,26 +296,26 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { object Graph { /** - * Construct a graph from a collection of edges encoded as vertex id - * pairs. + * Construct a graph from a collection of edges encoded as vertex id pairs. * * @param rawEdges a collection of edges in (src,dst) form. - * @param uniqueEdges if multiple identical edges are found they are - * combined and the edge attribute is set to the sum. Otherwise - * duplicate edges are treated as separate. + * @param uniqueEdges if multiple identical edges are found they are combined and the edge + * attribute is set to the sum. Otherwise duplicate edges are treated as separate. To enable + * uniqueEdges, a [[PartitionStrategy]] must be provided. * - * @return a graph with edge attributes containing either the count - * of duplicate edges or 1 (if `uniqueEdges=false`) and vertex - * attributes containing the total degree of each vertex. + * @return a graph with edge attributes containing either the count of duplicate edges or 1 + * (if `uniqueEdges=None`) and vertex attributes containing the total degree of each vertex. */ def fromEdgeTuples[VD: ClassManifest]( rawEdges: RDD[(Vid, Vid)], defaultValue: VD, - uniqueEdges: Boolean = false, - partitionStrategy: PartitionStrategy = RandomVertexCut): 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, partitionStrategy) - if (uniqueEdges) graph.groupEdges((a, b) => a + b) else graph + val graph = GraphImpl(edges, defaultValue) + uniqueEdges match { + case Some(p) => graph.partitionBy(p).groupEdges((a, b) => a + b) + case None => graph + } } /** @@ -327,9 +329,8 @@ object Graph { */ def fromEdges[VD: ClassManifest, ED: ClassManifest]( edges: RDD[Edge[ED]], - defaultValue: VD, - partitionStrategy: PartitionStrategy = RandomVertexCut): Graph[VD, ED] = { - GraphImpl(edges, defaultValue, partitionStrategy) + defaultValue: VD): Graph[VD, ED] = { + GraphImpl(edges, defaultValue) } /** @@ -350,9 +351,8 @@ object Graph { def apply[VD: ClassManifest, ED: ClassManifest]( vertices: RDD[(Vid, VD)], edges: RDD[Edge[ED]], - defaultVertexAttr: VD = null.asInstanceOf[VD], - partitionStrategy: PartitionStrategy = RandomVertexCut): Graph[VD, ED] = { - GraphImpl(vertices, edges, defaultVertexAttr, partitionStrategy) + defaultVertexAttr: VD = null.asInstanceOf[VD]): Graph[VD, ED] = { + GraphImpl(vertices, edges, defaultVertexAttr) } /** diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala b/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala index b00c7c4afef68..a69bfde5322df 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala @@ -1,6 +1,7 @@ package org.apache.spark.graph import java.util.{Arrays => JArrays} +import org.apache.spark.graph.impl.EdgePartitionBuilder import org.apache.spark.{Logging, SparkContext} import org.apache.spark.graph.impl.{EdgePartition, GraphImpl} import org.apache.spark.util.collection.PrimitiveVector @@ -25,8 +26,7 @@ object GraphLoader extends Logging { sc: SparkContext, path: String, edgeParser: Array[String] => ED, - minEdgePartitions: Int = 1, - partitionStrategy: PartitionStrategy = RandomVertexCut): + minEdgePartitions: Int = 1): Graph[Int, ED] = { // Parse the edge data table val edges = sc.textFile(path, minEdgePartitions).mapPartitions( iter => @@ -43,7 +43,7 @@ object GraphLoader extends Logging { Edge(source, target, edata) }) val defaultVertexAttr = 1 - Graph.fromEdges(edges, defaultVertexAttr, partitionStrategy) + Graph.fromEdges(edges, defaultVertexAttr) } /** @@ -73,19 +73,17 @@ object GraphLoader extends Logging { * @tparam ED * @return */ - def edgeListFile[ED: ClassManifest]( + def edgeListFile( sc: SparkContext, path: String, canonicalOrientation: Boolean = false, - minEdgePartitions: Int = 1, - partitionStrategy: PartitionStrategy = RandomVertexCut): + minEdgePartitions: Int = 1): Graph[Int, Int] = { val startTime = System.currentTimeMillis - // Parse the edge data table - val edges = sc.textFile(path, minEdgePartitions).mapPartitionsWithIndex { (index, iter) => - val srcIds = new PrimitiveVector[Long] - val dstIds = new PrimitiveVector[Long] + // Parse the edge data table directly into edge partitions + val edges = sc.textFile(path, minEdgePartitions).mapPartitionsWithIndex { (pid, iter) => + val builder = new EdgePartitionBuilder[Int] iter.foreach { line => if (!line.isEmpty && line(0) != '#') { val lineArray = line.split("\\s+") @@ -95,27 +93,19 @@ object GraphLoader extends Logging { val srcId = lineArray(0).toLong val dstId = lineArray(1).toLong if (canonicalOrientation && dstId > srcId) { - srcIds += dstId - dstIds += srcId + builder.add(dstId, srcId, 1) } else { - srcIds += srcId - dstIds += dstId + builder.add(srcId, dstId, 1) } } } - val srcIdArray = srcIds.trim().array - val dstIdArray = dstIds.trim().array - val data = new Array[Int](srcIdArray.length) - JArrays.fill(data, 1) - - Iterator((index, new EdgePartition[Int](srcIdArray, dstIdArray, data))) + Iterator((pid, builder.toEdgePartition)) }.cache() edges.count() logInfo("It took %d ms to load the edges".format(System.currentTimeMillis - startTime)) - val defaultVertexAttr = 1 - GraphImpl.fromEdgePartitions(edges, defaultVertexAttr, partitionStrategy) + GraphImpl.fromEdgePartitions(edges, defaultVertexAttr = 1) } // end of edgeListFile } diff --git a/graph/src/main/scala/org/apache/spark/graph/algorithms/TriangleCount.scala b/graph/src/main/scala/org/apache/spark/graph/algorithms/TriangleCount.scala index 963986d20dcde..b1cd3c47d0cf1 100644 --- a/graph/src/main/scala/org/apache/spark/graph/algorithms/TriangleCount.scala +++ b/graph/src/main/scala/org/apache/spark/graph/algorithms/TriangleCount.scala @@ -16,7 +16,9 @@ object TriangleCount { * triangle is counted twice. * * - * @param graph a graph with `sourceId` less than `destId` + * @param graph a graph with `sourceId` less than `destId`. The graph must have been partitioned + * using Graph.partitionBy. + * * @return */ def run[VD: ClassManifest, ED: ClassManifest](graph: Graph[VD,ED]): Graph[Int, ED] = { diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index 1e17fd5a67fc4..fea8cfa712710 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -1,5 +1,6 @@ package org.apache.spark.graph.impl +import org.apache.spark.util.collection.PrimitiveVector import org.apache.spark.{HashPartitioner, Partitioner} import org.apache.spark.SparkContext._ import org.apache.spark.graph._ @@ -40,6 +41,12 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( this(vertices, edges, vertexPlacement, new VTableReplicated(vertices, edges, vertexPlacement)) } + def this( + vertices: VertexRDD[VD], + edges: EdgeRDD[ED]) = { + this(vertices, edges, new VertexPlacement(edges, vertices)) + } + /** Return a RDD that brings edges with its source and destination vertices together. */ @transient override val triplets: RDD[EdgeTriplet[VD, ED]] = { val vdManifest = classManifest[VD] @@ -60,6 +67,27 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( override def cache(): Graph[VD, ED] = persist(StorageLevel.MEMORY_ONLY) + override def partitionBy(partitionStrategy: PartitionStrategy): Graph[VD, ED] = { + val numPartitions = edges.partitions.size + val newEdges = new EdgeRDD(edges.map { e => + val part: Pid = partitionStrategy.getPartition(e.srcId, e.dstId, numPartitions) + + // Should we be using 3-tuple or an optimized class + new MessageToPartition(part, (e.srcId, e.dstId, e.attr)) + } + .partitionBy(new HashPartitioner(numPartitions)) + .mapPartitionsWithIndex( { (pid, iter) => + val builder = new EdgePartitionBuilder[ED] + iter.foreach { message => + val data = message.data + builder.add(data._1, data._2, data._3) + } + val edgePartition = builder.toEdgePartition + Iterator((pid, edgePartition)) + }, preservesPartitioning = true).cache()) + new 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[Vid]]]): Double = { @@ -175,10 +203,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( Iterator((pid, edgePartition)) }, preservesPartitioning = true)).cache() - // Construct the VertexPlacement map - val newVertexPlacement = new VertexPlacement(newETable, newVTable) - - new GraphImpl(newVTable, newETable, newVertexPlacement) + new GraphImpl(newVTable, newETable) } // end of subgraph override def groupEdges(merge: (ED, ED) => ED): Graph[VD, ED] = { @@ -246,6 +271,14 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( changedVerts, edges, vertexPlacement, Some(vTableReplicated)) new GraphImpl(newVerts, edges, vertexPlacement, newVTableReplicated) } + + private def accessesVertexAttr[VD, ED](closure: AnyRef, attrName: String): Boolean = { + try { + BytecodeUtils.invokedMethod(closure, classOf[EdgeTriplet[VD, ED]], attrName) + } catch { + case _: ClassNotFoundException => true // if we don't know, be conservative + } + } } // end of class GraphImpl @@ -253,48 +286,35 @@ object GraphImpl { def apply[VD: ClassManifest, ED: ClassManifest]( edges: RDD[Edge[ED]], - defaultVertexAttr: VD, - partitionStrategy: PartitionStrategy): GraphImpl[VD, ED] = + defaultVertexAttr: VD): GraphImpl[VD, ED] = { - val etable = createETable(edges, partitionStrategy).cache() - fromEdgeRDD(etable, defaultVertexAttr) + fromEdgeRDD(createETable(edges), defaultVertexAttr) + } + + def fromEdgePartitions[VD: ClassManifest, ED: ClassManifest]( + edges: RDD[(Pid, EdgePartition[ED])], + defaultVertexAttr: VD): GraphImpl[VD, ED] = { + fromEdgeRDD(createETableFromEdgePartitions(edges), defaultVertexAttr) } def apply[VD: ClassManifest, ED: ClassManifest]( vertices: RDD[(Vid, VD)], edges: RDD[Edge[ED]], - defaultVertexAttr: VD, - partitionStrategy: PartitionStrategy): GraphImpl[VD, ED] = + defaultVertexAttr: VD): GraphImpl[VD, ED] = { - vertices.cache() - val etable = createETable(edges, partitionStrategy).cache() + val etable = createETable(edges).cache() + // Get the set of all vids val partitioner = Partitioner.defaultPartitioner(vertices) - val vPartitioned = vertices.partitionBy(partitioner) - - val vidsFromEdges = { - etable.partitionsRDD.flatMap { case (_, p) => Array.concat(p.srcIds, p.dstIds) } - .map(vid => (vid, 0)) - .partitionBy(partitioner) - } - + val vidsFromEdges = collectVidsFromEdges(etable, partitioner) val vids = vPartitioned.zipPartitions(vidsFromEdges) { (vertexIter, vidsFromEdgesIter) => vertexIter.map(_._1) ++ vidsFromEdgesIter.map(_._1) } val vtable = VertexRDD(vids, vPartitioned, defaultVertexAttr) - val vertexPlacement = new VertexPlacement(etable, vtable) - new GraphImpl(vtable, etable, vertexPlacement) - } - - def fromEdgePartitions[VD: ClassManifest, ED: ClassManifest]( - edges: RDD[(Pid, EdgePartition[ED])], - defaultVertexAttr: VD, - partitionStrategy: PartitionStrategy): GraphImpl[VD, ED] = { - val etable = createETableFromEdgePartitions(edges, partitionStrategy) - fromEdgeRDD(etable, defaultVertexAttr) + new GraphImpl(vtable, etable) } /** @@ -305,63 +325,41 @@ object GraphImpl { * key-value pair: the key is the partition id, and the value is an EdgePartition object * containing all the edges in a partition. */ - protected def createETable[ED: ClassManifest]( - edges: RDD[Edge[ED]], - partitionStrategy: PartitionStrategy): EdgeRDD[ED] = { - // Get the number of partitions - val numPartitions = edges.partitions.size - - val eTable = edges.map { e => - val part: Pid = partitionStrategy.getPartition(e.srcId, e.dstId, numPartitions) - - // Should we be using 3-tuple or an optimized class - new MessageToPartition(part, (e.srcId, e.dstId, e.attr)) + private def createETable[ED: ClassManifest]( + edges: RDD[Edge[ED]]): EdgeRDD[ED] = { + val eTable = edges.mapPartitionsWithIndex { (pid, iter) => + val builder = new EdgePartitionBuilder[ED] + iter.foreach { e => + builder.add(e.srcId, e.dstId, e.attr) + } + Iterator((pid, builder.toEdgePartition)) } - .partitionBy(new HashPartitioner(numPartitions)) - .mapPartitionsWithIndex( { (pid, iter) => - val builder = new EdgePartitionBuilder[ED] - iter.foreach { message => - val data = message.data - builder.add(data._1, data._2, data._3) - } - val edgePartition = builder.toEdgePartition - Iterator((pid, edgePartition)) - }, preservesPartitioning = true).cache() new EdgeRDD(eTable) } - protected def createETableFromEdgePartitions[ED: ClassManifest]( - edges: RDD[(Pid, EdgePartition[ED])], - partitionStrategy: PartitionStrategy): EdgeRDD[ED] = { - // TODO(ankurdave): provide option to repartition edges using partitionStrategy + private def createETableFromEdgePartitions[ED: ClassManifest]( + edges: RDD[(Pid, EdgePartition[ED])]): EdgeRDD[ED] = { new EdgeRDD(edges) } private def fromEdgeRDD[VD: ClassManifest, ED: ClassManifest]( edges: EdgeRDD[ED], defaultVertexAttr: VD): GraphImpl[VD, ED] = { + edges.cache() // Get the set of all vids - val vids = edges.flatMap { e => - Iterator((e.srcId, 0), (e.dstId, 0)) - } - - // Shuffle the vids and create the VertexRDD. - // TODO: Consider doing map side distinct before shuffle. - val shuffled = new ShuffledRDD[Vid, Int, (Vid, Int)]( - vids, new HashPartitioner(edges.partitions.size)) - shuffled.setSerializer(classOf[VidMsgSerializer].getName) - val vtable = VertexRDD(shuffled.mapValues(x => defaultVertexAttr)) - - val vertexPlacement = new VertexPlacement(edges, vtable) - new GraphImpl(vtable, edges, vertexPlacement) + val vids = collectVidsFromEdges(edges, new HashPartitioner(edges.partitions.size)) + // Create the VertexRDD. + val vtable = VertexRDD(vids.mapValues(x => defaultVertexAttr)) + new GraphImpl(vtable, edges) } - private def accessesVertexAttr[VD, ED](closure: AnyRef, attrName: String): Boolean = { - try { - BytecodeUtils.invokedMethod(closure, classOf[EdgeTriplet[VD, ED]], attrName) - } catch { - case _: ClassNotFoundException => true // if we don't know, be conservative - } + /** Collects all vids mentioned in edges and partitions them by partitioner. */ + private def collectVidsFromEdges( + edges: EdgeRDD[_], + partitioner: Partitioner): RDD[(Vid, Int)] = { + // TODO: Consider doing map side distinct before shuffle. + new ShuffledRDD[Vid, Int, (Vid, Int)]( + edges.collectVids.map(vid => (vid, 0)), partitioner) + .setSerializer(classOf[VidMsgSerializer].getName) } - } // end of object GraphImpl diff --git a/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala b/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala index 2e6b57a8ec267..61cafcd7c35ec 100644 --- a/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala @@ -251,7 +251,7 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { withSpark(new SparkContext("local", "test")) { sc => val rawEdges = sc.parallelize(Array(0L -> 1L, 1L -> 2L, 2L -> 0L) ++ Array(0L -> 1L, 1L -> 2L, 2L -> 0L), 2) - val graph = Graph.fromEdgeTuples(rawEdges, true).cache() + val graph = Graph.fromEdgeTuples(rawEdges, Some(RandomVertexCut)).cache() val triangleCount = TriangleCount.run(graph) val verts = triangleCount.vertices verts.collect.foreach { case (vid, count) => assert(count === 1) } From d00cc8092bf9c09fffedafbe6d040e2f7bc1da5a Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Thu, 12 Dec 2013 17:40:00 -0800 Subject: [PATCH 275/531] Fix argument bug and closure capture --- .../src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala | 3 ++- .../src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index fea8cfa712710..ba2ebe6497934 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -69,6 +69,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( override def partitionBy(partitionStrategy: PartitionStrategy): Graph[VD, ED] = { val numPartitions = edges.partitions.size + val edManifest = classManifest[ED] val newEdges = new EdgeRDD(edges.map { e => val part: Pid = partitionStrategy.getPartition(e.srcId, e.dstId, numPartitions) @@ -77,7 +78,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( } .partitionBy(new HashPartitioner(numPartitions)) .mapPartitionsWithIndex( { (pid, iter) => - val builder = new EdgePartitionBuilder[ED] + val builder = new EdgePartitionBuilder[ED]()(edManifest) iter.foreach { message => val data = message.data builder.add(data._1, data._2, data._3) diff --git a/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala b/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala index 61cafcd7c35ec..b413b4587ea72 100644 --- a/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala @@ -251,7 +251,7 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { withSpark(new SparkContext("local", "test")) { sc => val rawEdges = sc.parallelize(Array(0L -> 1L, 1L -> 2L, 2L -> 0L) ++ Array(0L -> 1L, 1L -> 2L, 2L -> 0L), 2) - val graph = Graph.fromEdgeTuples(rawEdges, Some(RandomVertexCut)).cache() + val graph = Graph.fromEdgeTuples(rawEdges, true, uniqueEdges = Some(RandomVertexCut)).cache() val triangleCount = TriangleCount.run(graph) val verts = triangleCount.vertices verts.collect.foreach { case (vid, count) => assert(count === 1) } From 016cabceca624611b2be33c29649a78aee9c0f39 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Thu, 12 Dec 2013 17:48:24 -0800 Subject: [PATCH 276/531] Clean up imports in EdgeRDD --- graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala b/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala index cbc3ca54c27e0..9a1ebbcdccea4 100644 --- a/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala @@ -1,8 +1,6 @@ package org.apache.spark.graph - -import org.apache.spark.Partitioner -import org.apache.spark.{TaskContext, Partition, OneToOneDependency} +import org.apache.spark.{OneToOneDependency, Partition, Partitioner, TaskContext} import org.apache.spark.graph.impl.EdgePartition import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel From c3e0f01ad91e561190e3aea685006fda8ac9af40 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Thu, 12 Dec 2013 22:12:26 -0800 Subject: [PATCH 277/531] Remove calls to ClosureCleaner in GraphX All Spark RDD methods already clean closures so we don't need to. --- graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala | 6 ++---- graph/src/main/scala/org/apache/spark/graph/GraphOps.scala | 6 ------ graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala | 6 ++---- 3 files changed, 4 insertions(+), 14 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala b/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala index 24844262bc1a9..126379d1ce5fe 100644 --- a/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala @@ -40,20 +40,18 @@ class EdgeRDD[@specialized ED: ClassManifest]( def mapEdgePartitions[ED2: ClassManifest](f: EdgePartition[ED] => EdgePartition[ED2]) : EdgeRDD[ED2]= { - val cleanF = sparkContext.clean(f) new EdgeRDD[ED2](partitionsRDD.mapPartitions({ iter => val (pid, ep) = iter.next() - Iterator(Tuple2(pid, cleanF(ep))) + Iterator(Tuple2(pid, f(ep))) }, preservesPartitioning = true)) } def zipEdgePartitions[T: ClassManifest, U: ClassManifest] (other: RDD[T]) (f: (EdgePartition[ED], Iterator[T]) => Iterator[U]): RDD[U] = { - val cleanF = sparkContext.clean(f) partitionsRDD.zipPartitions(other, preservesPartitioning = true) { (ePartIter, otherIter) => val (_, edgePartition) = ePartIter.next() - cleanF(edgePartition, otherIter) + f(edgePartition, otherIter) } } diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala b/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala index ee74a36c21b71..7b9eb88b7bc7b 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala @@ -2,7 +2,6 @@ package org.apache.spark.graph import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext._ -import org.apache.spark.util.ClosureCleaner import org.apache.spark.SparkException @@ -116,9 +115,6 @@ class GraphOps[VD: ClassManifest, ED: ClassManifest](graph: Graph[VD, ED]) { dir: EdgeDirection) : VertexRDD[A] = { - ClosureCleaner.clean(mapFunc) - ClosureCleaner.clean(reduceFunc) - // Define a new map function over edge triplets val mf = (et: EdgeTriplet[VD,ED]) => { // Compute the message to the dst vertex @@ -140,7 +136,6 @@ class GraphOps[VD: ClassManifest, ED: ClassManifest](graph: Graph[VD, ED]) { } } - ClosureCleaner.clean(mf) graph.mapReduceTriplets(mf, reduceFunc) } // end of aggregateNeighbors @@ -233,7 +228,6 @@ class GraphOps[VD: ClassManifest, ED: ClassManifest](graph: Graph[VD, ED]) { */ def joinVertices[U: ClassManifest](table: RDD[(Vid, U)])(mapFunc: (Vid, VD, U) => VD) : Graph[VD, ED] = { - ClosureCleaner.clean(mapFunc) val uf = (id: Vid, data: VD, o: Option[U]) => { o match { case Some(u) => mapFunc(id, data, u) diff --git a/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala b/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala index 33e0b892fb191..f66a9ff0ad137 100644 --- a/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala @@ -113,8 +113,7 @@ class VertexRDD[@specialized VD: ClassManifest]( */ def mapVertexPartitions[VD2: ClassManifest](f: VertexPartition[VD] => VertexPartition[VD2]) : VertexRDD[VD2] = { - val cleanF = sparkContext.clean(f) - val newPartitionsRDD = partitionsRDD.mapPartitions(_.map(cleanF), preservesPartitioning = true) + val newPartitionsRDD = partitionsRDD.mapPartitions(_.map(f), preservesPartitioning = true) new VertexRDD(newPartitionsRDD) } @@ -125,13 +124,12 @@ class VertexRDD[@specialized VD: ClassManifest]( private def zipVertexPartitions[VD2: ClassManifest, VD3: ClassManifest] (other: VertexRDD[VD2]) (f: (VertexPartition[VD], VertexPartition[VD2]) => VertexPartition[VD3]): VertexRDD[VD3] = { - val cleanF = sparkContext.clean(f) val newPartitionsRDD = partitionsRDD.zipPartitions( other.partitionsRDD, preservesPartitioning = true ) { (thisIter, otherIter) => val thisPart = thisIter.next() val otherPart = otherIter.next() - Iterator(cleanF(thisPart, otherPart)) + Iterator(f(thisPart, otherPart)) } new VertexRDD(newPartitionsRDD) } From 87f290956174052037ddab3e68e7f31faf58a5a2 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Thu, 12 Dec 2013 22:15:01 -0800 Subject: [PATCH 278/531] Incremental view maintenance for all graph ops All GraphImpl operations now support incremental maintenance of the replicated vertex view by reusing vTableReplicated whenever possible. This has two consequences: - This is not confined to the deltaJoinVertices operator anymore, so we rename it to the more intuitive "updateVertices" instead. - This enables mrTriplets to support skipStale. --- .../scala/org/apache/spark/graph/Graph.scala | 9 ++- .../scala/org/apache/spark/graph/Pregel.scala | 4 +- .../org/apache/spark/graph/VertexRDD.scala | 76 +++++++++++------- .../apache/spark/graph/impl/GraphImpl.scala | 80 ++++++++++++------- .../spark/graph/impl/VTableReplicated.scala | 70 ++++++++-------- .../spark/graph/impl/VertexPartition.scala | 56 ++++++------- .../org/apache/spark/graph/GraphSuite.scala | 4 +- 7 files changed, 173 insertions(+), 126 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/Graph.scala b/graph/src/main/scala/org/apache/spark/graph/Graph.scala index a0907c319aac6..475553b6abaf9 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Graph.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Graph.scala @@ -242,7 +242,9 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { */ def mapReduceTriplets[A: ClassManifest]( mapFunc: EdgeTriplet[VD, ED] => Iterator[(Vid, A)], - reduceFunc: (A, A) => A) + reduceFunc: (A, A) => A, + skipStaleSrc: Boolean = false, + skipStaleDst: Boolean = false) : VertexRDD[A] /** @@ -278,7 +280,10 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { (mapFunc: (Vid, VD, Option[U]) => VD2) : Graph[VD2, ED] - def deltaJoinVertices(changedVerts: VertexRDD[VD]): Graph[VD, ED] + /** + * Replace vertices in the graph with corresponding vertices in `updates`. + */ + def updateVertices(updates: VertexRDD[VD]): Graph[VD, ED] // 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. diff --git a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala index 9fb1d3fd8cbb0..4a1c10283fc50 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala @@ -107,9 +107,9 @@ object Pregel { var i = 0 while (activeMessages > 0 && i < maxIterations) { // receive the messages - val changedVerts = g.vertices.deltaJoin(messages)(vprog).cache() // updating the vertices + val changedVerts = g.vertices.zipJoin(messages)(vprog).cache() // updating the vertices // replicate the changed vertices - g = g.deltaJoinVertices(changedVerts) + g = g.updateVertices(changedVerts) val oldMessages = messages // compute the messages diff --git a/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala b/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala index f66a9ff0ad137..0c6b20fd1d970 100644 --- a/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala @@ -60,7 +60,7 @@ class VertexRDD[@specialized VD: ClassManifest]( /** * Construct a new VertexRDD that is indexed by only the keys in the RDD. - * The resulting VertexSet will be based on a different index and can + * The resulting VertexRDD will be based on a different index and can * no longer be quickly joined with this RDD. */ def reindex(): VertexRDD[VD] = new VertexRDD(partitionsRDD.map(_.reindex())) @@ -157,7 +157,7 @@ class VertexRDD[@specialized VD: ClassManifest]( * * @param f the function applied to each value in the RDD * @return a new VertexRDD with values obtained by applying `f` to - * each of the entries in the original VertexSet. The resulting + * each of the entries in the original VertexRDD. The resulting * VertexRDD retains the same index. */ def mapValues[VD2: ClassManifest](f: VD => VD2): VertexRDD[VD2] = @@ -171,12 +171,16 @@ class VertexRDD[@specialized VD: ClassManifest]( * * @param f the function applied to each value in the RDD * @return a new VertexRDD with values obtained by applying `f` to - * each of the entries in the original VertexSet. The resulting + * each of the entries in the original VertexRDD. The resulting * VertexRDD retains the same index. */ def mapValues[VD2: ClassManifest](f: (Vid, VD) => VD2): VertexRDD[VD2] = this.mapVertexPartitions(_.map(f)) + /** + * 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] = { this.zipVertexPartitions(other) { (thisPart, otherPart) => thisPart.diff(otherPart) @@ -197,24 +201,14 @@ class VertexRDD[@specialized VD: ClassManifest]( * this and the other vertex set to a new vertex attribute. * @return a VertexRDD containing only the vertices in both this * and the other VertexSet and with tuple attributes. - * */ def zipJoin[VD2: ClassManifest, VD3: ClassManifest] - (other: VertexRDD[VD2])(f: (Vid, VD, VD2) => VD3): VertexRDD[VD3] = - { + (other: VertexRDD[VD2])(f: (Vid, VD, VD2) => VD3): VertexRDD[VD3] = { this.zipVertexPartitions(other) { (thisPart, otherPart) => thisPart.join(otherPart)(f) } } - def deltaJoin[VD2: ClassManifest] - (other: VertexRDD[VD2])(f: (Vid, VD, VD2) => VD): VertexRDD[VD] = - { - this.zipVertexPartitions(other) { (thisPart, otherPart) => - thisPart.deltaJoin(otherPart)(f) - } - } - /** * Left join this VertexSet with another VertexSet which has the * same Index. This function will fail if both VertexSets do not @@ -234,31 +228,30 @@ class VertexRDD[@specialized VD: ClassManifest]( * */ def leftZipJoin[VD2: ClassManifest, VD3: ClassManifest] - (other: VertexRDD[VD2])(f: (Vid, VD, Option[VD2]) => VD3): VertexRDD[VD3] = - { + (other: VertexRDD[VD2])(f: (Vid, VD, Option[VD2]) => VD3): VertexRDD[VD3] = { this.zipVertexPartitions(other) { (thisPart, otherPart) => thisPart.leftJoin(otherPart)(f) } } /** - * Left join this VertexSet with an RDD containing vertex attribute - * pairs. If the other RDD is backed by a VertexSet with the same + * Left join this VertexRDD with an RDD containing vertex attribute + * pairs. If the other RDD is backed by a VertexRDD with the same * index than the efficient leftZipJoin implementation is used. The * resulting vertex set contains an entry for each vertex in this - * set. If the other VertexSet is missing any vertex in this - * VertexSet then a `None` attribute is generated. + * set. If the other VertexRDD is missing any vertex in this + * VertexRDD then a `None` attribute is generated. * * If there are duplicates, the vertex is picked at random. * - * @tparam VD2 the attribute type of the other VertexSet - * @tparam VD3 the attribute type of the resulting VertexSet + * @tparam VD2 the attribute type of the other VertexRDD + * @tparam VD3 the attribute type of the resulting VertexRDD * - * @param other the other VertexSet with which to join. + * @param other the other VertexRDD with which to join. * @param f the function mapping a vertex id and its attributes in * this and the other vertex set to a new vertex attribute. * @return a VertexRDD containing all the vertices in this - * VertexSet with the attribute emitted by f. + * VertexRDD with the attribute emitted by f. */ def leftJoin[VD2: ClassManifest, VD3: ClassManifest] (other: RDD[(Vid, VD2)]) @@ -282,13 +275,42 @@ class VertexRDD[@specialized VD: ClassManifest]( } } + /** + * Same effect as leftJoin(other) { (vid, a, bOpt) => bOpt.getOrElse(a) }, but `this` and `other` + * must have the same index. + */ + def zipUpdate(other: VertexRDD[VD]): VertexRDD[VD] = { + this.zipVertexPartitions(other) { (thisPart, otherPart) => + thisPart.update(otherPart.iterator) + } + } + + /** Same effect as leftJoin(other) { (vid, a, bOpt) => bOpt.getOrElse(a) } */ + def update(other: RDD[(Vid, VD)]): VertexRDD[VD] = { + // Test if the other vertex is a VertexRDD to choose the optimal join strategy. + // If the other set is a VertexRDD then we use the much more efficient leftOuterZipJoin + other match { + case other: VertexRDD[_] => + zipUpdate(other) + case _ => + new VertexRDD[VD]( + partitionsRDD.zipPartitions( + other.partitionBy(this.partitioner.get), preservesPartitioning = true) + { (part, msgs) => + val vertexPartition: VertexPartition[VD] = part.next() + Iterator(vertexPartition.update(msgs)) + } + ) + } + } + def aggregateUsingIndex[VD2: ClassManifest]( messages: RDD[(Vid, VD2)], reduceFunc: (VD2, VD2) => VD2): VertexRDD[VD2] = { val shuffled = MsgRDDFunctions.partitionForAggregation(messages, this.partitioner.get) val parts = partitionsRDD.zipPartitions(shuffled, true) { (thisIter, msgIter) => - val vertextPartition: VertexPartition[VD] = thisIter.next() - Iterator(vertextPartition.aggregateUsingIndex(msgIter, reduceFunc)) + val vertexPartition: VertexPartition[VD] = thisIter.next() + Iterator(vertexPartition.aggregateUsingIndex(msgIter, reduceFunc)) } new VertexRDD[VD2](parts) } @@ -297,7 +319,7 @@ class VertexRDD[@specialized VD: ClassManifest]( /** - * The VertexRDD singleton is used to construct VertexSets + * The VertexRDD singleton is used to construct VertexRDDs */ object VertexRDD { diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index 771c460345655..42db3c8e7f580 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -129,13 +129,25 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( } // end of printLineage override def reverse: Graph[VD, ED] = - new GraphImpl(vertices, edges.mapEdgePartitions(_.reverse), vertexPlacement) - - override def mapVertices[VD2: ClassManifest](f: (Vid, VD) => VD2): Graph[VD2, ED] = - new GraphImpl(vertices.mapVertexPartitions(_.map(f)), edges, vertexPlacement) + new GraphImpl(vertices, edges.mapEdgePartitions(_.reverse), vertexPlacement, vTableReplicated) + + override def mapVertices[VD2: ClassManifest](f: (Vid, VD) => VD2): Graph[VD2, ED] = { + if (classManifest[VD] equals classManifest[VD2]) { + // The map preserves type, so we can use incremental replication + val newVerts = vertices.mapVertexPartitions(_.map(f)) + val changedVerts = vertices.asInstanceOf[VertexRDD[VD2]].diff(newVerts) + val newVTableReplicated = new VTableReplicated[VD2]( + newVerts, edges, vertexPlacement, + Some(vTableReplicated.asInstanceOf[VTableReplicated[VD2]])) + new GraphImpl(newVerts, edges, vertexPlacement, newVTableReplicated) + } else { + // The map does not preserve type, so we must re-replicate all vertices + new GraphImpl(vertices.mapVertexPartitions(_.map(f)), edges, vertexPlacement) + } + } override def mapEdges[ED2: ClassManifest](f: Edge[ED] => ED2): Graph[VD, ED2] = - new GraphImpl(vertices, edges.mapEdgePartitions(_.map(f)), vertexPlacement) + new GraphImpl(vertices, edges.mapEdgePartitions(_.map(f)), vertexPlacement, vTableReplicated) override def mapTriplets[ED2: ClassManifest](f: EdgeTriplet[VD, ED] => ED2): Graph[VD, ED2] = { // Use an explicit manifest in PrimitiveKeyOpenHashMap init so we don't pull in the implicit @@ -153,7 +165,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( } Iterator((pid, newEdgePartition)) } - new GraphImpl(vertices, new EdgeRDD(newETable), vertexPlacement) + new GraphImpl(vertices, new EdgeRDD(newETable), vertexPlacement, vTableReplicated) } override def subgraph( @@ -184,7 +196,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( override def groupEdges(merge: (ED, ED) => ED): Graph[VD, ED] = { ClosureCleaner.clean(merge) val newETable = edges.mapEdgePartitions(_.groupEdges(merge)) - new GraphImpl(vertices, newETable, vertexPlacement) + new GraphImpl(vertices, newETable, vertexPlacement, vTableReplicated) } ////////////////////////////////////////////////////////////////////////////////////////////////// @@ -193,7 +205,9 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( override def mapReduceTriplets[A: ClassManifest]( mapFunc: EdgeTriplet[VD, ED] => Iterator[(Vid, A)], - reduceFunc: (A, A) => A): VertexRDD[A] = { + reduceFunc: (A, A) => A, + skipStaleSrc: Boolean = false, + skipStaleDst: Boolean = false): VertexRDD[A] = { ClosureCleaner.clean(mapFunc) ClosureCleaner.clean(reduceFunc) @@ -211,14 +225,21 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( // Iterate over the partition val et = new EdgeTriplet[VD, ED](vertexPartition) val filteredEdges = edgePartition.iterator.flatMap { e => - et.set(e) - if (mapUsesSrcAttr) { - et.srcAttr = vertexPartition(e.srcId) - } - if (mapUsesDstAttr) { - et.dstAttr = vertexPartition(e.dstId) + // Ensure that the edge meets the requirements of skipStaleSrc and skipStaleDst + val srcVertexOK = !skipStaleSrc || vertexPartition.isDefined(e.srcId) + val dstVertexOK = !skipStaleDst || vertexPartition.isDefined(e.dstId) + if (srcVertexOK && dstVertexOK) { + et.set(e) + if (mapUsesSrcAttr) { + et.srcAttr = vertexPartition(e.srcId) + } + if (mapUsesDstAttr) { + et.dstAttr = vertexPartition(e.dstId) + } + mapFunc(et) + } else { + Iterator.empty } - mapFunc(et) } // Note: This doesn't allow users to send messages to arbitrary vertices. vertexPartition.aggregateUsingIndex(filteredEdges, reduceFunc).iterator @@ -229,21 +250,26 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( } // end of mapReduceTriplets override def outerJoinVertices[U: ClassManifest, VD2: ClassManifest] - (updates: RDD[(Vid, U)])(updateF: (Vid, VD, Option[U]) => VD2): Graph[VD2, ED] = { - ClosureCleaner.clean(updateF) - val newVTable = vertices.leftJoin(updates)(updateF) - new GraphImpl(newVTable, edges, vertexPlacement) + (updates: RDD[(Vid, U)])(updateF: (Vid, VD, Option[U]) => VD2): Graph[VD2, ED] = { + if (classManifest[VD] equals classManifest[VD2]) { + // updateF preserves type, so we can use incremental replication + val newVerts = vertices.leftJoin(updates)(updateF) + val changedVerts = vertices.asInstanceOf[VertexRDD[VD2]].diff(newVerts) + val newVTableReplicated = new VTableReplicated[VD2]( + newVerts, edges, vertexPlacement, + Some(vTableReplicated.asInstanceOf[VTableReplicated[VD2]])) + new GraphImpl(newVerts, edges, vertexPlacement, newVTableReplicated) + } else { + // updateF does not preserve type, so we must re-replicate all vertices + val newVerts = vertices.leftJoin(updates)(updateF) + new GraphImpl(newVerts, edges, vertexPlacement) + } } - override def deltaJoinVertices(changedVerts: VertexRDD[VD]): Graph[VD, ED] = { - val newVerts = vertices.leftZipJoin(changedVerts) { (vid, oldAttr, newAttrOpt) => - newAttrOpt match { - case Some(newAttr) => newAttr - case None => oldAttr - } - } + override def updateVertices(updates: VertexRDD[VD]): Graph[VD, ED] = { + val newVerts = vertices.update(updates) val newVTableReplicated = new VTableReplicated( - changedVerts, edges, vertexPlacement, Some(vTableReplicated)) + updates, edges, vertexPlacement, Some(vTableReplicated)) new GraphImpl(newVerts, edges, vertexPlacement, newVTableReplicated) } } // end of class GraphImpl diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala b/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala index 6cbeb50186452..c88c133426a75 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala @@ -7,26 +7,26 @@ import org.apache.spark.util.collection.{PrimitiveVector, OpenHashSet} import org.apache.spark.graph._ /** - * Stores the vertex attribute values after they are replicated. + * A view of the vertices after they are shipped to the join sites specified in + * `vertexPlacement`. The resulting view is co-partitioned with `edges`. If `prevVTableReplicated` + * is specified, `updatedVerts` are treated as incremental updates to the previous view. Otherwise, + * a fresh view is created. + * + * The view is always cached (i.e., once it is created, it remains materialized). This avoids + * constructing it twice if the user calls graph.triplets followed by graph.mapReduceTriplets, for + * example. */ private[impl] class VTableReplicated[VD: ClassManifest]( - vTable: VertexRDD[VD], - eTable: EdgeRDD[_], + updatedVerts: VertexRDD[VD], + edges: EdgeRDD[_], vertexPlacement: VertexPlacement, prevVTableReplicated: Option[VTableReplicated[VD]] = None) { - val bothAttrs: RDD[(Pid, VertexPartition[VD])] = - createVTableReplicated(vTable, eTable, vertexPlacement, true, true) - - val srcAttrOnly: RDD[(Pid, VertexPartition[VD])] = - createVTableReplicated(vTable, eTable, vertexPlacement, true, false) - - val dstAttrOnly: RDD[(Pid, VertexPartition[VD])] = - createVTableReplicated(vTable, eTable, vertexPlacement, false, true) - - val noAttrs: RDD[(Pid, VertexPartition[VD])] = - createVTableReplicated(vTable, eTable, vertexPlacement, false, false) + val bothAttrs: RDD[(Pid, VertexPartition[VD])] = createVTableReplicated(true, true) + val srcAttrOnly: RDD[(Pid, VertexPartition[VD])] = createVTableReplicated(true, false) + val dstAttrOnly: RDD[(Pid, VertexPartition[VD])] = createVTableReplicated(false, true) + val noAttrs: RDD[(Pid, VertexPartition[VD])] = createVTableReplicated(false, false) def get(includeSrc: Boolean, includeDst: Boolean): RDD[(Pid, VertexPartition[VD])] = { (includeSrc, includeDst) match { @@ -38,29 +38,28 @@ class VTableReplicated[VD: ClassManifest]( } private def createVTableReplicated( - vTable: VertexRDD[VD], - eTable: EdgeRDD[_], - vertexPlacement: VertexPlacement, - includeSrcAttr: Boolean, - includeDstAttr: Boolean): RDD[(Pid, VertexPartition[VD])] = { + includeSrcAttr: Boolean, includeDstAttr: Boolean): RDD[(Pid, VertexPartition[VD])] = { val placement = vertexPlacement.get(includeSrcAttr, includeDstAttr) val vdManifest = classManifest[VD] - // Send each edge partition the vertex attributes it wants, as specified in - // vertexPlacement - val msgsByPartition = placement.zipPartitions(vTable.partitionsRDD)(VTableReplicated.buildBuffer(_, _)(vdManifest)) - .partitionBy(eTable.partitioner.get).cache() + // Ship vertex attributes to edge partitions according to vertexPlacement + val shippedVerts = placement + .zipPartitions(updatedVerts.partitionsRDD)(VTableReplicated.buildBuffer(_, _)(vdManifest)) + .partitionBy(edges.partitioner.get).cache() // TODO: Consider using a specialized shuffler. prevVTableReplicated match { case Some(vTableReplicated) => - val prev: RDD[(Pid, VertexPartition[VD])] = + val prevView: RDD[(Pid, VertexPartition[VD])] = vTableReplicated.get(includeSrcAttr, includeDstAttr) - prev.zipPartitions(msgsByPartition) { (vTableIter, msgsIter) => - val (pid, vertexPartition) = vTableIter.next() - val newVPart = vertexPartition.updateUsingIndex(msgsIter.flatMap(_._2.iterator))(vdManifest) + // Update vTableReplicated with updatedVerts, setting staleness flags in the resulting + // VertexPartitions + prevView.zipPartitions(shippedVerts) { (prevViewIter, shippedVertsIter) => + val (pid, prevVPart) = prevViewIter.next() + val newVPart = prevVPart.updateHideUnchanged( + shippedVertsIter.flatMap(_._2.iterator)) Iterator((pid, newVPart)) }.cache().setName("VTableReplicated delta %s %s".format(includeSrcAttr, includeDstAttr)) @@ -70,7 +69,7 @@ class VTableReplicated[VD: ClassManifest]( // will receive, because it stores vids from both the source and destination // of edges. It must always include both source and destination vids because // some operations, such as GraphImpl.mapReduceTriplets, rely on this. - val localVidMap = eTable.partitionsRDD.mapPartitions(_.map { + val localVidMap = edges.partitionsRDD.mapPartitions(_.map { case (pid, epart) => val vidToIndex = new VertexIdToIndexMap epart.foreach { e => @@ -82,12 +81,12 @@ class VTableReplicated[VD: ClassManifest]( // Within each edge partition, place the vertex attributes received from // msgsByPartition into the correct locations specified in localVidMap - localVidMap.zipPartitions(msgsByPartition) { (mapIter, msgsIter) => + localVidMap.zipPartitions(shippedVerts) { (mapIter, shippedVertsIter) => val (pid, vidToIndex) = mapIter.next() assert(!mapIter.hasNext) // Populate the vertex array using the vidToIndex map val vertexArray = vdManifest.newArray(vidToIndex.capacity) - for ((_, block) <- msgsIter) { + for ((_, block) <- shippedVertsIter) { for (i <- 0 until block.vids.size) { val vid = block.vids(i) val attr = block.attrs(i) @@ -95,17 +94,18 @@ class VTableReplicated[VD: ClassManifest]( vertexArray(ind) = attr } } - Iterator((pid, new VertexPartition(vidToIndex, vertexArray, vidToIndex.getBitSet)(vdManifest))) + val newVPart = new VertexPartition( + vidToIndex, vertexArray, vidToIndex.getBitSet)(vdManifest) + Iterator((pid, newVPart)) }.cache().setName("VTableReplicated %s %s".format(includeSrcAttr, includeDstAttr)) } } - } - object VTableReplicated { - - def buildBuffer[VD: ClassManifest](pid2vidIter: Iterator[Array[Array[Vid]]], vertexPartIter: Iterator[VertexPartition[VD]]) = { + protected def buildBuffer[VD: ClassManifest]( + pid2vidIter: Iterator[Array[Array[Vid]]], + vertexPartIter: Iterator[VertexPartition[VD]]) = { val pid2vid: Array[Array[Vid]] = pid2vidIter.next() val vertexPart: VertexPartition[VD] = vertexPartIter.next() diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala index fe005c872328f..0e139e11bba9f 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala @@ -94,19 +94,22 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassManifest]( new VertexPartition(index, values, newMask) } + /** + * Hides vertices that are the same between this and other. For vertices that are different, keeps + * the values from `other`. The indices of `this` and `other` must be the same. + */ def diff(other: VertexPartition[VD]): VertexPartition[VD] = { assert(index == other.index) val newMask = mask & other.mask - var i = newMask.nextSetBit(0) while (i >= 0) { if (values(i) == other.values(i)) { newMask.unset(i) } - i = mask.nextSetBit(i + 1) + i = newMask.nextSetBit(i + 1) } - new VertexPartition[VD](index, other.values, newMask) + new VertexPartition(index, other.values, newMask) } /** Inner join another VertexPartition. */ @@ -130,30 +133,6 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassManifest]( } } - /** Inner join another VertexPartition. */ - def deltaJoin[VD2: ClassManifest, VD3: ClassManifest] - (other: VertexPartition[VD2]) - (f: (Vid, VD, VD2) => VD3): VertexPartition[VD3] = - { - if (index != other.index) { - logWarning("Joining two VertexPartitions with different indexes is slow.") - join(createUsingIndex(other.iterator))(f) - } else { - val newValues = new Array[VD3](capacity) - val newMask = mask & other.mask - - var i = newMask.nextSetBit(0) - while (i >= 0) { - newValues(i) = f(index.getValue(i), values(i), other.values(i)) - if (newValues(i) == values(i)) { - newMask.unset(i) - } - i = mask.nextSetBit(i + 1) - } - new VertexPartition(index, newValues, newMask) - } - } - /** Left outer join another VertexPartition. */ def leftJoin[VD2: ClassManifest, VD3: ClassManifest] (other: VertexPartition[VD2]) @@ -196,17 +175,32 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassManifest]( new VertexPartition[VD2](index, newValues, newMask) } - def updateUsingIndex[VD2: ClassManifest](iter: Iterator[Product2[Vid, VD2]]) - : VertexPartition[VD2] = { + /** Same effect as leftJoin(iter) { (vid, a, bOpt) => bOpt.getOrElse(a) } */ + def update(iter: Iterator[Product2[Vid, VD]]): VertexPartition[VD] = { + val newValues = new Array[VD](capacity) + System.arraycopy(values, 0, newValues, 0, newValues.length) + iter.foreach { case (vid, vdata) => + val pos = index.getPos(vid) + newValues(pos) = vdata + } + new VertexPartition(index, newValues, mask) + } + + /** + * Same effect as leftJoin(iter) { (vid, a, bOpt) => bOpt.getOrElse(a) }, but unchanged vertices + * are hidden using the bitmask. + */ + def updateHideUnchanged(iter: Iterator[Product2[Vid, VD]]) + : VertexPartition[VD] = { val newMask = new BitSet(capacity) - val newValues = new Array[VD2](capacity) + val newValues = new Array[VD](capacity) System.arraycopy(values, 0, newValues, 0, newValues.length) iter.foreach { case (vid, vdata) => val pos = index.getPos(vid) newMask.set(pos) newValues(pos) = vdata } - new VertexPartition[VD2](index, newValues, newMask) + new VertexPartition(index, newValues, newMask) } def aggregateUsingIndex[VD2: ClassManifest]( diff --git a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala index 514d20b76c0bc..34ab503e9d34d 100644 --- a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala @@ -152,7 +152,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { } } - test("deltaJoinVertices") { + test("updateVertices") { withSpark(new SparkContext("local", "test")) { sc => // Create a star graph of 10 vertices val n = 10 @@ -162,7 +162,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { val changedVerts = star.vertices.filter(_._1 % 2 == 0).mapValues((vid, attr) => "v2") // Apply the modification to the graph - val changedStar = star.deltaJoinVertices(changedVerts) + val changedStar = star.updateVertices(changedVerts) val newVertices = star.vertices.leftZipJoin(changedVerts) { (vid, oldVd, newVdOpt) => newVdOpt match { From 6bf51a5e1591209e02ce6c08b86131d9ee4e6e90 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 13 Dec 2013 01:53:42 -0800 Subject: [PATCH 279/531] Don't expose {src,dst}Mask in EdgeTriplet Exposing this information is not necessary due to the addition of skipStale. --- .../scala/org/apache/spark/graph/EdgeTriplet.scala | 5 +---- .../main/scala/org/apache/spark/graph/Pregel.scala | 12 ++---------- 2 files changed, 3 insertions(+), 14 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/EdgeTriplet.scala b/graph/src/main/scala/org/apache/spark/graph/EdgeTriplet.scala index 1f92233df9586..28efb5d1c178a 100644 --- a/graph/src/main/scala/org/apache/spark/graph/EdgeTriplet.scala +++ b/graph/src/main/scala/org/apache/spark/graph/EdgeTriplet.scala @@ -13,7 +13,7 @@ import org.apache.spark.graph.impl.VertexPartition * tried specializing I got a warning about inherenting from a type * that is not a trait. */ -class EdgeTriplet[VD, ED](vPart: VertexPartition[VD] = null) extends Edge[ED] { +class EdgeTriplet[VD, ED] extends Edge[ED] { // class EdgeTriplet[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) VD: ClassManifest, // @specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED: ClassManifest] extends Edge[ED] { @@ -28,9 +28,6 @@ class EdgeTriplet[VD, ED](vPart: VertexPartition[VD] = null) extends Edge[ED] { */ var dstAttr: VD = _ //nullValue[VD] - def srcMask: Boolean = vPart.isDefined(srcId) - def dstMask: Boolean = vPart.isDefined(dstId) - /** * Set the edge properties of this triplet. */ diff --git a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala index 4a1c10283fc50..6a267d8c28530 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala @@ -91,17 +91,9 @@ object Pregel { mergeMsg: (A, A) => A) : Graph[VD, ED] = { - def sendMsgFun(edge: EdgeTriplet[VD, ED]): Iterator[(Vid, A)] = { - if (edge.srcMask) { - sendMsg(edge) - } else { - Iterator.empty - } - } - var g = graph.mapVertices( (vid, vdata) => vprog(vid, vdata, initialMsg) ) // compute the messages - var messages = g.mapReduceTriplets(sendMsgFun, mergeMsg).cache() + var messages = g.mapReduceTriplets(sendMsg, mergeMsg).cache() var activeMessages = messages.count() // Loop var i = 0 @@ -113,7 +105,7 @@ object Pregel { val oldMessages = messages // compute the messages - messages = g.mapReduceTriplets(sendMsgFun, mergeMsg).cache() + messages = g.mapReduceTriplets(sendMsg, mergeMsg).cache() activeMessages = messages.count() // after counting we can unpersist the old messages oldMessages.unpersist(blocking=false) From 4f4f0add2496c5478b0538c0dc861dba4419459c Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 13 Dec 2013 01:55:04 -0800 Subject: [PATCH 280/531] Fix bug in mapVertices and outerJoinVertices --- .../main/scala/org/apache/spark/graph/impl/GraphImpl.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index 42db3c8e7f580..e14718e67dcca 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -137,7 +137,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( val newVerts = vertices.mapVertexPartitions(_.map(f)) val changedVerts = vertices.asInstanceOf[VertexRDD[VD2]].diff(newVerts) val newVTableReplicated = new VTableReplicated[VD2]( - newVerts, edges, vertexPlacement, + changedVerts, edges, vertexPlacement, Some(vTableReplicated.asInstanceOf[VTableReplicated[VD2]])) new GraphImpl(newVerts, edges, vertexPlacement, newVTableReplicated) } else { @@ -256,7 +256,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( val newVerts = vertices.leftJoin(updates)(updateF) val changedVerts = vertices.asInstanceOf[VertexRDD[VD2]].diff(newVerts) val newVTableReplicated = new VTableReplicated[VD2]( - newVerts, edges, vertexPlacement, + changedVerts, edges, vertexPlacement, Some(vTableReplicated.asInstanceOf[VTableReplicated[VD2]])) new GraphImpl(newVerts, edges, vertexPlacement, newVTableReplicated) } else { From d076fe5c9ef3d83af0fb737608660992cb730a7c Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 13 Dec 2013 01:56:07 -0800 Subject: [PATCH 281/531] Finish up 49f7fc5ed2edd42b118182438d84ed7d4e7d521f --- .../src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index e14718e67dcca..dd058f42c07ba 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -223,7 +223,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( val (_, vertexPartition) = vTableReplicatedIter.next() // Iterate over the partition - val et = new EdgeTriplet[VD, ED](vertexPartition) + val et = new EdgeTriplet[VD, ED] val filteredEdges = edgePartition.iterator.flatMap { e => // Ensure that the edge meets the requirements of skipStaleSrc and skipStaleDst val srcVertexOK = !skipStaleSrc || vertexPartition.isDefined(e.srcId) From dc7214790072d57cab45326e4777452cc5740495 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 13 Dec 2013 01:57:03 -0800 Subject: [PATCH 282/531] Fix bug in interaction of incr. view maint., skipStale, and join rewrite --- .../scala/org/apache/spark/graph/impl/GraphImpl.scala | 6 +++--- .../org/apache/spark/graph/impl/VertexPartition.scala | 9 +++++++++ 2 files changed, 12 insertions(+), 3 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index dd058f42c07ba..7816a1d5e8d29 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -226,9 +226,9 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( val et = new EdgeTriplet[VD, ED] val filteredEdges = edgePartition.iterator.flatMap { e => // Ensure that the edge meets the requirements of skipStaleSrc and skipStaleDst - val srcVertexOK = !skipStaleSrc || vertexPartition.isDefined(e.srcId) - val dstVertexOK = !skipStaleDst || vertexPartition.isDefined(e.dstId) - if (srcVertexOK && dstVertexOK) { + val skipDueToSrc = skipStaleSrc && vertexPartition.isStale(e.srcId) + val skipDueToDst = skipStaleDst && vertexPartition.isStale(e.dstId) + if (!skipDueToSrc && !skipDueToDst) { et.set(e) if (mapUsesSrcAttr) { et.srcAttr = vertexPartition(e.srcId) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala index 0e139e11bba9f..363fb68dbc035 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala @@ -47,6 +47,15 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassManifest]( pos >= 0 && mask.get(pos) } + /** + * A vertex is stale if it is present in the index but hidden by the mask. In contrast, a vertex + * is nonexistent (possibly due to join rewrite) if it is not present in the index at all. + */ + def isStale(vid: Vid): Boolean = { + val pos = index.getPos(vid) + pos >= 0 && !mask.get(pos) + } + /** * Pass each vertex attribute along with the vertex id through a map * function and retain the original RDD's partitioning and index. From 5e20cbaf66359338b01f1bf070fd601d39b3c1fd Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 13 Dec 2013 01:59:52 -0800 Subject: [PATCH 283/531] Define localVidMap once per VTableReplicated --- .../spark/graph/impl/VTableReplicated.scala | 30 ++++++++++--------- 1 file changed, 16 insertions(+), 14 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala b/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala index c88c133426a75..72032aa5b281d 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala @@ -23,6 +23,22 @@ class VTableReplicated[VD: ClassManifest]( vertexPlacement: VertexPlacement, prevVTableReplicated: Option[VTableReplicated[VD]] = None) { + // Within each edge partition, create a local map from vid to an index into + // the attribute array. Each map contains a superset of the vertices that it + // will receive, because it stores vids from both the source and destination + // of edges. It must always include both source and destination vids because + // some operations, such as GraphImpl.mapReduceTriplets, rely on this. + val localVidMap: RDD[(Int, VertexIdToIndexMap)] = edges.partitionsRDD.mapPartitions(_.map { + case (pid, epart) => + val vidToIndex = new VertexIdToIndexMap + epart.foreach { e => + vidToIndex.add(e.srcId) + vidToIndex.add(e.dstId) + } + (pid, vidToIndex) + }, preservesPartitioning = true).cache() + + val bothAttrs: RDD[(Pid, VertexPartition[VD])] = createVTableReplicated(true, true) val srcAttrOnly: RDD[(Pid, VertexPartition[VD])] = createVTableReplicated(true, false) val dstAttrOnly: RDD[(Pid, VertexPartition[VD])] = createVTableReplicated(false, true) @@ -64,20 +80,6 @@ class VTableReplicated[VD: ClassManifest]( }.cache().setName("VTableReplicated delta %s %s".format(includeSrcAttr, includeDstAttr)) case None => - // Within each edge partition, create a local map from vid to an index into - // the attribute array. Each map contains a superset of the vertices that it - // will receive, because it stores vids from both the source and destination - // of edges. It must always include both source and destination vids because - // some operations, such as GraphImpl.mapReduceTriplets, rely on this. - val localVidMap = edges.partitionsRDD.mapPartitions(_.map { - case (pid, epart) => - val vidToIndex = new VertexIdToIndexMap - epart.foreach { e => - vidToIndex.add(e.srcId) - vidToIndex.add(e.dstId) - } - (pid, vidToIndex) - }, preservesPartitioning = true).cache() // Within each edge partition, place the vertex attributes received from // msgsByPartition into the correct locations specified in localVidMap From 8f4b8e9b954be5022e4bcc69ec45180a513b132c Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 13 Dec 2013 02:03:08 -0800 Subject: [PATCH 284/531] Reuse previous localVidMap if available --- .../spark/graph/impl/VTableReplicated.scala | 35 +++++++++++-------- 1 file changed, 20 insertions(+), 15 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala b/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala index 72032aa5b281d..4cccac02cf651 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala @@ -23,21 +23,26 @@ class VTableReplicated[VD: ClassManifest]( vertexPlacement: VertexPlacement, prevVTableReplicated: Option[VTableReplicated[VD]] = None) { - // Within each edge partition, create a local map from vid to an index into - // the attribute array. Each map contains a superset of the vertices that it - // will receive, because it stores vids from both the source and destination - // of edges. It must always include both source and destination vids because - // some operations, such as GraphImpl.mapReduceTriplets, rely on this. - val localVidMap: RDD[(Int, VertexIdToIndexMap)] = edges.partitionsRDD.mapPartitions(_.map { - case (pid, epart) => - val vidToIndex = new VertexIdToIndexMap - epart.foreach { e => - vidToIndex.add(e.srcId) - vidToIndex.add(e.dstId) - } - (pid, vidToIndex) - }, preservesPartitioning = true).cache() - + /** + * Within each edge partition, create a local map from vid to an index into the attribute + * array. Each map contains a superset of the vertices that it will receive, because it stores + * vids from both the source and destination of edges. It must always include both source and + * destination vids because some operations, such as GraphImpl.mapReduceTriplets, rely on this. + */ + private val localVidMap: RDD[(Int, VertexIdToIndexMap)] = prevVTableReplicated match { + case Some(prev) => + prev.localVidMap + case None => + edges.partitionsRDD.mapPartitions(_.map { + case (pid, epart) => + val vidToIndex = new VertexIdToIndexMap + epart.foreach { e => + vidToIndex.add(e.srcId) + vidToIndex.add(e.dstId) + } + (pid, vidToIndex) + }, preservesPartitioning = true).cache() + } val bothAttrs: RDD[(Pid, VertexPartition[VD])] = createVTableReplicated(true, true) val srcAttrOnly: RDD[(Pid, VertexPartition[VD])] = createVTableReplicated(true, false) From 45ea674e214750d8e3d8c04084cbe5bc7033cf02 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 13 Dec 2013 02:04:07 -0800 Subject: [PATCH 285/531] Handle diffing VertexPartitions with different indexes --- .../spark/graph/impl/VertexPartition.scala | 21 +++++++++++-------- 1 file changed, 12 insertions(+), 9 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala index 363fb68dbc035..55e7d74a28a18 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala @@ -108,17 +108,20 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassManifest]( * the values from `other`. The indices of `this` and `other` must be the same. */ def diff(other: VertexPartition[VD]): VertexPartition[VD] = { - assert(index == other.index) - - val newMask = mask & other.mask - var i = newMask.nextSetBit(0) - while (i >= 0) { - if (values(i) == other.values(i)) { - newMask.unset(i) + if (index != other.index) { + logWarning("Diffing two VertexPartitions with different indexes is slow.") + diff(createUsingIndex(other.iterator)) + } else { + val newMask = mask & other.mask + var i = newMask.nextSetBit(0) + while (i >= 0) { + if (values(i) == other.values(i)) { + newMask.unset(i) + } + i = newMask.nextSetBit(i + 1) } - i = newMask.nextSetBit(i + 1) + new VertexPartition(index, other.values, newMask) } - new VertexPartition(index, other.values, newMask) } /** Inner join another VertexPartition. */ From b2f595f55248236f780e5ef332dde691ae1ee6e6 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 13 Dec 2013 02:06:23 -0800 Subject: [PATCH 286/531] Test more graph ops and skipStale --- .../org/apache/spark/graph/GraphSuite.scala | 52 +++++++++++++++++-- 1 file changed, 49 insertions(+), 3 deletions(-) diff --git a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala index 34ab503e9d34d..dd6a540b92c97 100644 --- a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala @@ -35,12 +35,38 @@ class GraphSuite extends FunSuite with LocalSparkContext { } } + test("core operations") { + withSpark(new SparkContext("local", "test")) { sc => + val n = 5 + val star = Graph.fromEdgeTuples( + sc.parallelize((1 to n).map(x => (0: Vid, x: Vid)), 3), "v") + // triplets + assert(star.triplets.map(et => (et.srcId, et.dstId, et.srcAttr, et.dstAttr)).collect.toSet === + (1 to n).map(x => (0: Vid, x: Vid, "v", "v")).toSet) + // reverse + val reverseStar = star.reverse + assert(reverseStar.outDegrees.collect.toSet === (1 to n).map(x => (x: Vid, 1)).toSet) + // outerJoinVertices + val reverseStarDegrees = + reverseStar.outerJoinVertices(reverseStar.outDegrees) { (vid, a, bOpt) => bOpt.getOrElse(0) } + val neighborDegreeSums = reverseStarDegrees.mapReduceTriplets( + et => Iterator((et.srcId, et.dstAttr), (et.dstId, et.srcAttr)), + (a: Int, b: Int) => a + b).collect.toSet + assert(neighborDegreeSums === Set((0: Vid, n)) ++ (1 to n).map(x => (x: Vid, 0))) + // mapVertices preserving type + val mappedVAttrs = reverseStar.mapVertices((vid, attr) => attr + "2") + assert(mappedVAttrs.vertices.collect.toSet === (0 to n).map(x => (x: Vid, "v2")).toSet) + // mapVertices changing type + val mappedVAttrs2 = reverseStar.mapVertices((vid, attr) => attr.length) + assert(mappedVAttrs2.vertices.collect.toSet === (0 to n).map(x => (x: Vid, 1)).toSet) + } + } + test("mapEdges") { withSpark(new SparkContext("local", "test")) { sc => val n = 3 val star = Graph.fromEdgeTuples( - sc.parallelize((1 to n).map(x => (0: Vid, x: Vid))), - "defaultValue") + sc.parallelize((1 to n).map(x => (0: Vid, x: Vid))), "v") val starWithEdgeAttrs = star.mapEdges(e => e.dstId) // map(_.copy()) is a workaround for https://github.com/amplab/graphx/issues/25 @@ -52,13 +78,33 @@ class GraphSuite extends FunSuite with LocalSparkContext { test("mapReduceTriplets") { withSpark(new SparkContext("local", "test")) { sc => - val n = 3 + val n = 5 val star = Graph.fromEdgeTuples(sc.parallelize((1 to n).map(x => (0: Vid, x: Vid))), 0) val starDeg = star.joinVertices(star.degrees){ (vid, oldV, deg) => deg } val neighborDegreeSums = starDeg.mapReduceTriplets( edge => Iterator((edge.srcId, edge.dstAttr), (edge.dstId, edge.srcAttr)), (a: Int, b: Int) => a + b) assert(neighborDegreeSums.collect().toSet === (0 to n).map(x => (x, n)).toSet) + + // skipStale + val allPairs = for (x <- 1 to n; y <- x + 1 to n) yield (x: Vid, y: Vid) + val complete = Graph.fromEdgeTuples(sc.parallelize(allPairs, 1), 0) + val vids = complete.mapVertices((vid, attr) => vid).cache() + vids.triplets.foreach(x => {}) // force replicated vertex view to be materialized + val vids2 = vids.mapVertices((vid, attr) => if (attr % 2 == 0) -attr else attr).cache() + val numEvenNeighbors = vids2.mapReduceTriplets(et => { + // Map function should only run on changed vertices + if (et.srcId % 2 != 0) { + throw new Exception("map ran on edge with src vid %d, which is odd".format(et.srcId)) + } + if (et.dstId % 2 != 0) { + throw new Exception("map ran on edge with dst vid %d, which is odd".format(et.dstId)) + } + println(et.srcAttr) + println(et.dstAttr) + Iterator((et.srcId, 1), (et.dstId, 1)) + }, (a: Int, b: Int) => a + b, skipStaleSrc = true, skipStaleDst = true).collect.toSet + assert(numEvenNeighbors === (2 to n by 2).map(x => (x: Vid, n / 2 - 1)).toSet) } } From 4bbae5c39b89333fda3006671205de6d9fff1911 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 13 Dec 2013 02:08:23 -0800 Subject: [PATCH 287/531] Remove printlns from test --- graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala | 2 -- 1 file changed, 2 deletions(-) diff --git a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala index dd6a540b92c97..2f5b64a61212d 100644 --- a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala @@ -100,8 +100,6 @@ class GraphSuite extends FunSuite with LocalSparkContext { if (et.dstId % 2 != 0) { throw new Exception("map ran on edge with dst vid %d, which is odd".format(et.dstId)) } - println(et.srcAttr) - println(et.dstAttr) Iterator((et.srcId, 1), (et.dstId, 1)) }, (a: Int, b: Int) => a + b, skipStaleSrc = true, skipStaleDst = true).collect.toSet assert(numEvenNeighbors === (2 to n by 2).map(x => (x: Vid, n / 2 - 1)).toSet) From d161caa6eb42b2c399b7f0878bd5aea3978febcf Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 13 Dec 2013 16:35:37 -0800 Subject: [PATCH 288/531] Expose srcStale and dstStale --- .../src/main/scala/org/apache/spark/graph/EdgeTriplet.scala | 3 +++ .../main/scala/org/apache/spark/graph/impl/GraphImpl.scala | 6 ++++-- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/EdgeTriplet.scala b/graph/src/main/scala/org/apache/spark/graph/EdgeTriplet.scala index 28efb5d1c178a..76768489eed37 100644 --- a/graph/src/main/scala/org/apache/spark/graph/EdgeTriplet.scala +++ b/graph/src/main/scala/org/apache/spark/graph/EdgeTriplet.scala @@ -28,6 +28,9 @@ class EdgeTriplet[VD, ED] extends Edge[ED] { */ var dstAttr: VD = _ //nullValue[VD] + var srcStale: Boolean = false + var dstStale: Boolean = false + /** * Set the edge properties of this triplet. */ diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index 7816a1d5e8d29..8ba90fd9bb97d 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -226,8 +226,10 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( val et = new EdgeTriplet[VD, ED] val filteredEdges = edgePartition.iterator.flatMap { e => // Ensure that the edge meets the requirements of skipStaleSrc and skipStaleDst - val skipDueToSrc = skipStaleSrc && vertexPartition.isStale(e.srcId) - val skipDueToDst = skipStaleDst && vertexPartition.isStale(e.dstId) + et.srcStale = vertexPartition.isStale(e.srcId) + et.dstStale = vertexPartition.isStale(e.dstId) + val skipDueToSrc = skipStaleSrc && et.srcStale + val skipDueToDst = skipStaleDst && et.dstStale if (!skipDueToSrc && !skipDueToDst) { et.set(e) if (mapUsesSrcAttr) { From 77b92748ad0224f55df9e2dbb80aa98f1597a49a Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 13 Dec 2013 16:36:46 -0800 Subject: [PATCH 289/531] Replace update with innerJoin (has a bug) There is a conflict between vertices that didn't change so are not moved but still need to run, and vertices that were deleted by the innerJoin so should not run. --- .../scala/org/apache/spark/graph/Graph.scala | 7 +++-- .../scala/org/apache/spark/graph/Pregel.scala | 11 +++---- .../org/apache/spark/graph/VertexRDD.scala | 19 +++++++----- .../apache/spark/graph/impl/GraphImpl.scala | 23 ++++++++++---- .../spark/graph/impl/VTableReplicated.scala | 3 +- .../spark/graph/impl/VertexPartition.scala | 30 ++++++++++--------- 6 files changed, 58 insertions(+), 35 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/Graph.scala b/graph/src/main/scala/org/apache/spark/graph/Graph.scala index 475553b6abaf9..8c28cd4373891 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Graph.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Graph.scala @@ -281,9 +281,12 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { : Graph[VD2, ED] /** - * Replace vertices in the graph with corresponding vertices in `updates`. + * Replace vertices in the graph with corresponding vertices in `updates`, and restrict vertices + * without a corresponding vertex in `updates`. Edges adjacent to restricted vertices will still + * appear in graph.edges, but not in triplets or mapReduceTriplets. */ - def updateVertices(updates: VertexRDD[VD]): Graph[VD, ED] + def innerJoinVertices[U: ClassManifest, VD2: ClassManifest](table: RDD[(Vid, U)]) + (f: (Vid, VD, U) => VD2): Graph[VD2, ED] // 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. diff --git a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala index 6a267d8c28530..6bb8bd77e87b7 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala @@ -98,13 +98,14 @@ object Pregel { // Loop var i = 0 while (activeMessages > 0 && i < maxIterations) { - // receive the messages - val changedVerts = g.vertices.zipJoin(messages)(vprog).cache() // updating the vertices - // replicate the changed vertices - g = g.updateVertices(changedVerts) + // Receive the messages. Vertices that didn't get any messages do not appear in changedVerts. + val newVerts = g.vertices.innerJoin(messages)(vprog).cache() + // Update the graph with the new vertices, removing vertices that didn't get any messages. + g = g.innerJoinVertices(newVerts) val oldMessages = messages - // compute the messages + // Send new messages. Vertices that didn't get any messages in the previous round don't appear + // in the graph, so don't get to send messages. messages = g.mapReduceTriplets(sendMsg, mergeMsg).cache() activeMessages = messages.count() // after counting we can unpersist the old messages diff --git a/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala b/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala index 0c6b20fd1d970..5b8e00aa13892 100644 --- a/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala @@ -279,26 +279,31 @@ class VertexRDD[@specialized VD: ClassManifest]( * Same effect as leftJoin(other) { (vid, a, bOpt) => bOpt.getOrElse(a) }, but `this` and `other` * must have the same index. */ - def zipUpdate(other: VertexRDD[VD]): VertexRDD[VD] = { + def innerZipJoin[U: ClassManifest, VD2: ClassManifest](other: VertexRDD[U]) + (f: (Vid, VD, U) => VD2): VertexRDD[VD2] = { this.zipVertexPartitions(other) { (thisPart, otherPart) => - thisPart.update(otherPart.iterator) + thisPart.innerJoin(otherPart)(f) } } - /** Same effect as leftJoin(other) { (vid, a, bOpt) => bOpt.getOrElse(a) } */ - def update(other: RDD[(Vid, VD)]): VertexRDD[VD] = { + /** + * Replace vertices with corresponding vertices in `other`, and drop vertices without a + * corresponding vertex in `other. + */ + def innerJoin[U: ClassManifest, VD2: ClassManifest](other: RDD[(Vid, U)]) + (f: (Vid, VD, U) => VD2): VertexRDD[VD2] = { // Test if the other vertex is a VertexRDD to choose the optimal join strategy. - // If the other set is a VertexRDD then we use the much more efficient leftOuterZipJoin + // If the other set is a VertexRDD then we use the much more efficient innerZipJoin other match { case other: VertexRDD[_] => - zipUpdate(other) + innerZipJoin(other)(f) case _ => new VertexRDD[VD]( partitionsRDD.zipPartitions( other.partitionBy(this.partitioner.get), preservesPartitioning = true) { (part, msgs) => val vertexPartition: VertexPartition[VD] = part.next() - Iterator(vertexPartition.update(msgs)) + Iterator(vertexPartition.innerJoin(msgs)(f)) } ) } diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index 8ba90fd9bb97d..a954e7f907e84 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -268,11 +268,24 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( } } - override def updateVertices(updates: VertexRDD[VD]): Graph[VD, ED] = { - val newVerts = vertices.update(updates) - val newVTableReplicated = new VTableReplicated( - updates, edges, vertexPlacement, Some(vTableReplicated)) - new GraphImpl(newVerts, edges, vertexPlacement, newVTableReplicated) + override def innerJoinVertices[U: ClassManifest, VD2: ClassManifest](table: RDD[(Vid, U)]) + (f: (Vid, VD, U) => VD2): Graph[VD2, ED] + if (classManifest[VD] equals classManifest[VD2]) { + // f preserves type, so we can use incremental replication + val newVerts = vertices.innerJoin(table)(f) + val changedVerts = vertices.asInstanceOf[VertexRDD[VD2]].diff(newVerts) + // TODO(ankurdave): Need to resolve conflict between vertices that didn't change so are not + // moved but still need to run, and vertices that were deleted by the innerJoin so should not + // run + val newVTableReplicated = new VTableReplicated( + changedVerts, edges, vertexPlacement, + Some(vTableReplicated.asInstanceOf[VTableReplicated[VD2]])) + new GraphImpl(newVerts, edges, vertexPlacement, newVTableReplicated) + } else { + // updateF does not preserve type, so we must re-replicate all vertices in table + val newVerts = vertices.innerJoin(table)(f) + new GraphImpl(newVerts, edges, vertexPlacement) + } } } // end of class GraphImpl diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala b/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala index 4cccac02cf651..b90d2f14ee356 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala @@ -79,8 +79,7 @@ class VTableReplicated[VD: ClassManifest]( // VertexPartitions prevView.zipPartitions(shippedVerts) { (prevViewIter, shippedVertsIter) => val (pid, prevVPart) = prevViewIter.next() - val newVPart = prevVPart.updateHideUnchanged( - shippedVertsIter.flatMap(_._2.iterator)) + val newVPart = prevVPart.innerJoinKeepLeft(shippedVertsIter.flatMap(_._2.iterator)) Iterator((pid, newVPart)) }.cache().setName("VTableReplicated delta %s %s".format(includeSrcAttr, includeDstAttr)) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala index 55e7d74a28a18..c4aff6455dafe 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala @@ -172,6 +172,19 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassManifest]( leftJoin(createUsingIndex(other))(f) } + /** Inner join another VertexPartition. */ + def innerJoin[U: ClassManifest, VD2: ClassManifest](other: VertexPartition[U]) + (f: (Vid, VD, U) => VD2): VertexPartition[VD2] = { + val newMask = mask & other.mask + val newValues = new Array[VD2](capacity) + var i = newMask.nextSetBit(0) + while (i >= 0) { + newValues(i) = f(index.getValue(i), values(i), other.values(i)) + i = newMask.nextSetBit(i + 1) + } + new VertexPartition(index, newValues, newMask) + } + /** * Similar effect as aggregateUsingIndex((a, b) => a) */ @@ -187,22 +200,11 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassManifest]( new VertexPartition[VD2](index, newValues, newMask) } - /** Same effect as leftJoin(iter) { (vid, a, bOpt) => bOpt.getOrElse(a) } */ - def update(iter: Iterator[Product2[Vid, VD]]): VertexPartition[VD] = { - val newValues = new Array[VD](capacity) - System.arraycopy(values, 0, newValues, 0, newValues.length) - iter.foreach { case (vid, vdata) => - val pos = index.getPos(vid) - newValues(pos) = vdata - } - new VertexPartition(index, newValues, mask) - } - /** - * Same effect as leftJoin(iter) { (vid, a, bOpt) => bOpt.getOrElse(a) }, but unchanged vertices - * are hidden using the bitmask. + * Similar to innerJoin, but vertices from the left side that don't appear in iter will remain in + * the partition, hidden by the bitmask. */ - def updateHideUnchanged(iter: Iterator[Product2[Vid, VD]]) + def innerJoinKeepLeft(iter: Iterator[Product2[Vid, VD]]) : VertexPartition[VD] = { val newMask = new BitSet(capacity) val newValues = new Array[VD](capacity) From 7a8952e9bbf282c261c4d27ab3462b7d28ca575c Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 13 Dec 2013 20:52:16 -0800 Subject: [PATCH 290/531] Replace skipStale with activeSetOpt in mrTriplets (fails Pregel) --- .../scala/org/apache/spark/graph/Graph.scala | 16 +- .../scala/org/apache/spark/graph/Pregel.scala | 12 +- .../org/apache/spark/graph/VertexRDD.scala | 4 +- .../apache/spark/graph/impl/GraphImpl.scala | 60 +++---- .../spark/graph/impl/VTableReplicated.scala | 152 ++++++++++++------ .../spark/graph/impl/VertexPartition.scala | 35 ++-- .../org/apache/spark/graph/GraphSuite.scala | 50 +----- 7 files changed, 174 insertions(+), 155 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/Graph.scala b/graph/src/main/scala/org/apache/spark/graph/Graph.scala index 8c28cd4373891..f824b75af9ac0 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Graph.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Graph.scala @@ -225,6 +225,11 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * be commutative and assosciative and is used to combine the output * of the map phase. * + * @param activeSet 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. `activeSet` must have the same index as the + * graph's vertices. + * * @example We can use this function to compute the inDegree of each * vertex * {{{ @@ -243,8 +248,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { def mapReduceTriplets[A: ClassManifest]( mapFunc: EdgeTriplet[VD, ED] => Iterator[(Vid, A)], reduceFunc: (A, A) => A, - skipStaleSrc: Boolean = false, - skipStaleDst: Boolean = false) + activeSetOpt: Option[(VertexRDD[_], EdgeDirection)] = None) : VertexRDD[A] /** @@ -280,14 +284,6 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { (mapFunc: (Vid, VD, Option[U]) => VD2) : Graph[VD2, ED] - /** - * Replace vertices in the graph with corresponding vertices in `updates`, and restrict vertices - * without a corresponding vertex in `updates`. Edges adjacent to restricted vertices will still - * appear in graph.edges, but not in triplets or mapReduceTriplets. - */ - def innerJoinVertices[U: ClassManifest, VD2: ClassManifest](table: RDD[(Vid, U)]) - (f: (Vid, VD, U) => VD2): Graph[VD2, ED] - // 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) diff --git a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala index 6bb8bd77e87b7..285e857b693fc 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala @@ -98,15 +98,15 @@ object Pregel { // Loop var i = 0 while (activeMessages > 0 && i < maxIterations) { - // Receive the messages. Vertices that didn't get any messages do not appear in changedVerts. + // Receive the messages. Vertices that didn't get any messages do not appear in newVerts. val newVerts = g.vertices.innerJoin(messages)(vprog).cache() - // Update the graph with the new vertices, removing vertices that didn't get any messages. - g = g.innerJoinVertices(newVerts) + // Update the graph with the new vertices. + g = g.outerJoinVertices(newVerts) { (vid, old, newOpt) => newOpt.getOrElse(old) } val oldMessages = messages - // Send new messages. Vertices that didn't get any messages in the previous round don't appear - // in the graph, so don't get to send messages. - messages = g.mapReduceTriplets(sendMsg, mergeMsg).cache() + // Send new messages. Vertices that didn't get any messages don't appear in newVerts, so don't + // get to send messages. + messages = g.mapReduceTriplets(sendMsg, mergeMsg, Some((newVerts, EdgeDirection.Out))).cache() activeMessages = messages.count() // after counting we can unpersist the old messages oldMessages.unpersist(blocking=false) diff --git a/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala b/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala index 5b8e00aa13892..65164e842f8d1 100644 --- a/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala @@ -121,7 +121,7 @@ class VertexRDD[@specialized VD: ClassManifest]( * Return a new VertexRDD by applying a function to corresponding * VertexPartitions of this VertexRDD and another one. */ - private def zipVertexPartitions[VD2: ClassManifest, VD3: ClassManifest] + def zipVertexPartitions[VD2: ClassManifest, VD3: ClassManifest] (other: VertexRDD[VD2]) (f: (VertexPartition[VD], VertexPartition[VD2]) => VertexPartition[VD3]): VertexRDD[VD3] = { val newPartitionsRDD = partitionsRDD.zipPartitions( @@ -298,7 +298,7 @@ class VertexRDD[@specialized VD: ClassManifest]( case other: VertexRDD[_] => innerZipJoin(other)(f) case _ => - new VertexRDD[VD]( + new VertexRDD( partitionsRDD.zipPartitions( other.partitionBy(this.partitioner.get), preservesPartitioning = true) { (part, msgs) => diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index a954e7f907e84..aeae3780512eb 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -40,14 +40,14 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( this(vertices, edges, vertexPlacement, new VTableReplicated(vertices, edges, vertexPlacement)) } - /** Return a RDD that brings edges with its source and destination vertices together. */ + /** Return a RDD that brings edges together with their source and destination vertices. */ @transient override val triplets: RDD[EdgeTriplet[VD, ED]] = { val vdManifest = classManifest[VD] val edManifest = classManifest[ED] - edges.zipEdgePartitions(vTableReplicated.bothAttrs) { (edgePartition, vTableReplicatedIter) => - val (_, vPart) = vTableReplicatedIter.next() - new EdgeTripletIterator(vPart.index, vPart.values, edgePartition)(vdManifest, edManifest) + edges.zipEdgePartitions(vTableReplicated.get(true, true, None)) { (ePart, vPartIter) => + val (_, vPart) = vPartIter.next() + new EdgeTripletIterator(vPart.index, vPart.values, ePart)(vdManifest, edManifest) } } @@ -120,9 +120,6 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( println("\n\nvertexPlacement.bothAttrs -------------------------------") traverseLineage(vertexPlacement.bothAttrs, " ", visited) visited += (vertexPlacement.bothAttrs.id -> "vertexPlacement.bothAttrs") - println("\n\nvTableReplicated.bothAttrs ----------------") - traverseLineage(vTableReplicated.bothAttrs, " ", visited) - visited += (vTableReplicated.bothAttrs.id -> "vTableReplicated.bothAttrs") println("\n\ntriplets ----------------------------------------") traverseLineage(triplets, " ", visited) println(visited) @@ -154,7 +151,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( // manifest from GraphImpl (which would require serializing GraphImpl). val vdManifest = classManifest[VD] val newETable = - edges.zipEdgePartitions(vTableReplicated.bothAttrs) { (edgePartition, vTableReplicatedIter) => + edges.zipEdgePartitions(vTableReplicated.get(true, true, None)) { (edgePartition, vTableReplicatedIter) => val (pid, vPart) = vTableReplicatedIter.next() val et = new EdgeTriplet[VD, ED] val newEdgePartition = edgePartition.map { e => @@ -206,8 +203,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( override def mapReduceTriplets[A: ClassManifest]( mapFunc: EdgeTriplet[VD, ED] => Iterator[(Vid, A)], reduceFunc: (A, A) => A, - skipStaleSrc: Boolean = false, - skipStaleDst: Boolean = false): VertexRDD[A] = { + activeSetOpt: Option[(VertexRDD[_], EdgeDirection)] = None) = { ClosureCleaner.clean(mapFunc) ClosureCleaner.clean(reduceFunc) @@ -216,21 +212,27 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( // in the relevant position in an edge. val mapUsesSrcAttr = accessesVertexAttr[VD, ED](mapFunc, "srcAttr") val mapUsesDstAttr = accessesVertexAttr[VD, ED](mapFunc, "dstAttr") - val vs = vTableReplicated.get(mapUsesSrcAttr, mapUsesDstAttr) + val vs = vTableReplicated.get(mapUsesSrcAttr, mapUsesDstAttr, activeSetOpt.map(_._1)) + val activeDirectionOpt = activeSetOpt.map(_._2) // Map and combine. val preAgg = edges.zipEdgePartitions(vs) { (edgePartition, vTableReplicatedIter) => - val (_, vertexPartition) = vTableReplicatedIter.next() - + val (pid, vertexPartition) = vTableReplicatedIter.next() // Iterate over the partition val et = new EdgeTriplet[VD, ED] val filteredEdges = edgePartition.iterator.flatMap { e => - // Ensure that the edge meets the requirements of skipStaleSrc and skipStaleDst - et.srcStale = vertexPartition.isStale(e.srcId) - et.dstStale = vertexPartition.isStale(e.dstId) - val skipDueToSrc = skipStaleSrc && et.srcStale - val skipDueToDst = skipStaleDst && et.dstStale - if (!skipDueToSrc && !skipDueToDst) { + // Ensure the edge is adjacent to a vertex in activeSet if necessary + val adjacent = activeDirectionOpt match { + case Some(EdgeDirection.In) => + vertexPartition.isActive(e.dstId) + case Some(EdgeDirection.Out) => + vertexPartition.isActive(e.srcId) + case Some(EdgeDirection.Both) => + vertexPartition.isActive(e.srcId) && vertexPartition.isActive(e.dstId) + case None => + true + } + if (adjacent) { et.set(e) if (mapUsesSrcAttr) { et.srcAttr = vertexPartition(e.srcId) @@ -267,26 +269,6 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( new GraphImpl(newVerts, edges, vertexPlacement) } } - - override def innerJoinVertices[U: ClassManifest, VD2: ClassManifest](table: RDD[(Vid, U)]) - (f: (Vid, VD, U) => VD2): Graph[VD2, ED] - if (classManifest[VD] equals classManifest[VD2]) { - // f preserves type, so we can use incremental replication - val newVerts = vertices.innerJoin(table)(f) - val changedVerts = vertices.asInstanceOf[VertexRDD[VD2]].diff(newVerts) - // TODO(ankurdave): Need to resolve conflict between vertices that didn't change so are not - // moved but still need to run, and vertices that were deleted by the innerJoin so should not - // run - val newVTableReplicated = new VTableReplicated( - changedVerts, edges, vertexPlacement, - Some(vTableReplicated.asInstanceOf[VTableReplicated[VD2]])) - new GraphImpl(newVerts, edges, vertexPlacement, newVTableReplicated) - } else { - // updateF does not preserve type, so we must re-replicate all vertices in table - val newVerts = vertices.innerJoin(table)(f) - new GraphImpl(newVerts, edges, vertexPlacement) - } - } } // end of class GraphImpl diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala b/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala index b90d2f14ee356..eb045b7f87e03 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala @@ -44,66 +44,103 @@ class VTableReplicated[VD: ClassManifest]( }, preservesPartitioning = true).cache() } - val bothAttrs: RDD[(Pid, VertexPartition[VD])] = createVTableReplicated(true, true) - val srcAttrOnly: RDD[(Pid, VertexPartition[VD])] = createVTableReplicated(true, false) - val dstAttrOnly: RDD[(Pid, VertexPartition[VD])] = createVTableReplicated(false, true) - val noAttrs: RDD[(Pid, VertexPartition[VD])] = createVTableReplicated(false, false) - - def get(includeSrc: Boolean, includeDst: Boolean): RDD[(Pid, VertexPartition[VD])] = { - (includeSrc, includeDst) match { - case (true, true) => bothAttrs - case (true, false) => srcAttrOnly - case (false, true) => dstAttrOnly - case (false, false) => noAttrs - } - } + def get(includeSrc: Boolean, includeDst: Boolean, activesOpt: Option[VertexRDD[_]] = None) + : RDD[(Pid, VertexPartition[VD])] = { - private def createVTableReplicated( - includeSrcAttr: Boolean, includeDstAttr: Boolean): RDD[(Pid, VertexPartition[VD])] = { - - val placement = vertexPlacement.get(includeSrcAttr, includeDstAttr) val vdManifest = classManifest[VD] // Ship vertex attributes to edge partitions according to vertexPlacement - val shippedVerts = placement - .zipPartitions(updatedVerts.partitionsRDD)(VTableReplicated.buildBuffer(_, _)(vdManifest)) + val verts = updatedVerts.partitionsRDD + val shippedVerts = vertexPlacement.get(includeSrc, includeDst) + .zipPartitions(verts)(VTableReplicated.buildBuffer(_, _)(vdManifest)) .partitionBy(edges.partitioner.get).cache() // TODO: Consider using a specialized shuffler. + // Ship active sets to edge partitions using vertexPlacement, but ignoring includeSrc and + // includeDst. These flags govern attribute shipping, but the activeness of a vertex must be + // shipped to all edges mentioning that vertex, regardless of whether the vertex attribute is + // also shipped there. + val shippedActivesOpt = activesOpt.map { actives => + vertexPlacement.get(true, true) + .zipPartitions(actives.partitionsRDD)(VTableReplicated.buildActiveBuffer(_, _)) + .partitionBy(edges.partitioner.get).cache() // TODO(ankurdave): Why do we cache this? + } + prevVTableReplicated match { case Some(vTableReplicated) => val prevView: RDD[(Pid, VertexPartition[VD])] = - vTableReplicated.get(includeSrcAttr, includeDstAttr) - - // Update vTableReplicated with updatedVerts, setting staleness flags in the resulting - // VertexPartitions - prevView.zipPartitions(shippedVerts) { (prevViewIter, shippedVertsIter) => - val (pid, prevVPart) = prevViewIter.next() - val newVPart = prevVPart.innerJoinKeepLeft(shippedVertsIter.flatMap(_._2.iterator)) - Iterator((pid, newVPart)) - }.cache().setName("VTableReplicated delta %s %s".format(includeSrcAttr, includeDstAttr)) + vTableReplicated.get(includeSrc, includeDst) + + // Update vTableReplicated with updatedVerts, setting staleness and activeness flags in the + // resulting VertexPartitions + shippedActivesOpt match { + case Some(shippedActives) => + prevView.zipPartitions(shippedVerts, shippedActives) { + (prevViewIter, shippedVertsIter, shippedActivesIter) => + val (pid, prevVPart) = prevViewIter.next() + val newVPart = prevVPart + .innerJoinKeepLeft(shippedVertsIter.flatMap(_._2.iterator)) + .replaceActives(shippedActivesIter.flatMap(_._2.iterator)) + Iterator((pid, newVPart)) + }.cache().setName("VTableReplicated delta actives %s %s".format(includeSrc, includeDst)) + case None => + prevView.zipPartitions(shippedVerts) { + (prevViewIter, shippedVertsIter) => + val (pid, prevVPart) = prevViewIter.next() + val newVPart = prevVPart + .innerJoinKeepLeft(shippedVertsIter.flatMap(_._2.iterator)) + Iterator((pid, newVPart)) + }.cache().setName("VTableReplicated delta %s %s".format(includeSrc, includeDst)) + } case None => - // Within each edge partition, place the vertex attributes received from // msgsByPartition into the correct locations specified in localVidMap - localVidMap.zipPartitions(shippedVerts) { (mapIter, shippedVertsIter) => - val (pid, vidToIndex) = mapIter.next() - assert(!mapIter.hasNext) - // Populate the vertex array using the vidToIndex map - val vertexArray = vdManifest.newArray(vidToIndex.capacity) - for ((_, block) <- shippedVertsIter) { - for (i <- 0 until block.vids.size) { - val vid = block.vids(i) - val attr = block.attrs(i) - val ind = vidToIndex.getPos(vid) & OpenHashSet.POSITION_MASK - vertexArray(ind) = attr - } - } - val newVPart = new VertexPartition( - vidToIndex, vertexArray, vidToIndex.getBitSet)(vdManifest) - Iterator((pid, newVPart)) - }.cache().setName("VTableReplicated %s %s".format(includeSrcAttr, includeDstAttr)) + shippedActivesOpt match { + case Some(shippedActives) => + localVidMap.zipPartitions(shippedVerts, shippedActives) { + (mapIter, shippedVertsIter, shippedActivesIter) => + val (pid, vidToIndex) = mapIter.next() + assert(!mapIter.hasNext) + // Populate the vertex array using the vidToIndex map + val vertexArray = vdManifest.newArray(vidToIndex.capacity) + for ((_, block) <- shippedVertsIter) { + for (i <- 0 until block.vids.size) { + val vid = block.vids(i) + val attr = block.attrs(i) + val ind = vidToIndex.getPos(vid) & OpenHashSet.POSITION_MASK + vertexArray(ind) = attr + } + } + // Populate the activeSet with the received actives + val activeSet = new VertexSet(vidToIndex.capacity) + for (activeVid <- shippedActivesIter.flatMap(_._2.iterator)) { + activeSet.add(activeVid) + } + val newVPart = new VertexPartition( + vidToIndex, vertexArray, vidToIndex.getBitSet, Some(activeSet))(vdManifest) + Iterator((pid, newVPart)) + }.cache().setName("VTableReplicated active %s %s".format(includeSrc, includeDst)) + + case None => + localVidMap.zipPartitions(shippedVerts) { (mapIter, shippedVertsIter) => + val (pid, vidToIndex) = mapIter.next() + assert(!mapIter.hasNext) + // Populate the vertex array using the vidToIndex map + val vertexArray = vdManifest.newArray(vidToIndex.capacity) + for ((_, block) <- shippedVertsIter) { + for (i <- 0 until block.vids.size) { + val vid = block.vids(i) + val attr = block.attrs(i) + val ind = vidToIndex.getPos(vid) & OpenHashSet.POSITION_MASK + vertexArray(ind) = attr + } + } + val newVPart = new VertexPartition( + vidToIndex, vertexArray, vidToIndex.getBitSet)(vdManifest) + Iterator((pid, newVPart)) + }.cache().setName("VTableReplicated %s %s".format(includeSrc, includeDst)) + } } } } @@ -132,6 +169,29 @@ object VTableReplicated { (pid, new VertexAttributeBlock(vids.trim().array, attrs.trim().array)) } } + + protected def buildActiveBuffer( + pid2vidIter: Iterator[Array[Array[Vid]]], + activePartIter: Iterator[VertexPartition[_]]) + : Iterator[(Int, Array[Vid])] = { + val pid2vid: Array[Array[Vid]] = pid2vidIter.next() + val activePart: VertexPartition[_] = activePartIter.next() + + Iterator.tabulate(pid2vid.size) { pid => + val vidsCandidate = pid2vid(pid) + val size = vidsCandidate.length + val actives = new PrimitiveVector[Vid](vidsCandidate.size) + var i = 0 + while (i < size) { + val vid = vidsCandidate(i) + if (activePart.isDefined(vid)) { + actives += vid + } + i += 1 + } + (pid, actives.trim().array) + } + } } class VertexAttributeBlock[VD: ClassManifest](val vids: Array[Vid], val attrs: Array[VD]) { diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala index c4aff6455dafe..66ebf7d5aa7fe 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala @@ -32,7 +32,9 @@ private[graph] class VertexPartition[@specialized(Long, Int, Double) VD: ClassManifest]( val index: VertexIdToIndexMap, val values: Array[VD], - val mask: BitSet) + val mask: BitSet, + /** A set of vids of active vertices. May contain vids not in index due to join rewrite. */ + private val activeSet: Option[VertexSet] = None) extends Logging { val capacity: Int = index.capacity @@ -47,13 +49,10 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassManifest]( pos >= 0 && mask.get(pos) } - /** - * A vertex is stale if it is present in the index but hidden by the mask. In contrast, a vertex - * is nonexistent (possibly due to join rewrite) if it is not present in the index at all. - */ - def isStale(vid: Vid): Boolean = { + /** Look up vid in activeSet, throwing an exception if it is None. */ + def isActive(vid: Vid): Boolean = { val pos = index.getPos(vid) - pos >= 0 && !mask.get(pos) + pos >= 0 && activeSet.get.contains(pos) } /** @@ -66,7 +65,7 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassManifest]( * attribute in the RDD * * @return a new VertexPartition with values obtained by applying `f` to - * each of the entries in the original VertexSet. The resulting + * each of the entries in the original VertexRDD. The resulting * VertexPartition retains the same index. */ def map[VD2: ClassManifest](f: (Vid, VD) => VD2): VertexPartition[VD2] = { @@ -185,6 +184,15 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassManifest]( new VertexPartition(index, newValues, newMask) } + /** + * Inner join an iterator of messages. + */ + def innerJoin[U: ClassManifest, VD2: ClassManifest] + (iter: Iterator[Product2[Vid, U]]) + (f: (Vid, VD, U) => VD2): VertexPartition[VD2] = { + innerJoin(createUsingIndex(iter))(f) + } + /** * Similar effect as aggregateUsingIndex((a, b) => a) */ @@ -204,8 +212,7 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassManifest]( * Similar to innerJoin, but vertices from the left side that don't appear in iter will remain in * the partition, hidden by the bitmask. */ - def innerJoinKeepLeft(iter: Iterator[Product2[Vid, VD]]) - : VertexPartition[VD] = { + def innerJoinKeepLeft(iter: Iterator[Product2[Vid, VD]]): VertexPartition[VD] = { val newMask = new BitSet(capacity) val newValues = new Array[VD](capacity) System.arraycopy(values, 0, newValues, 0, newValues.length) @@ -236,6 +243,12 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassManifest]( new VertexPartition[VD2](index, newValues, newMask) } + def replaceActives(iter: Iterator[Vid]): VertexPartition[VD] = { + val newActiveSet = new VertexSet + iter.foreach(newActiveSet.add(_)) + new VertexPartition(index, values, mask, Some(newActiveSet)) + } + /** * Construct a new VertexPartition whose index contains only the vertices in the mask. */ @@ -249,4 +262,6 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassManifest]( } def iterator: Iterator[(Vid, VD)] = mask.iterator.map(ind => (index.getValue(ind), values(ind))) + + def vidIterator: Iterator[Vid] = mask.iterator.map(ind => index.getValue(ind)) } diff --git a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala index 2f5b64a61212d..6a3838c72f429 100644 --- a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala @@ -86,23 +86,19 @@ class GraphSuite extends FunSuite with LocalSparkContext { (a: Int, b: Int) => a + b) assert(neighborDegreeSums.collect().toSet === (0 to n).map(x => (x, n)).toSet) - // skipStale - val allPairs = for (x <- 1 to n; y <- x + 1 to n) yield (x: Vid, y: Vid) + // activeSetOpt + val allPairs = for (x <- 1 to n; y <- 1 to n) yield (x: Vid, y: Vid) val complete = Graph.fromEdgeTuples(sc.parallelize(allPairs, 1), 0) val vids = complete.mapVertices((vid, attr) => vid).cache() - vids.triplets.foreach(x => {}) // force replicated vertex view to be materialized - val vids2 = vids.mapVertices((vid, attr) => if (attr % 2 == 0) -attr else attr).cache() - val numEvenNeighbors = vids2.mapReduceTriplets(et => { - // Map function should only run on changed vertices - if (et.srcId % 2 != 0) { - throw new Exception("map ran on edge with src vid %d, which is odd".format(et.srcId)) - } + val active = vids.vertices.filter { case (vid, attr) => attr % 2 == 0 } + val numEvenNeighbors = vids.mapReduceTriplets(et => { + // Map function should only run on edges with destination in the active set if (et.dstId % 2 != 0) { throw new Exception("map ran on edge with dst vid %d, which is odd".format(et.dstId)) } - Iterator((et.srcId, 1), (et.dstId, 1)) - }, (a: Int, b: Int) => a + b, skipStaleSrc = true, skipStaleDst = true).collect.toSet - assert(numEvenNeighbors === (2 to n by 2).map(x => (x: Vid, n / 2 - 1)).toSet) + Iterator((et.srcId, 1)) + }, (a: Int, b: Int) => a + b, Some((active, EdgeDirection.In))).collect.toSet + assert(numEvenNeighbors === (1 to n).map(x => (x: Vid, n / 2)).toSet) } } @@ -195,34 +191,4 @@ class GraphSuite extends FunSuite with LocalSparkContext { assert(subgraph.edges.map(_.copy()).collect().toSet === (2 to n by 2).map(x => Edge(0, x, 1)).toSet) } } - - test("updateVertices") { - withSpark(new SparkContext("local", "test")) { sc => - // Create a star graph of 10 vertices - val n = 10 - val star = Graph.fromEdgeTuples(sc.parallelize((1 to n).map(x => (0: Vid, x: Vid))), "v1").cache() - - // Modify only vertices whose vids are even - val changedVerts = star.vertices.filter(_._1 % 2 == 0).mapValues((vid, attr) => "v2") - - // Apply the modification to the graph - val changedStar = star.updateVertices(changedVerts) - - val newVertices = star.vertices.leftZipJoin(changedVerts) { (vid, oldVd, newVdOpt) => - newVdOpt match { - case Some(newVd) => newVd - case None => oldVd - } - } - - // The graph's vertices should be correct - assert(changedStar.vertices.collect().toSet === newVertices.collect().toSet) - - // Send the leaf attributes to the center - val sums = changedStar.mapReduceTriplets( - edge => Iterator((edge.srcId, Set(edge.dstAttr))), - (a: Set[String], b: Set[String]) => a ++ b) - assert(sums.collect().toSet === Set((0, Set("v1", "v2")))) - } - } } From b40824bc2bbe8b1641123245d60c52c1d246c0f4 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 13 Dec 2013 21:56:08 -0800 Subject: [PATCH 291/531] Allow innerJoining VertexPartitions with different indexes --- .../scala/org/apache/spark/graph/impl/VertexPartition.scala | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala index 66ebf7d5aa7fe..01b96f91df9ea 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala @@ -174,6 +174,10 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassManifest]( /** Inner join another VertexPartition. */ def innerJoin[U: ClassManifest, VD2: ClassManifest](other: VertexPartition[U]) (f: (Vid, VD, U) => VD2): VertexPartition[VD2] = { + if (index != other.index) { + logWarning("Joining two VertexPartitions with different indexes is slow.") + innerJoin(createUsingIndex(other.iterator))(f) + } val newMask = mask & other.mask val newValues = new Array[VD2](capacity) var i = newMask.nextSetBit(0) From ef17ab58cbc7feaf6c381383b1759211569f7ec8 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 13 Dec 2013 21:56:28 -0800 Subject: [PATCH 292/531] Add another mrTriplets test --- .../org/apache/spark/graph/GraphSuite.scala | 17 ++++++++++++++++- 1 file changed, 16 insertions(+), 1 deletion(-) diff --git a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala index 6a3838c72f429..68a171b12f211 100644 --- a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala @@ -88,7 +88,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { // activeSetOpt val allPairs = for (x <- 1 to n; y <- 1 to n) yield (x: Vid, y: Vid) - val complete = Graph.fromEdgeTuples(sc.parallelize(allPairs, 1), 0) + val complete = Graph.fromEdgeTuples(sc.parallelize(allPairs, 3), 0) val vids = complete.mapVertices((vid, attr) => vid).cache() val active = vids.vertices.filter { case (vid, attr) => attr % 2 == 0 } val numEvenNeighbors = vids.mapReduceTriplets(et => { @@ -99,6 +99,21 @@ class GraphSuite extends FunSuite with LocalSparkContext { Iterator((et.srcId, 1)) }, (a: Int, b: Int) => a + b, Some((active, EdgeDirection.In))).collect.toSet assert(numEvenNeighbors === (1 to n).map(x => (x: Vid, n / 2)).toSet) + + // outerJoinVertices followed by mapReduceTriplets(activeSetOpt) + val ring = Graph.fromEdgeTuples(sc.parallelize((0 until n).map(x => (x: Vid, (x+1) % n: Vid)), 3), 0) + .mapVertices((vid, attr) => vid).cache() + val changed = ring.vertices.filter { case (vid, attr) => attr % 2 == 1 }.mapValues(-_) + val changedGraph = ring.outerJoinVertices(changed) { (vid, old, newOpt) => newOpt.getOrElse(old) } + val numOddNeighbors = changedGraph.mapReduceTriplets(et => { + // Map function should only run on edges with source in the active set + if (et.srcId % 2 != 1) { + throw new Exception("map ran on edge with src vid %d, which is even".format(et.dstId)) + } + Iterator((et.dstId, 1)) + }, (a: Int, b: Int) => a + b, Some(changed, EdgeDirection.Out)).collect.toSet + assert(numOddNeighbors === (2 to n by 2).map(x => (x: Vid, 1)).toSet) + } } From 9c8b6224ec58df8ba391363cadb5327d1dbf0872 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Sat, 14 Dec 2013 03:21:17 -0800 Subject: [PATCH 293/531] Remove unused bound variable --- .../src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index aeae3780512eb..d15423e611a2f 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -217,7 +217,8 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( // Map and combine. val preAgg = edges.zipEdgePartitions(vs) { (edgePartition, vTableReplicatedIter) => - val (pid, vertexPartition) = vTableReplicatedIter.next() + val (_, vertexPartition) = vTableReplicatedIter.next() + // Iterate over the partition val et = new EdgeTriplet[VD, ED] val filteredEdges = edgePartition.iterator.flatMap { e => From cf6288b993033d087dfb243c590c9428d91004b8 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Sat, 14 Dec 2013 03:21:39 -0800 Subject: [PATCH 294/531] Add PregelSuite --- .../org/apache/spark/graph/PregelSuite.scala | 43 +++++++++++++++++++ 1 file changed, 43 insertions(+) create mode 100644 graph/src/test/scala/org/apache/spark/graph/PregelSuite.scala diff --git a/graph/src/test/scala/org/apache/spark/graph/PregelSuite.scala b/graph/src/test/scala/org/apache/spark/graph/PregelSuite.scala new file mode 100644 index 0000000000000..cc6631358ffd6 --- /dev/null +++ b/graph/src/test/scala/org/apache/spark/graph/PregelSuite.scala @@ -0,0 +1,43 @@ +package org.apache.spark.graph + +import org.scalatest.FunSuite + +import org.apache.spark.SparkContext +import org.apache.spark.graph.LocalSparkContext._ +import org.apache.spark.rdd._ + +class PregelSuite extends FunSuite with LocalSparkContext { + + System.setProperty("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + System.setProperty("spark.kryo.registrator", "org.apache.spark.graph.GraphKryoRegistrator") + + test("1 iteration") { + withSpark(new SparkContext("local", "test")) { sc => + val n = 5 + val star = Graph.fromEdgeTuples(sc.parallelize((1 to n).map(x => (0: Vid, x: Vid)), 3), "v") + val result = Pregel(star, 0)( + (vid, attr, msg) => attr, + et => Iterator.empty, + (a: Int, b: Int) => throw new Exception("mergeMsg run unexpectedly")) + assert(result.vertices.collect.toSet === star.vertices.collect.toSet) + } + } + + test("chain propagation") { + withSpark(new SparkContext("local", "test")) { sc => + val n = 5 + val chain = Graph.fromEdgeTuples( + sc.parallelize((1 until n).map(x => (x: Vid, x + 1: Vid)), 3), + 0).cache() + assert(chain.vertices.collect.toSet === (1 to n).map(x => (x: Vid, 0)).toSet) + val chainWithSeed = chain.mapVertices { (vid, attr) => if (vid == 1) 1 else 0 } + assert(chainWithSeed.vertices.collect.toSet === Set((1: Vid, 1)) ++ (2 to n).map(x => (x: Vid, 0)).toSet) + val result = Pregel(chainWithSeed, 0)( + (vid, attr, msg) => { println("vprog on " + (vid, attr, msg)); math.max(msg, attr) }, + et => { println("sendMsg on " + ((et.srcId, et.srcAttr), (et.dstId, et.dstAttr))); Iterator((et.dstId, et.srcAttr)) }, + (a: Int, b: Int) => { println("mergeMsg on " + (a, b)); math.max(a, b) }) + assert(result.vertices.collect.toSet === + chain.vertices.mapValues { (vid, attr) => attr + 1 }.collect.toSet) + } + } +} From 4d3bba3a1336bb2762e050ed515de0fac3add252 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Sat, 14 Dec 2013 03:22:03 -0800 Subject: [PATCH 295/531] Add debug logging to Pregel --- graph/src/main/scala/org/apache/spark/graph/Pregel.scala | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala index 285e857b693fc..31ca7ff09a2cc 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala @@ -92,22 +92,29 @@ object Pregel { : Graph[VD, ED] = { var g = graph.mapVertices( (vid, vdata) => vprog(vid, vdata, initialMsg) ) + println("[pre] g: " + g.vertices.cache().collect.mkString(",")) // compute the messages var messages = g.mapReduceTriplets(sendMsg, mergeMsg).cache() + println("[pre] messages: " + messages.collect.mkString(",")) var activeMessages = messages.count() + println("Pregel pre-run, %d active messages".format(activeMessages)) // Loop var i = 0 while (activeMessages > 0 && i < maxIterations) { // Receive the messages. Vertices that didn't get any messages do not appear in newVerts. val newVerts = g.vertices.innerJoin(messages)(vprog).cache() + println("newVerts: " + newVerts.collect.mkString(",")) // Update the graph with the new vertices. g = g.outerJoinVertices(newVerts) { (vid, old, newOpt) => newOpt.getOrElse(old) } + println("g: " + g.vertices.cache().collect.mkString(",")) val oldMessages = messages // Send new messages. Vertices that didn't get any messages don't appear in newVerts, so don't // get to send messages. messages = g.mapReduceTriplets(sendMsg, mergeMsg, Some((newVerts, EdgeDirection.Out))).cache() + println("messages: " + messages.collect.mkString(",")) activeMessages = messages.count() + println("Pregel iter %d, %d active messages".format(i, activeMessages)) // after counting we can unpersist the old messages oldMessages.unpersist(blocking=false) // count the iteration From ee5c69e481c4a8c883d3992b9ab5a7e8f87a0b41 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Sat, 14 Dec 2013 04:21:55 -0800 Subject: [PATCH 296/531] Fix bug in VertexPartition.isActive This took me ~5 hours to find! --- .../scala/org/apache/spark/graph/impl/VertexPartition.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala index 01b96f91df9ea..1c589c9b72a4c 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala @@ -51,8 +51,7 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassManifest]( /** Look up vid in activeSet, throwing an exception if it is None. */ def isActive(vid: Vid): Boolean = { - val pos = index.getPos(vid) - pos >= 0 && activeSet.get.contains(pos) + activeSet.get.contains(vid) } /** From 59f625b7454d882c06461222c3e305910fff9594 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Sat, 14 Dec 2013 04:24:40 -0800 Subject: [PATCH 297/531] Revert "Add debug logging to Pregel" This reverts commit e62013cdd92137507a96b6a5b347a1d239209587. --- graph/src/main/scala/org/apache/spark/graph/Pregel.scala | 7 ------- 1 file changed, 7 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala index 31ca7ff09a2cc..285e857b693fc 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala @@ -92,29 +92,22 @@ object Pregel { : Graph[VD, ED] = { var g = graph.mapVertices( (vid, vdata) => vprog(vid, vdata, initialMsg) ) - println("[pre] g: " + g.vertices.cache().collect.mkString(",")) // compute the messages var messages = g.mapReduceTriplets(sendMsg, mergeMsg).cache() - println("[pre] messages: " + messages.collect.mkString(",")) var activeMessages = messages.count() - println("Pregel pre-run, %d active messages".format(activeMessages)) // Loop var i = 0 while (activeMessages > 0 && i < maxIterations) { // Receive the messages. Vertices that didn't get any messages do not appear in newVerts. val newVerts = g.vertices.innerJoin(messages)(vprog).cache() - println("newVerts: " + newVerts.collect.mkString(",")) // Update the graph with the new vertices. g = g.outerJoinVertices(newVerts) { (vid, old, newOpt) => newOpt.getOrElse(old) } - println("g: " + g.vertices.cache().collect.mkString(",")) val oldMessages = messages // Send new messages. Vertices that didn't get any messages don't appear in newVerts, so don't // get to send messages. messages = g.mapReduceTriplets(sendMsg, mergeMsg, Some((newVerts, EdgeDirection.Out))).cache() - println("messages: " + messages.collect.mkString(",")) activeMessages = messages.count() - println("Pregel iter %d, %d active messages".format(i, activeMessages)) // after counting we can unpersist the old messages oldMessages.unpersist(blocking=false) // count the iteration From 0c3fc1c1b62be702e89a651ef1ef442a984f1bba Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Sat, 14 Dec 2013 05:22:46 -0800 Subject: [PATCH 298/531] Avoid re-creating the view RDDs multiple times Previously, successive operations that support incremental view maintenance would inadvertently recreate previous view RDDs by calling VTableReplicated.get(), which created the RDDs anew though they were already cached. This change memoizes the RDDs and separates the process of shipping active-set information to an existing view. --- .../apache/spark/graph/impl/GraphImpl.scala | 10 +- .../spark/graph/impl/VTableReplicated.scala | 146 ++++++++---------- .../spark/graph/impl/VertexPlacement.scala | 13 +- 3 files changed, 74 insertions(+), 95 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index d15423e611a2f..e3d5e37c8d1ac 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -45,7 +45,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( val vdManifest = classManifest[VD] val edManifest = classManifest[ED] - edges.zipEdgePartitions(vTableReplicated.get(true, true, None)) { (ePart, vPartIter) => + edges.zipEdgePartitions(vTableReplicated.get(true, true)) { (ePart, vPartIter) => val (_, vPart) = vPartIter.next() new EdgeTripletIterator(vPart.index, vPart.values, ePart)(vdManifest, edManifest) } @@ -54,7 +54,6 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( override def persist(newLevel: StorageLevel): Graph[VD, ED] = { vertices.persist(newLevel) edges.persist(newLevel) - vertexPlacement.persist(newLevel) this } @@ -151,7 +150,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( // manifest from GraphImpl (which would require serializing GraphImpl). val vdManifest = classManifest[VD] val newETable = - edges.zipEdgePartitions(vTableReplicated.get(true, true, None)) { (edgePartition, vTableReplicatedIter) => + edges.zipEdgePartitions(vTableReplicated.get(true, true)) { (edgePartition, vTableReplicatedIter) => val (pid, vPart) = vTableReplicatedIter.next() val et = new EdgeTriplet[VD, ED] val newEdgePartition = edgePartition.map { e => @@ -212,7 +211,10 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( // in the relevant position in an edge. val mapUsesSrcAttr = accessesVertexAttr[VD, ED](mapFunc, "srcAttr") val mapUsesDstAttr = accessesVertexAttr[VD, ED](mapFunc, "dstAttr") - val vs = vTableReplicated.get(mapUsesSrcAttr, mapUsesDstAttr, activeSetOpt.map(_._1)) + val vs = activeSetOpt match { + case Some((activeSet, _)) => vTableReplicated.get(mapUsesSrcAttr, mapUsesDstAttr, activeSet) + case None => vTableReplicated.get(mapUsesSrcAttr, mapUsesDstAttr) + } val activeDirectionOpt = activeSetOpt.map(_._2) // Map and combine. diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala b/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala index eb045b7f87e03..6124dfab83a78 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala @@ -41,106 +41,88 @@ class VTableReplicated[VD: ClassManifest]( vidToIndex.add(e.dstId) } (pid, vidToIndex) - }, preservesPartitioning = true).cache() + }, preservesPartitioning = true).cache().setName("VTableReplicated localVidMap") } - def get(includeSrc: Boolean, includeDst: Boolean, activesOpt: Option[VertexRDD[_]] = None) - : RDD[(Pid, VertexPartition[VD])] = { + private lazy val bothAttrs: RDD[(Pid, VertexPartition[VD])] = create(true, true) + private lazy val srcAttrOnly: RDD[(Pid, VertexPartition[VD])] = create(true, false) + private lazy val dstAttrOnly: RDD[(Pid, VertexPartition[VD])] = create(false, true) + private lazy val noAttrs: RDD[(Pid, VertexPartition[VD])] = create(false, false) + + def get(includeSrc: Boolean, includeDst: Boolean): RDD[(Pid, VertexPartition[VD])] = { + (includeSrc, includeDst) match { + case (true, true) => bothAttrs + case (true, false) => srcAttrOnly + case (false, true) => dstAttrOnly + case (false, false) => noAttrs + } + } + + def get( + includeSrc: Boolean, + includeDst: Boolean, + actives: VertexRDD[_]): RDD[(Pid, VertexPartition[VD])] = { + + // Ship active sets to edge partitions using vertexPlacement, but ignoring includeSrc and + // includeDst. These flags govern attribute shipping, but the activeness of a vertex must be + // shipped to all edges mentioning that vertex, regardless of whether the vertex attribute is + // also shipped there. + val shippedActives = vertexPlacement.get(true, true) + .zipPartitions(actives.partitionsRDD)(VTableReplicated.buildActiveBuffer(_, _)) + .partitionBy(edges.partitioner.get)// .cache().setName("VTableReplicated shippedActives") + // Update vTableReplicated with shippedActives, setting activeness flags in the resulting + // VertexPartitions + get(includeSrc, includeDst).zipPartitions(shippedActives) { (viewIter, shippedActivesIter) => + val (pid, vPart) = viewIter.next() + val newPart = vPart.replaceActives(shippedActivesIter.flatMap(_._2.iterator)) + Iterator((pid, newPart)) + } + } + private def create(includeSrc: Boolean, includeDst: Boolean) + : RDD[(Pid, VertexPartition[VD])] = { val vdManifest = classManifest[VD] // Ship vertex attributes to edge partitions according to vertexPlacement val verts = updatedVerts.partitionsRDD val shippedVerts = vertexPlacement.get(includeSrc, includeDst) .zipPartitions(verts)(VTableReplicated.buildBuffer(_, _)(vdManifest)) - .partitionBy(edges.partitioner.get).cache() + .partitionBy(edges.partitioner.get)// .cache().setName("VTableReplicated shippedVerts") // TODO: Consider using a specialized shuffler. - // Ship active sets to edge partitions using vertexPlacement, but ignoring includeSrc and - // includeDst. These flags govern attribute shipping, but the activeness of a vertex must be - // shipped to all edges mentioning that vertex, regardless of whether the vertex attribute is - // also shipped there. - val shippedActivesOpt = activesOpt.map { actives => - vertexPlacement.get(true, true) - .zipPartitions(actives.partitionsRDD)(VTableReplicated.buildActiveBuffer(_, _)) - .partitionBy(edges.partitioner.get).cache() // TODO(ankurdave): Why do we cache this? - } - prevVTableReplicated match { case Some(vTableReplicated) => val prevView: RDD[(Pid, VertexPartition[VD])] = vTableReplicated.get(includeSrc, includeDst) - // Update vTableReplicated with updatedVerts, setting staleness and activeness flags in the - // resulting VertexPartitions - shippedActivesOpt match { - case Some(shippedActives) => - prevView.zipPartitions(shippedVerts, shippedActives) { - (prevViewIter, shippedVertsIter, shippedActivesIter) => - val (pid, prevVPart) = prevViewIter.next() - val newVPart = prevVPart - .innerJoinKeepLeft(shippedVertsIter.flatMap(_._2.iterator)) - .replaceActives(shippedActivesIter.flatMap(_._2.iterator)) - Iterator((pid, newVPart)) - }.cache().setName("VTableReplicated delta actives %s %s".format(includeSrc, includeDst)) - case None => - prevView.zipPartitions(shippedVerts) { - (prevViewIter, shippedVertsIter) => - val (pid, prevVPart) = prevViewIter.next() - val newVPart = prevVPart - .innerJoinKeepLeft(shippedVertsIter.flatMap(_._2.iterator)) - Iterator((pid, newVPart)) - }.cache().setName("VTableReplicated delta %s %s".format(includeSrc, includeDst)) - } + // Update vTableReplicated with shippedVerts, setting staleness flags in the resulting + // VertexPartitions + prevView.zipPartitions(shippedVerts) { (prevViewIter, shippedVertsIter) => + val (pid, prevVPart) = prevViewIter.next() + val newVPart = prevVPart.innerJoinKeepLeft(shippedVertsIter.flatMap(_._2.iterator)) + Iterator((pid, newVPart)) + }.cache().setName("VTableReplicated delta %s %s".format(includeSrc, includeDst)) case None => - // Within each edge partition, place the vertex attributes received from - // msgsByPartition into the correct locations specified in localVidMap - shippedActivesOpt match { - case Some(shippedActives) => - localVidMap.zipPartitions(shippedVerts, shippedActives) { - (mapIter, shippedVertsIter, shippedActivesIter) => - val (pid, vidToIndex) = mapIter.next() - assert(!mapIter.hasNext) - // Populate the vertex array using the vidToIndex map - val vertexArray = vdManifest.newArray(vidToIndex.capacity) - for ((_, block) <- shippedVertsIter) { - for (i <- 0 until block.vids.size) { - val vid = block.vids(i) - val attr = block.attrs(i) - val ind = vidToIndex.getPos(vid) & OpenHashSet.POSITION_MASK - vertexArray(ind) = attr - } - } - // Populate the activeSet with the received actives - val activeSet = new VertexSet(vidToIndex.capacity) - for (activeVid <- shippedActivesIter.flatMap(_._2.iterator)) { - activeSet.add(activeVid) - } - val newVPart = new VertexPartition( - vidToIndex, vertexArray, vidToIndex.getBitSet, Some(activeSet))(vdManifest) - Iterator((pid, newVPart)) - }.cache().setName("VTableReplicated active %s %s".format(includeSrc, includeDst)) - - case None => - localVidMap.zipPartitions(shippedVerts) { (mapIter, shippedVertsIter) => - val (pid, vidToIndex) = mapIter.next() - assert(!mapIter.hasNext) - // Populate the vertex array using the vidToIndex map - val vertexArray = vdManifest.newArray(vidToIndex.capacity) - for ((_, block) <- shippedVertsIter) { - for (i <- 0 until block.vids.size) { - val vid = block.vids(i) - val attr = block.attrs(i) - val ind = vidToIndex.getPos(vid) & OpenHashSet.POSITION_MASK - vertexArray(ind) = attr - } - } - val newVPart = new VertexPartition( - vidToIndex, vertexArray, vidToIndex.getBitSet)(vdManifest) - Iterator((pid, newVPart)) - }.cache().setName("VTableReplicated %s %s".format(includeSrc, includeDst)) - } + // Within each edge partition, place the shipped vertex attributes into the correct + // locations specified in localVidMap + localVidMap.zipPartitions(shippedVerts) { (mapIter, shippedVertsIter) => + val (pid, vidToIndex) = mapIter.next() + assert(!mapIter.hasNext) + // Populate the vertex array using the vidToIndex map + val vertexArray = vdManifest.newArray(vidToIndex.capacity) + for ((_, block) <- shippedVertsIter) { + for (i <- 0 until block.vids.size) { + val vid = block.vids(i) + val attr = block.attrs(i) + val ind = vidToIndex.getPos(vid) & OpenHashSet.POSITION_MASK + vertexArray(ind) = attr + } + } + val newVPart = new VertexPartition( + vidToIndex, vertexArray, vidToIndex.getBitSet)(vdManifest) + Iterator((pid, newVPart)) + }.cache().setName("VTableReplicated %s %s".format(includeSrc, includeDst)) } } } diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPlacement.scala b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPlacement.scala index 24fdf0db45cc1..44a0a05f74c62 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPlacement.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPlacement.scala @@ -7,8 +7,10 @@ import org.apache.spark.storage.StorageLevel import org.apache.spark.util.collection.PrimitiveVector /** - * Stores the layout of replicated vertex attributes for GraphImpl. Tells each - * partition of the vertex data where it should go. + * Stores the locations of edge-partition join sites for each vertex attribute in `vTable`; that is, + * the routing information for shipping vertex attributes to edge partitions. This is always cached + * because it may be used multiple times in VTableReplicated -- once to ship the vertex attributes + * and (possibly) once to ship the active-set information. */ class VertexPlacement(eTable: EdgeRDD[_], vTable: VertexRDD[_]) { @@ -25,13 +27,6 @@ class VertexPlacement(eTable: EdgeRDD[_], vTable: VertexRDD[_]) { case (false, false) => noAttrs } - def persist(newLevel: StorageLevel) { - bothAttrs.persist(newLevel) - srcAttrOnly.persist(newLevel) - dstAttrOnly.persist(newLevel) - noAttrs.persist(newLevel) - } - private def createPid2Vid( includeSrcAttr: Boolean, includeDstAttr: Boolean): RDD[Array[Array[Vid]]] = { // Determine which vertices each edge partition needs by creating a mapping from vid to pid. From 7dbd3bf8252125e7ea3726802b160af8a0aafae2 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Sat, 14 Dec 2013 15:45:23 -0800 Subject: [PATCH 299/531] Remove debug printing from PregelSuite --- .../src/test/scala/org/apache/spark/graph/PregelSuite.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/graph/src/test/scala/org/apache/spark/graph/PregelSuite.scala b/graph/src/test/scala/org/apache/spark/graph/PregelSuite.scala index cc6631358ffd6..0897d9783eaf8 100644 --- a/graph/src/test/scala/org/apache/spark/graph/PregelSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/PregelSuite.scala @@ -33,9 +33,9 @@ class PregelSuite extends FunSuite with LocalSparkContext { val chainWithSeed = chain.mapVertices { (vid, attr) => if (vid == 1) 1 else 0 } assert(chainWithSeed.vertices.collect.toSet === Set((1: Vid, 1)) ++ (2 to n).map(x => (x: Vid, 0)).toSet) val result = Pregel(chainWithSeed, 0)( - (vid, attr, msg) => { println("vprog on " + (vid, attr, msg)); math.max(msg, attr) }, - et => { println("sendMsg on " + ((et.srcId, et.srcAttr), (et.dstId, et.dstAttr))); Iterator((et.dstId, et.srcAttr)) }, - (a: Int, b: Int) => { println("mergeMsg on " + (a, b)); math.max(a, b) }) + (vid, attr, msg) => math.max(msg, attr), + et => Iterator((et.dstId, et.srcAttr)), + (a: Int, b: Int) => math.max(a, b)) assert(result.vertices.collect.toSet === chain.vertices.mapValues { (vid, attr) => attr + 1 }.collect.toSet) } From 1ad3d240b09849e389dce0fae578736f96978e36 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Sat, 14 Dec 2013 15:49:00 -0800 Subject: [PATCH 300/531] Remove commented code --- .../scala/org/apache/spark/graph/algorithms/PageRank.scala | 2 +- .../scala/org/apache/spark/graph/impl/VTableReplicated.scala | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/algorithms/PageRank.scala b/graph/src/main/scala/org/apache/spark/graph/algorithms/PageRank.scala index 07fc8595ba7c5..f77dffd7b4156 100644 --- a/graph/src/main/scala/org/apache/spark/graph/algorithms/PageRank.scala +++ b/graph/src/main/scala/org/apache/spark/graph/algorithms/PageRank.scala @@ -178,7 +178,7 @@ object PageRank extends Logging { .mapReduceTriplets[Double]( et => Iterator((et.dstId, et.srcAttr * et.attr * weight)), _ + _, - prevDeltas.map(d => (d, EdgeDirection.Out))) + prevDeltas.map((_, EdgeDirection.Out))) .filter { case (vid, delta) => delta > tol } .cache() prevDeltas = Some(deltas) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala b/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala index 6124dfab83a78..161c98f1588d4 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala @@ -69,7 +69,7 @@ class VTableReplicated[VD: ClassManifest]( // also shipped there. val shippedActives = vertexPlacement.get(true, true) .zipPartitions(actives.partitionsRDD)(VTableReplicated.buildActiveBuffer(_, _)) - .partitionBy(edges.partitioner.get)// .cache().setName("VTableReplicated shippedActives") + .partitionBy(edges.partitioner.get) // Update vTableReplicated with shippedActives, setting activeness flags in the resulting // VertexPartitions get(includeSrc, includeDst).zipPartitions(shippedActives) { (viewIter, shippedActivesIter) => @@ -87,7 +87,7 @@ class VTableReplicated[VD: ClassManifest]( val verts = updatedVerts.partitionsRDD val shippedVerts = vertexPlacement.get(includeSrc, includeDst) .zipPartitions(verts)(VTableReplicated.buildBuffer(_, _)(vdManifest)) - .partitionBy(edges.partitioner.get)// .cache().setName("VTableReplicated shippedVerts") + .partitionBy(edges.partitioner.get) // TODO: Consider using a specialized shuffler. prevVTableReplicated match { From 0459747c1c146f036f99b38a0db75372257e41a1 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Sat, 14 Dec 2013 17:08:04 -0800 Subject: [PATCH 301/531] Fix typo --- graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala b/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala index 65164e842f8d1..90ac6dc61dc3a 100644 --- a/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala @@ -288,7 +288,7 @@ class VertexRDD[@specialized VD: ClassManifest]( /** * Replace vertices with corresponding vertices in `other`, and drop vertices without a - * corresponding vertex in `other. + * corresponding vertex in `other`. */ def innerJoin[U: ClassManifest, VD2: ClassManifest](other: RDD[(Vid, U)]) (f: (Vid, VD, U) => VD2): VertexRDD[VD2] = { From 62bdc44a1e6a28d313e693474071da04caf41c02 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 6 Dec 2013 17:36:09 -0800 Subject: [PATCH 302/531] Unrolled while loop in readVarLong. (cherry picked from commit 45ffb1ae3c0527aae50502741a3585c411875b9a) Signed-off-by: Ankur Dave --- .../apache/spark/graph/impl/Serializers.scala | 44 ++++++++++++++----- 1 file changed, 34 insertions(+), 10 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala b/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala index 68b38de2b8686..e4fa4a4421786 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala @@ -295,7 +295,7 @@ abstract class ShuffleDeserializationStream(s: InputStream) extends Deserializat var i: Int = 0 def readOrThrow(): Int = { val in = s.read() - if (in < 0) throw new java.io.EOFException + if (in < 0) throw new EOFException in & 0xFF } var b: Int = readOrThrow() @@ -309,21 +309,45 @@ abstract class ShuffleDeserializationStream(s: InputStream) extends Deserializat } def readVarLong(optimizePositive: Boolean): Long = { - // TODO: unroll the while loop. - var value: Long = 0L def readOrThrow(): Int = { val in = s.read() - if (in < 0) throw new java.io.EOFException + if (in < 0) throw new EOFException in & 0xFF } - var i: Int = 0 - var b: Int = readOrThrow() - while (i < 56 && (b & 0x80) != 0) { - value |= (b & 0x7F).toLong << i - i += 7 + var b = readOrThrow() + var ret: Long = b & 0x7F + if ((b & 0x80) != 0) { b = readOrThrow() + ret |= (b & 0x7F) << 7 + if ((b & 0x80) != 0) { + b = readOrThrow() + ret |= (b & 0x7F) << 14 + if ((b & 0x80) != 0) { + b = readOrThrow() + ret |= (b & 0x7F) << 21 + if ((b & 0x80) != 0) { + b = readOrThrow() + ret |= (b & 0x7F).toLong << 28 + if ((b & 0x80) != 0) { + b = readOrThrow() + ret |= (b & 0x7F).toLong << 35 + if ((b & 0x80) != 0) { + b = readOrThrow() + ret |= (b & 0x7F).toLong << 42 + if ((b & 0x80) != 0) { + b = readOrThrow() + ret |= (b & 0x7F).toLong << 49 + if ((b & 0x80) != 0) { + b = readOrThrow() + ret |= b.toLong << 56 + } + } + } + } + } + } + } } - val ret = value | (b.toLong << i) if (!optimizePositive) (ret >>> 1) ^ -(ret & 1) else ret } From bad85b051d14270a776152524eebc89c926517d1 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 6 Dec 2013 17:36:09 -0800 Subject: [PATCH 303/531] Use murmur3 hash for open hashset. (cherry picked from commit 212ff6834515543163aa63a3f4f762ebe641f8ca) Signed-off-by: Ankur Dave --- .../scala/org/apache/spark/util/collection/OpenHashSet.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala b/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala index b8716f1db70e1..36e2a05b9c9a0 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala @@ -249,8 +249,7 @@ class OpenHashSet[@specialized(Long, Int) T: ClassManifest]( * in the lower bits, similar to java.util.HashMap */ private def hashcode(h: Int): Int = { - val r = h ^ (h >>> 20) ^ (h >>> 12) - r ^ (r >>> 7) ^ (r >>> 4) + it.unimi.dsi.fastutil.HashCommon.murmurHash3(h) } private def nextPowerOf2(n: Int): Int = { From 5eb21e6720e46aec45c13de73a9c396c7901b50a Mon Sep 17 00:00:00 2001 From: Wang Jianping J Date: Mon, 16 Dec 2013 20:01:31 +0800 Subject: [PATCH 304/531] SVD++ demo --- .../apache/spark/graph/algorithms/Svdpp.scala | 143 ++++++++++++++++++ 1 file changed, 143 insertions(+) create mode 100644 graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala diff --git a/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala b/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala new file mode 100644 index 0000000000000..a3110667bef0c --- /dev/null +++ b/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala @@ -0,0 +1,143 @@ +package org.apache.spark.graph.algorithms + +import org.apache.spark._ +import org.apache.spark.graph._ +import scala.util.Random +import org.apache.commons.math.linear._ + +class VD ( + var v1: RealVector, // v1: p for user node, q for item node + var v2: RealVector, // v2: pu + |N(u)|^(-0.5)*sum(y) for user node, y for item node + var bias: Double, + var norm: Double // only for user node +) extends Serializable + +class Msg ( + var v1: RealVector, + var v2: RealVector, + var bias: Double +) extends Serializable + +object Svdpp { + // implement SVD++ based on http://public.research.att.com/~volinsky/netflix/kdd08koren.pdf + + def run(sc: SparkContext, input: String): Graph[VD,Double] = { + // defalut parameters + val rank = 4 + val maxIters = 15 + val minVal = 0.0 + val maxVal = 5.0 + val gamma1 = 0.007 + val gamma2 = 0.007 + val gamma6 = 0.005 + val gamma7 = 0.015 + + def defaultF(rank: Int) = { + val p = new ArrayRealVector(rank) + val w = new ArrayRealVector(rank) + for (i <- 0 until rank) { + p.setEntry(i, Random.nextDouble) + w.setEntry(i, Random.nextDouble) + } + var vd = new VD(p, w, 0.0, 0.0) + vd + } + + // calculate initial norm and bias + def mapF0(et: EdgeTriplet[VD, Double]): Iterator[(Vid, (Long, Double))] = { + assert(et.srcAttr != null && et.dstAttr != null) + Iterator((et.srcId, (1L, et.attr)), (et.dstId, (1L, et.attr))) + } + def reduceF0(g1: (Long, Double), g2: (Long, Double)) = { + (g1._1 + g2._1, g1._2 + g2._2) + } + def updateF0(vid: Vid, vd: VD, msg: Option[(Long, Double)]) = { + if (msg.isDefined) { + vd.bias = msg.get._2 / msg.get._1 + vd.norm = 1.0 / scala.math.sqrt(msg.get._1) + } + vd + } + + // read textfile + val edges = sc.textFile(input).map { line => + val fields = line.split(",") + Edge(fields(0).toLong * 2, fields(1).toLong * 2 + 1, fields(2).toDouble) + } + + // calculate global rating mean + val (rs, rc) = edges.map(e => (e.attr, 1L)).reduce((a, b) => (a._1 + b._1, a._2 + b._2)) + val u = rs / rc // global rating mean + + // make graph + var g = Graph.fromEdges(edges, defaultF(rank), RandomVertexCut).cache() + + // calculate initial norm and bias + val t0 = g.mapReduceTriplets(mapF0, reduceF0) + g.outerJoinVertices(t0) {updateF0} + + // phase 1 + def mapF1(et: EdgeTriplet[VD, Double]): Iterator[(Vid, RealVector)] = { + assert(et.srcAttr != null && et.dstAttr != null) + Iterator((et.srcId, et.dstAttr.v2)) // sum up y of connected item nodes + } + def reduceF1(g1: RealVector, g2: RealVector) = { + g1.add(g2) + } + def updateF1(vid: Vid, vd: VD, msg: Option[RealVector]) = { + if (msg.isDefined) { + vd.v2 = vd.v1.add(msg.get.mapMultiply(vd.norm)) // pu + |N(u)|^(-0.5)*sum(y) + } + vd + } + + // phase 2 + def mapF2(et: EdgeTriplet[VD, Double]): Iterator[(Vid, Msg)] = { + assert(et.srcAttr != null && et.dstAttr != null) + val usr = et.srcAttr + val itm = et.dstAttr + var p = usr.v1 + var q = itm.v1 + val itmBias = 0.0 + val usrBias = 0.0 + var pred = u + usr.bias + itm.bias + q.dotProduct(usr.v2) + if (pred < minVal) { + pred = minVal + } + if (pred > maxVal) { + pred = maxVal + } + val err = et.attr - pred + val y = (q.mapMultiply(err*usr.norm)).subtract((usr.v2).mapMultiply(gamma7)) + val newP = (q.mapMultiply(err)).subtract(p.mapMultiply(gamma7)) // for each connected item q + val newQ = (usr.v2.mapMultiply(err)).subtract(q.mapMultiply(gamma7)) + Iterator((et.srcId, new Msg(newP, y, err - gamma6*usr.bias)), (et.dstId, new Msg(newQ, y, err - gamma6*itm.bias))) + } + def reduceF2(g1: Msg, g2: Msg):Msg = { + g1.v1 = g1.v1.add(g2.v1) + g1.v2 = g1.v2.add(g2.v2) + g1.bias += g2.bias + g1 + } + def updateF2(vid: Vid, vd: VD, msg: Option[Msg]) = { + if (msg.isDefined) { + vd.v1 = vd.v1.add(msg.get.v1.mapMultiply(gamma2)) + if (vid % 2 == 1) { // item node update y + vd.v2 = vd.v2.add(msg.get.v2.mapMultiply(gamma2)) + } + vd.bias += msg.get.bias*gamma1 + } + vd + } + + for (i <- 0 until maxIters) { + // phase 1 + val t1: VertexRDD[RealVector] = g.mapReduceTriplets(mapF1, reduceF1) + g.outerJoinVertices(t1) {updateF1} + // phase 2 + val t2: VertexRDD[Msg] = g.mapReduceTriplets(mapF2, reduceF2) + g.outerJoinVertices(t2) {updateF2} + } + g + } +} From 3ade8be8f218cdec3b90e48069595a3c556e0f27 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Sun, 15 Dec 2013 15:08:08 -0800 Subject: [PATCH 305/531] Add clustered index on edges by source vertex This allows efficient edge scan in mapReduceTriplets when many source vertices are inactive. The scan method switches from edge scan to clustered index scan when less than 80% of source vertices are active. --- .../scala/org/apache/spark/graph/Edge.scala | 7 ++ .../spark/graph/impl/EdgePartition.scala | 99 +++++++++++++------ .../graph/impl/EdgePartitionBuilder.scala | 39 ++++++-- .../apache/spark/graph/impl/GraphImpl.scala | 57 ++++++----- .../spark/graph/impl/VTableReplicated.scala | 2 +- .../spark/graph/impl/VertexPartition.scala | 3 + .../org/apache/spark/graph/GraphSuite.scala | 25 +++++ 7 files changed, 167 insertions(+), 65 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/Edge.scala b/graph/src/main/scala/org/apache/spark/graph/Edge.scala index 1aa1b36b4725a..7e8ae7c790b84 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Edge.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Edge.scala @@ -41,3 +41,10 @@ case class Edge[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED] def relativeDirection(vid: Vid): EdgeDirection = if (vid == srcId) EdgeDirection.Out else { assert(vid == dstId); EdgeDirection.In } } + +object Edge { + def lexicographicOrdering[ED] = new Ordering[Edge[ED]] { + override def compare(a: Edge[ED], b: Edge[ED]): Int = + Ordering[(Vid, Vid)].compare((a.srcId, a.dstId), (b.srcId, b.dstId)) + } +} diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala b/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala index eb3fd60d74f54..bfdafcc54248b 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala @@ -1,29 +1,36 @@ package org.apache.spark.graph.impl import org.apache.spark.graph._ -import org.apache.spark.util.collection.OpenHashMap +import org.apache.spark.util.collection.PrimitiveKeyOpenHashMap /** - * A collection of edges stored in 3 large columnar arrays (src, dst, attribute). + * A collection of edges stored in 3 large columnar arrays (src, dst, attribute). The arrays are + * clustered by src. * * @param srcIds the source vertex id of each edge * @param dstIds the destination vertex id of each edge * @param data the attribute associated with each edge + * @param index a clustered index on source vertex id * @tparam ED the edge attribute type. */ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED: ClassManifest]( val srcIds: Array[Vid], val dstIds: Array[Vid], - val data: Array[ED]) { + val data: Array[ED], + val index: PrimitiveKeyOpenHashMap[Vid, Int]) { /** * Reverse all the edges in this partition. * - * @note No new data structures are created. - * * @return a new edge partition with all edges reversed. */ - def reverse: EdgePartition[ED] = new EdgePartition(dstIds, srcIds, data) + def reverse: EdgePartition[ED] = { + val builder = new EdgePartitionBuilder(size) + for (e <- iterator) { + builder.add(e.dstId, e.srcId, e.attr) + } + builder.toEdgePartition + } /** * Construct a new edge partition by applying the function f to all @@ -46,7 +53,7 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) newData(i) = f(edge) i += 1 } - new EdgePartition(srcIds, dstIds, newData) + new EdgePartition(srcIds, dstIds, newData, index) } /** @@ -54,17 +61,8 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) * * @param f an external state mutating user defined function. */ - def foreach(f: Edge[ED] => Unit) { - val edge = new Edge[ED] - val size = data.size - var i = 0 - while (i < size) { - edge.srcId = srcIds(i) - edge.dstId = dstIds(i) - edge.attr = data(i) - f(edge) - i += 1 - } + def foreach(f: Edge[ED] => Unit) { + iterator.foreach(f) } /** @@ -75,21 +73,29 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) * @return a new edge partition without duplicate edges */ def groupEdges(merge: (ED, ED) => ED): EdgePartition[ED] = { - // Aggregate all matching edges in a hashmap - val agg = new OpenHashMap[(Vid,Vid), ED] - foreach { e => agg.setMerge((e.srcId, e.dstId), e.attr, merge) } - // Populate new srcId, dstId, and data, arrays - val newSrcIds = new Array[Vid](agg.size) - val newDstIds = new Array[Vid](agg.size) - val newData = new Array[ED](agg.size) + val builder = new EdgePartitionBuilder[ED] + var firstIter: Boolean = true + var currSrcId: Vid = nullValue[Vid] + var currDstId: Vid = nullValue[Vid] + var currAttr: ED = nullValue[ED] var i = 0 - agg.foreach { kv => - newSrcIds(i) = kv._1._1 - newDstIds(i) = kv._1._2 - newData(i) = kv._2 + while (i < size) { + if (i > 0 && currSrcId == srcIds(i) && currDstId == dstIds(i)) { + currAttr = merge(currAttr, data(i)) + } else { + if (i > 0) { + builder.add(currSrcId, currDstId, currAttr) + } + currSrcId = srcIds(i) + currDstId = dstIds(i) + currAttr = data(i) + } i += 1 } - new EdgePartition(newSrcIds, newDstIds, newData) + if (size > 0) { + builder.add(currSrcId, currDstId, currAttr) + } + builder.toEdgePartition } /** @@ -99,6 +105,9 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) */ def size: Int = srcIds.size + /** The number of unique source vertices in the partition. */ + def indexSize: Int = index.size + /** * Get an iterator over the edges in this partition. * @@ -118,4 +127,34 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) edge } } + + /** + * Get an iterator over the edges in this partition whose source vertex ids match srcIdPred. The + * iterator is generated using an index scan, so it is efficient at skipping edges that don't + * match srcIdPred. + */ + def indexIterator(srcIdPred: Vid => Boolean): Iterator[Edge[ED]] = + index.iterator.filter(kv => srcIdPred(kv._1)).flatMap(Function.tupled(clusterIterator)) + + /** + * Get an iterator over the cluster of edges in this partition with source vertex id `srcId`. The + * cluster must start at position `index`. + */ + private def clusterIterator(srcId: Vid, index: Int) = new Iterator[Edge[ED]] { + private[this] val edge = new Edge[ED] + private[this] var pos = index + + override def hasNext: Boolean = { + pos >= 0 && pos < EdgePartition.this.size && srcIds(pos) == srcId + } + + override def next(): Edge[ED] = { + assert(srcIds(pos) == srcId) + edge.srcId = srcIds(pos) + edge.dstId = dstIds(pos) + edge.attr = data(pos) + pos += 1 + edge + } + } } diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartitionBuilder.scala b/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartitionBuilder.scala index 76c11a364cfda..38762733692cc 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartitionBuilder.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartitionBuilder.scala @@ -1,24 +1,45 @@ package org.apache.spark.graph.impl +import scala.util.Sorting + import org.apache.spark.graph._ -import org.apache.spark.util.collection.PrimitiveVector +import org.apache.spark.util.collection.{PrimitiveKeyOpenHashMap, PrimitiveVector} //private[graph] -class EdgePartitionBuilder[@specialized(Long, Int, Double) ED: ClassManifest] { +class EdgePartitionBuilder[@specialized(Long, Int, Double) ED: ClassManifest](size: Int = 64) { - val srcIds = new PrimitiveVector[Vid] - val dstIds = new PrimitiveVector[Vid] - var dataBuilder = new PrimitiveVector[ED] + var edges = new PrimitiveVector[Edge[ED]](size) /** Add a new edge to the partition. */ def add(src: Vid, dst: Vid, d: ED) { - srcIds += src - dstIds += dst - dataBuilder += d + edges += Edge(src, dst, d) } def toEdgePartition: EdgePartition[ED] = { - new EdgePartition(srcIds.trim().array, dstIds.trim().array, dataBuilder.trim().array) + val edgeArray = edges.trim().array + Sorting.quickSort(edgeArray)(Edge.lexicographicOrdering) + val srcIds = new Array[Vid](edgeArray.size) + val dstIds = new Array[Vid](edgeArray.size) + val data = new Array[ED](edgeArray.size) + val index = new PrimitiveKeyOpenHashMap[Vid, Int] + // Copy edges into columnar structures, tracking the beginnings of source vertex id clusters and + // adding them to the index + if (edgeArray.length > 0) { + index.update(srcIds(0), 0) + var currSrcId: Vid = srcIds(0) + var i = 0 + while (i < edgeArray.size) { + srcIds(i) = edgeArray(i).srcId + dstIds(i) = edgeArray(i).dstId + data(i) = edgeArray(i).attr + if (edgeArray(i).srcId != currSrcId) { + currSrcId = edgeArray(i).srcId + index.update(currSrcId, i) + } + i += 1 + } + } + new EdgePartition(srcIds, dstIds, data, index) } } diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index 08bef821506a3..0adc350187323 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -245,37 +245,44 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( // Map and combine. val preAgg = edges.zipEdgePartitions(vs) { (edgePartition, vTableReplicatedIter) => - val (_, vertexPartition) = vTableReplicatedIter.next() + val (_, vPart) = vTableReplicatedIter.next() + + // Choose scan method + val activeFraction = vPart.numActives.getOrElse(0) / edgePartition.indexSize.toFloat + val edgeIter = activeDirectionOpt match { + case Some(EdgeDirection.Both) => + if (activeFraction < 0.8) { + edgePartition.indexIterator(srcVid => vPart.isActive(srcVid)) + .filter(e => vPart.isActive(e.dstId)) + } else { + edgePartition.iterator.filter(e => vPart.isActive(e.srcId) && vPart.isActive(e.dstId)) + } + case Some(EdgeDirection.Out) => + if (activeFraction < 0.8) { + edgePartition.indexIterator(srcVid => vPart.isActive(srcVid)) + } else { + edgePartition.iterator.filter(e => vPart.isActive(e.srcId)) + } + case Some(EdgeDirection.In) => + edgePartition.iterator.filter(e => vPart.isActive(e.dstId)) + case None => + edgePartition.iterator + } - // Iterate over the partition + // Scan edges and run the map function val et = new EdgeTriplet[VD, ED] - val filteredEdges = edgePartition.iterator.flatMap { e => - // Ensure the edge is adjacent to a vertex in activeSet if necessary - val adjacent = activeDirectionOpt match { - case Some(EdgeDirection.In) => - vertexPartition.isActive(e.dstId) - case Some(EdgeDirection.Out) => - vertexPartition.isActive(e.srcId) - case Some(EdgeDirection.Both) => - vertexPartition.isActive(e.srcId) && vertexPartition.isActive(e.dstId) - case None => - true + val mapOutputs = edgeIter.flatMap { e => + et.set(e) + if (mapUsesSrcAttr) { + et.srcAttr = vPart(e.srcId) } - if (adjacent) { - et.set(e) - if (mapUsesSrcAttr) { - et.srcAttr = vertexPartition(e.srcId) - } - if (mapUsesDstAttr) { - et.dstAttr = vertexPartition(e.dstId) - } - mapFunc(et) - } else { - Iterator.empty + if (mapUsesDstAttr) { + et.dstAttr = vPart(e.dstId) } + mapFunc(et) } // Note: This doesn't allow users to send messages to arbitrary vertices. - vertexPartition.aggregateUsingIndex(filteredEdges, reduceFunc).iterator + vPart.aggregateUsingIndex(mapOutputs, reduceFunc).iterator } // do the final reduction reusing the index map diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala b/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala index 161c98f1588d4..b9b2a4705bbe9 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala @@ -115,7 +115,7 @@ class VTableReplicated[VD: ClassManifest]( for (i <- 0 until block.vids.size) { val vid = block.vids(i) val attr = block.attrs(i) - val ind = vidToIndex.getPos(vid) & OpenHashSet.POSITION_MASK + val ind = vidToIndex.getPos(vid) vertexArray(ind) = attr } } diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala index 1c589c9b72a4c..ccbc83c512fc6 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala @@ -54,6 +54,9 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassManifest]( activeSet.get.contains(vid) } + /** The number of active vertices, if any exist. */ + def numActives: Option[Int] = activeSet.map(_.size) + /** * Pass each vertex attribute along with the vertex id through a map * function and retain the original RDD's partitioning and index. diff --git a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala index 68a171b12f211..a85a31f79d89e 100644 --- a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala @@ -1,9 +1,12 @@ package org.apache.spark.graph +import scala.util.Random + import org.scalatest.FunSuite import org.apache.spark.SparkContext import org.apache.spark.graph.LocalSparkContext._ +import org.apache.spark.graph.impl.EdgePartitionBuilder import org.apache.spark.rdd._ class GraphSuite extends FunSuite with LocalSparkContext { @@ -59,6 +62,13 @@ class GraphSuite extends FunSuite with LocalSparkContext { // mapVertices changing type val mappedVAttrs2 = reverseStar.mapVertices((vid, attr) => attr.length) assert(mappedVAttrs2.vertices.collect.toSet === (0 to n).map(x => (x: Vid, 1)).toSet) + // groupEdges + val doubleStar = Graph.fromEdgeTuples( + sc.parallelize((1 to n).flatMap(x => List((0: Vid, x: Vid), (0: Vid, x: Vid))), 1), "v") + val star2 = doubleStar.groupEdges { (a, b) => a} + assert(star2.edges.collect.toArray.sorted(Edge.lexicographicOrdering[Int]) === + star.edges.collect.toArray.sorted(Edge.lexicographicOrdering[Int])) + assert(star2.vertices.collect.toSet === star.vertices.collect.toSet) } } @@ -206,4 +216,19 @@ class GraphSuite extends FunSuite with LocalSparkContext { assert(subgraph.edges.map(_.copy()).collect().toSet === (2 to n by 2).map(x => Edge(0, x, 1)).toSet) } } + + test("EdgePartition.sort") { + val edgesFrom0 = List(Edge(0, 1, 0)) + val edgesFrom1 = List(Edge(1, 0, 0), Edge(1, 2, 0)) + val sortedEdges = edgesFrom0 ++ edgesFrom1 + val builder = new EdgePartitionBuilder[Int] + for (e <- Random.shuffle(sortedEdges)) { + builder.add(e.srcId, e.dstId, e.attr) + } + + val edgePartition = builder.toEdgePartition + assert(edgePartition.iterator.map(_.copy()).toList === sortedEdges) + assert(edgePartition.indexIterator(_ == 0).map(_.copy()).toList === edgesFrom0) + assert(edgePartition.indexIterator(_ == 1).map(_.copy()).toList === edgesFrom1) + } } From fb1d1e45f1c0e44170682ca06884a6a17601bf12 Mon Sep 17 00:00:00 2001 From: Wang Jianping J Date: Tue, 17 Dec 2013 19:17:40 +0800 Subject: [PATCH 306/531] Update Svdpp.scala --- .../apache/spark/graph/algorithms/Svdpp.scala | 56 ++++++++++++------- 1 file changed, 36 insertions(+), 20 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala b/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala index a3110667bef0c..28ccaa89a92f0 100644 --- a/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala +++ b/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala @@ -1,6 +1,7 @@ package org.apache.spark.graph.algorithms import org.apache.spark._ +import org.apache.spark.rdd._ import org.apache.spark.graph._ import scala.util.Random import org.apache.commons.math.linear._ @@ -21,10 +22,10 @@ class Msg ( object Svdpp { // implement SVD++ based on http://public.research.att.com/~volinsky/netflix/kdd08koren.pdf - def run(sc: SparkContext, input: String): Graph[VD,Double] = { + def run(edges: RDD[Edge[Double]]): Graph[VD,Double] = { // defalut parameters - val rank = 4 - val maxIters = 15 + val rank = 10 + val maxIters = 20 val minVal = 0.0 val maxVal = 5.0 val gamma1 = 0.007 @@ -33,13 +34,13 @@ object Svdpp { val gamma7 = 0.015 def defaultF(rank: Int) = { - val p = new ArrayRealVector(rank) - val w = new ArrayRealVector(rank) + val v1 = new ArrayRealVector(rank) + val v2 = new ArrayRealVector(rank) for (i <- 0 until rank) { - p.setEntry(i, Random.nextDouble) - w.setEntry(i, Random.nextDouble) + v1.setEntry(i, Random.nextDouble) + v2.setEntry(i, Random.nextDouble) } - var vd = new VD(p, w, 0.0, 0.0) + var vd = new VD(v1, v2, 0.0, 0.0) vd } @@ -58,12 +59,6 @@ object Svdpp { } vd } - - // read textfile - val edges = sc.textFile(input).map { line => - val fields = line.split(",") - Edge(fields(0).toLong * 2, fields(1).toLong * 2 + 1, fields(2).toDouble) - } // calculate global rating mean val (rs, rc) = edges.map(e => (e.attr, 1L)).reduce((a, b) => (a._1 + b._1, a._2 + b._2)) @@ -101,12 +96,9 @@ object Svdpp { val itmBias = 0.0 val usrBias = 0.0 var pred = u + usr.bias + itm.bias + q.dotProduct(usr.v2) - if (pred < minVal) { - pred = minVal - } - if (pred > maxVal) { - pred = maxVal - } + println(pred) + pred = math.max(pred, minVal) + pred = math.min(pred, maxVal) val err = et.attr - pred val y = (q.mapMultiply(err*usr.norm)).subtract((usr.v2).mapMultiply(gamma7)) val newP = (q.mapMultiply(err)).subtract(p.mapMultiply(gamma7)) // for each connected item q @@ -138,6 +130,30 @@ object Svdpp { val t2: VertexRDD[Msg] = g.mapReduceTriplets(mapF2, reduceF2) g.outerJoinVertices(t2) {updateF2} } + + // calculate error on training set + def mapF3(et: EdgeTriplet[VD, Double]): Iterator[(Vid, Double)] = { + assert(et.srcAttr != null && et.dstAttr != null) + val usr = et.srcAttr + val itm = et.dstAttr + var p = usr.v1 + var q = itm.v1 + val itmBias = 0.0 + val usrBias = 0.0 + var pred = u + usr.bias + itm.bias + q.dotProduct(usr.v2) + pred = math.max(pred, minVal) + pred = math.min(pred, maxVal) + val err = (et.attr - pred)*(et.attr - pred) + Iterator((et.dstId, err)) + } + def updateF3(vid: Vid, vd: VD, msg: Option[Double]) = { + if (msg.isDefined && vid % 2 == 1) { // item sum up the errors + vd.norm = msg.get + } + vd + } + val t3: VertexRDD[Double] = g.mapReduceTriplets(mapF3, _ + _) + g.outerJoinVertices(t3) {updateF3} g } } From aee71156f324950f6e545c572d4fd6748353679a Mon Sep 17 00:00:00 2001 From: Wang Jianping J Date: Tue, 17 Dec 2013 19:20:25 +0800 Subject: [PATCH 307/531] Test SVD++ --- .../apache/spark/graph/AnalyticsSuite.scala | 104 +++++++++--------- 1 file changed, 49 insertions(+), 55 deletions(-) diff --git a/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala b/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala index 2e6b57a8ec267..fd70306803a5f 100644 --- a/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala @@ -51,38 +51,35 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { System.setProperty("spark.serializer", "org.apache.spark.serializer.KryoSerializer") System.setProperty("spark.kryo.registrator", "org.apache.spark.graph.GraphKryoRegistrator") - def compareRanks(a: VertexRDD[Double], b: VertexRDD[Double]): Double = { - a.leftJoin(b) { case (id, a, bOpt) => (a - bOpt.getOrElse(0.0)) * (a - bOpt.getOrElse(0.0)) } - .map { case (id, error) => error }.sum - } test("Star PageRank") { withSpark(new SparkContext("local", "test")) { sc => val nVertices = 100 val starGraph = GraphGenerators.starGraph(sc, nVertices).cache() val resetProb = 0.15 - val errorTol = 1.0e-5 - - val staticRanks1 = PageRank.run(starGraph, numIter = 1, resetProb).vertices.cache() - val staticRanks2 = PageRank.run(starGraph, numIter = 2, resetProb).vertices.cache() + val prGraph1 = PageRank.run(starGraph, 1, resetProb) + val prGraph2 = PageRank.run(starGraph, 2, resetProb) - // Static PageRank should only take 2 iterations to converge - val notMatching = staticRanks1.zipJoin(staticRanks2) { (vid, pr1, pr2) => - if (pr1 != pr2) 1 else 0 + val notMatching = prGraph1.vertices.zipJoin(prGraph2.vertices) { (vid, pr1, pr2) => + if (pr1 != pr2) { 1 } else { 0 } }.map { case (vid, test) => test }.sum assert(notMatching === 0) - - val staticErrors = staticRanks2.map { case (vid, pr) => + //prGraph2.vertices.foreach(println(_)) + val errors = prGraph2.vertices.map { case (vid, pr) => val correct = (vid > 0 && pr == resetProb) || - (vid == 0 && math.abs(pr - (resetProb + (1.0 - resetProb) * (resetProb * (nVertices - 1)) )) < 1.0E-5) - if (!correct) 1 else 0 + (vid == 0 && math.abs(pr - (resetProb + (1.0 - resetProb) * (resetProb * (nVertices - 1)) )) < 1.0E-5) + if ( !correct ) { 1 } else { 0 } } - assert(staticErrors.sum === 0) + assert(errors.sum === 0) - val dynamicRanks = PageRank.runUntillConvergence(starGraph, 0, resetProb).vertices.cache() - val standaloneRanks = PageRank.runStandalone(starGraph, 0, resetProb).cache() - assert(compareRanks(staticRanks2, dynamicRanks) < errorTol) - assert(compareRanks(staticRanks2, standaloneRanks) < errorTol) + val prGraph3 = PageRank.runUntillConvergence(starGraph, 0, resetProb) + val errors2 = prGraph2.vertices.leftJoin(prGraph3.vertices){ (vid, pr1, pr2Opt) => + pr2Opt match { + case Some(pr2) if(pr1 == pr2) => 0 + case _ => 1 + } + }.map { case (vid, test) => test }.sum + assert(errors2 === 0) } } // end of test Star PageRank @@ -90,46 +87,27 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { test("Grid PageRank") { withSpark(new SparkContext("local", "test")) { sc => - val rows = 10 - val cols = 10 + val gridGraph = GraphGenerators.gridGraph(sc, 10, 10).cache() val resetProb = 0.15 - val tol = 0.0001 - val numIter = 50 - val errorTol = 1.0e-5 - val gridGraph = GraphGenerators.gridGraph(sc, rows, cols).cache() - - val staticRanks = PageRank.run(gridGraph, numIter, resetProb).vertices.cache() - val dynamicRanks = PageRank.runUntillConvergence(gridGraph, tol, resetProb).vertices.cache() - val standaloneRanks = PageRank.runStandalone(gridGraph, tol, resetProb).cache() - val referenceRanks = VertexRDD(sc.parallelize(GridPageRank(rows, cols, numIter, resetProb))) - - assert(compareRanks(staticRanks, referenceRanks) < errorTol) - assert(compareRanks(dynamicRanks, referenceRanks) < errorTol) - assert(compareRanks(standaloneRanks, referenceRanks) < errorTol) + val prGraph1 = PageRank.run(gridGraph, 50, resetProb).cache() + val prGraph2 = PageRank.runUntillConvergence(gridGraph, 0.0001, resetProb).cache() + val error = prGraph1.vertices.zipJoin(prGraph2.vertices) { case (id, a, b) => (a - b) * (a - b) } + .map { case (id, error) => error }.sum + //prGraph1.vertices.zipJoin(prGraph2.vertices) { (id, a, b) => (a, b, a-b) }.foreach(println(_)) + println(error) + assert(error < 1.0e-5) + val pr3: RDD[(Vid, Double)] = sc.parallelize(GridPageRank(10,10, 50, resetProb)) + val error2 = prGraph1.vertices.leftJoin(pr3) { (id, a, bOpt) => + val b: Double = bOpt.get + (a - b) * (a - b) + }.map { case (id, error) => error }.sum + //prGraph1.vertices.leftJoin(pr3) { (id, a, b) => (a, b) }.foreach( println(_) ) + println(error2) + assert(error2 < 1.0e-5) } } // end of Grid PageRank - test("Chain PageRank") { - withSpark(new SparkContext("local", "test")) { sc => - val chain1 = (0 until 9).map(x => (x, x+1) ) - val rawEdges = sc.parallelize(chain1, 1).map { case (s,d) => (s.toLong, d.toLong) } - val chain = Graph.fromEdgeTuples(rawEdges, 1.0).cache() - val resetProb = 0.15 - val tol = 0.0001 - val numIter = 10 - val errorTol = 1.0e-5 - - val staticRanks = PageRank.run(chain, numIter, resetProb).vertices.cache() - val dynamicRanks = PageRank.runUntillConvergence(chain, tol, resetProb).vertices.cache() - val standaloneRanks = PageRank.runStandalone(chain, tol, resetProb).cache() - - assert(compareRanks(staticRanks, dynamicRanks) < errorTol) - assert(compareRanks(dynamicRanks, standaloneRanks) < errorTol) - } - } - - test("Grid Connected Components") { withSpark(new SparkContext("local", "test")) { sc => val gridGraph = GraphGenerators.gridGraph(sc, 10, 10).cache() @@ -189,6 +167,7 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { } } val ccMap = vertices.toMap + println(ccMap) for ( id <- 0 until 20 ) { if (id < 10) { assert(ccMap(id) === 0) @@ -257,4 +236,19 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { verts.collect.foreach { case (vid, count) => assert(count === 1) } } } + + test("Test SVD++ with mean square error on training set") { + withSpark(new SparkContext("local", "test")) { sc => + val SvdppErr = 0.01 + val edges = sc.textFile("mllib/data/als/test.data").map { line => + val fields = line.split(",") + Edge(fields(0).toLong * 2, fields(1).toLong * 2 + 1, fields(2).toDouble) + } + val graph = Svdpp.run(edges) + val err = graph.vertices.collect.map{ case (vid, vd) => + if (vid % 2 == 1) { vd.norm } else { 0.0 } + }.reduce(_ + _) / graph.triplets.collect.size + assert(err < SvdppErr) + } + } } // end of AnalyticsSuite From 9d2351f501b47d02399391475a57ee19ac2a28e1 Mon Sep 17 00:00:00 2001 From: Wang Jianping J Date: Tue, 17 Dec 2013 19:30:07 +0800 Subject: [PATCH 308/531] Update Svdpp.scala --- .../main/scala/org/apache/spark/graph/algorithms/Svdpp.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala b/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala index 28ccaa89a92f0..a0f025d708704 100644 --- a/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala +++ b/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala @@ -65,7 +65,7 @@ object Svdpp { val u = rs / rc // global rating mean // make graph - var g = Graph.fromEdges(edges, defaultF(rank), RandomVertexCut).cache() + var g = Graph.fromEdges(edges, defaultF(rank)).cache() // calculate initial norm and bias val t0 = g.mapReduceTriplets(mapF0, reduceF0) @@ -96,7 +96,6 @@ object Svdpp { val itmBias = 0.0 val usrBias = 0.0 var pred = u + usr.bias + itm.bias + q.dotProduct(usr.v2) - println(pred) pred = math.max(pred, minVal) pred = math.min(pred, maxVal) val err = et.attr - pred From 772b192910207b5a8cbfaae9573c30efa3add7c3 Mon Sep 17 00:00:00 2001 From: Wang Jianping J Date: Tue, 17 Dec 2013 19:41:04 +0800 Subject: [PATCH 309/531] Update AnalyticsSuite.scala --- .../apache/spark/graph/AnalyticsSuite.scala | 91 ++++++++++++------- 1 file changed, 56 insertions(+), 35 deletions(-) diff --git a/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala b/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala index fd70306803a5f..05ebe2b84d772 100644 --- a/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala @@ -51,35 +51,38 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { System.setProperty("spark.serializer", "org.apache.spark.serializer.KryoSerializer") System.setProperty("spark.kryo.registrator", "org.apache.spark.graph.GraphKryoRegistrator") + def compareRanks(a: VertexRDD[Double], b: VertexRDD[Double]): Double = { + a.leftJoin(b) { case (id, a, bOpt) => (a - bOpt.getOrElse(0.0)) * (a - bOpt.getOrElse(0.0)) } + .map { case (id, error) => error }.sum + } test("Star PageRank") { withSpark(new SparkContext("local", "test")) { sc => val nVertices = 100 val starGraph = GraphGenerators.starGraph(sc, nVertices).cache() val resetProb = 0.15 - val prGraph1 = PageRank.run(starGraph, 1, resetProb) - val prGraph2 = PageRank.run(starGraph, 2, resetProb) + val errorTol = 1.0e-5 + + val staticRanks1 = PageRank.run(starGraph, numIter = 1, resetProb).vertices.cache() + val staticRanks2 = PageRank.run(starGraph, numIter = 2, resetProb).vertices.cache() - val notMatching = prGraph1.vertices.zipJoin(prGraph2.vertices) { (vid, pr1, pr2) => - if (pr1 != pr2) { 1 } else { 0 } + // Static PageRank should only take 2 iterations to converge + val notMatching = staticRanks1.zipJoin(staticRanks2) { (vid, pr1, pr2) => + if (pr1 != pr2) 1 else 0 }.map { case (vid, test) => test }.sum assert(notMatching === 0) - //prGraph2.vertices.foreach(println(_)) - val errors = prGraph2.vertices.map { case (vid, pr) => + + val staticErrors = staticRanks2.map { case (vid, pr) => val correct = (vid > 0 && pr == resetProb) || - (vid == 0 && math.abs(pr - (resetProb + (1.0 - resetProb) * (resetProb * (nVertices - 1)) )) < 1.0E-5) - if ( !correct ) { 1 } else { 0 } + (vid == 0 && math.abs(pr - (resetProb + (1.0 - resetProb) * (resetProb * (nVertices - 1)) )) < 1.0E-5) + if (!correct) 1 else 0 } - assert(errors.sum === 0) + assert(staticErrors.sum === 0) - val prGraph3 = PageRank.runUntillConvergence(starGraph, 0, resetProb) - val errors2 = prGraph2.vertices.leftJoin(prGraph3.vertices){ (vid, pr1, pr2Opt) => - pr2Opt match { - case Some(pr2) if(pr1 == pr2) => 0 - case _ => 1 - } - }.map { case (vid, test) => test }.sum - assert(errors2 === 0) + val dynamicRanks = PageRank.runUntillConvergence(starGraph, 0, resetProb).vertices.cache() + val standaloneRanks = PageRank.runStandalone(starGraph, 0, resetProb).cache() + assert(compareRanks(staticRanks2, dynamicRanks) < errorTol) + assert(compareRanks(staticRanks2, standaloneRanks) < errorTol) } } // end of test Star PageRank @@ -87,27 +90,46 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { test("Grid PageRank") { withSpark(new SparkContext("local", "test")) { sc => - val gridGraph = GraphGenerators.gridGraph(sc, 10, 10).cache() + val rows = 10 + val cols = 10 val resetProb = 0.15 - val prGraph1 = PageRank.run(gridGraph, 50, resetProb).cache() - val prGraph2 = PageRank.runUntillConvergence(gridGraph, 0.0001, resetProb).cache() - val error = prGraph1.vertices.zipJoin(prGraph2.vertices) { case (id, a, b) => (a - b) * (a - b) } - .map { case (id, error) => error }.sum - //prGraph1.vertices.zipJoin(prGraph2.vertices) { (id, a, b) => (a, b, a-b) }.foreach(println(_)) - println(error) - assert(error < 1.0e-5) - val pr3: RDD[(Vid, Double)] = sc.parallelize(GridPageRank(10,10, 50, resetProb)) - val error2 = prGraph1.vertices.leftJoin(pr3) { (id, a, bOpt) => - val b: Double = bOpt.get - (a - b) * (a - b) - }.map { case (id, error) => error }.sum - //prGraph1.vertices.leftJoin(pr3) { (id, a, b) => (a, b) }.foreach( println(_) ) - println(error2) - assert(error2 < 1.0e-5) + val tol = 0.0001 + val numIter = 50 + val errorTol = 1.0e-5 + val gridGraph = GraphGenerators.gridGraph(sc, rows, cols).cache() + + val staticRanks = PageRank.run(gridGraph, numIter, resetProb).vertices.cache() + val dynamicRanks = PageRank.runUntillConvergence(gridGraph, tol, resetProb).vertices.cache() + val standaloneRanks = PageRank.runStandalone(gridGraph, tol, resetProb).cache() + val referenceRanks = VertexRDD(sc.parallelize(GridPageRank(rows, cols, numIter, resetProb))) + + assert(compareRanks(staticRanks, referenceRanks) < errorTol) + assert(compareRanks(dynamicRanks, referenceRanks) < errorTol) + assert(compareRanks(standaloneRanks, referenceRanks) < errorTol) } } // end of Grid PageRank + test("Chain PageRank") { + withSpark(new SparkContext("local", "test")) { sc => + val chain1 = (0 until 9).map(x => (x, x+1) ) + val rawEdges = sc.parallelize(chain1, 1).map { case (s,d) => (s.toLong, d.toLong) } + val chain = Graph.fromEdgeTuples(rawEdges, 1.0).cache() + val resetProb = 0.15 + val tol = 0.0001 + val numIter = 10 + val errorTol = 1.0e-5 + + val staticRanks = PageRank.run(chain, numIter, resetProb).vertices.cache() + val dynamicRanks = PageRank.runUntillConvergence(chain, tol, resetProb).vertices.cache() + val standaloneRanks = PageRank.runStandalone(chain, tol, resetProb).cache() + + assert(compareRanks(staticRanks, dynamicRanks) < errorTol) + assert(compareRanks(dynamicRanks, standaloneRanks) < errorTol) + } + } + + test("Grid Connected Components") { withSpark(new SparkContext("local", "test")) { sc => val gridGraph = GraphGenerators.gridGraph(sc, 10, 10).cache() @@ -167,7 +189,6 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { } } val ccMap = vertices.toMap - println(ccMap) for ( id <- 0 until 20 ) { if (id < 10) { assert(ccMap(id) === 0) @@ -230,7 +251,7 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { withSpark(new SparkContext("local", "test")) { sc => val rawEdges = sc.parallelize(Array(0L -> 1L, 1L -> 2L, 2L -> 0L) ++ Array(0L -> 1L, 1L -> 2L, 2L -> 0L), 2) - val graph = Graph.fromEdgeTuples(rawEdges, true).cache() + val graph = Graph.fromEdgeTuples(rawEdges, true, uniqueEdges = Some(RandomVertexCut)).cache() val triangleCount = TriangleCount.run(graph) val verts = triangleCount.vertices verts.collect.foreach { case (vid, count) => assert(count === 1) } From e94fe39d0f89ba102b1ebadf8becf99c051eb58d Mon Sep 17 00:00:00 2001 From: Wang Jianping J Date: Wed, 18 Dec 2013 06:39:28 +0800 Subject: [PATCH 310/531] Update Svdpp.scala --- .../apache/spark/graph/algorithms/Svdpp.scala | 27 ++++++++++--------- 1 file changed, 14 insertions(+), 13 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala b/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala index a0f025d708704..ef266bb551ddd 100644 --- a/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala +++ b/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala @@ -6,14 +6,14 @@ import org.apache.spark.graph._ import scala.util.Random import org.apache.commons.math.linear._ -class VD ( +class VT ( // vertex type var v1: RealVector, // v1: p for user node, q for item node var v2: RealVector, // v2: pu + |N(u)|^(-0.5)*sum(y) for user node, y for item node var bias: Double, var norm: Double // only for user node ) extends Serializable -class Msg ( +class Msg ( // message var v1: RealVector, var v2: RealVector, var bias: Double @@ -22,7 +22,7 @@ class Msg ( object Svdpp { // implement SVD++ based on http://public.research.att.com/~volinsky/netflix/kdd08koren.pdf - def run(edges: RDD[Edge[Double]]): Graph[VD,Double] = { + def run(edges: RDD[Edge[Double]]): Graph[VT, Double] = { // defalut parameters val rank = 10 val maxIters = 20 @@ -40,19 +40,19 @@ object Svdpp { v1.setEntry(i, Random.nextDouble) v2.setEntry(i, Random.nextDouble) } - var vd = new VD(v1, v2, 0.0, 0.0) + var vd = new VT(v1, v2, 0.0, 0.0) vd } // calculate initial norm and bias - def mapF0(et: EdgeTriplet[VD, Double]): Iterator[(Vid, (Long, Double))] = { + def mapF0(et: EdgeTriplet[VT, Double]): Iterator[(Vid, (Long, Double))] = { assert(et.srcAttr != null && et.dstAttr != null) Iterator((et.srcId, (1L, et.attr)), (et.dstId, (1L, et.attr))) } def reduceF0(g1: (Long, Double), g2: (Long, Double)) = { (g1._1 + g2._1, g1._2 + g2._2) } - def updateF0(vid: Vid, vd: VD, msg: Option[(Long, Double)]) = { + def updateF0(vid: Vid, vd: VT, msg: Option[(Long, Double)]) = { if (msg.isDefined) { vd.bias = msg.get._2 / msg.get._1 vd.norm = 1.0 / scala.math.sqrt(msg.get._1) @@ -65,21 +65,21 @@ object Svdpp { val u = rs / rc // global rating mean // make graph - var g = Graph.fromEdges(edges, defaultF(rank)).cache() + var g = Graph.fromEdges(edges, defaultF(rank), RandomVertexCut).cache() // calculate initial norm and bias val t0 = g.mapReduceTriplets(mapF0, reduceF0) g.outerJoinVertices(t0) {updateF0} // phase 1 - def mapF1(et: EdgeTriplet[VD, Double]): Iterator[(Vid, RealVector)] = { + def mapF1(et: EdgeTriplet[VT, Double]): Iterator[(Vid, RealVector)] = { assert(et.srcAttr != null && et.dstAttr != null) Iterator((et.srcId, et.dstAttr.v2)) // sum up y of connected item nodes } def reduceF1(g1: RealVector, g2: RealVector) = { g1.add(g2) } - def updateF1(vid: Vid, vd: VD, msg: Option[RealVector]) = { + def updateF1(vid: Vid, vd: VT, msg: Option[RealVector]) = { if (msg.isDefined) { vd.v2 = vd.v1.add(msg.get.mapMultiply(vd.norm)) // pu + |N(u)|^(-0.5)*sum(y) } @@ -87,7 +87,7 @@ object Svdpp { } // phase 2 - def mapF2(et: EdgeTriplet[VD, Double]): Iterator[(Vid, Msg)] = { + def mapF2(et: EdgeTriplet[VT, Double]): Iterator[(Vid, Msg)] = { assert(et.srcAttr != null && et.dstAttr != null) val usr = et.srcAttr val itm = et.dstAttr @@ -96,6 +96,7 @@ object Svdpp { val itmBias = 0.0 val usrBias = 0.0 var pred = u + usr.bias + itm.bias + q.dotProduct(usr.v2) + println(pred) pred = math.max(pred, minVal) pred = math.min(pred, maxVal) val err = et.attr - pred @@ -110,7 +111,7 @@ object Svdpp { g1.bias += g2.bias g1 } - def updateF2(vid: Vid, vd: VD, msg: Option[Msg]) = { + def updateF2(vid: Vid, vd: VT, msg: Option[Msg]) = { if (msg.isDefined) { vd.v1 = vd.v1.add(msg.get.v1.mapMultiply(gamma2)) if (vid % 2 == 1) { // item node update y @@ -131,7 +132,7 @@ object Svdpp { } // calculate error on training set - def mapF3(et: EdgeTriplet[VD, Double]): Iterator[(Vid, Double)] = { + def mapF3(et: EdgeTriplet[VT, Double]): Iterator[(Vid, Double)] = { assert(et.srcAttr != null && et.dstAttr != null) val usr = et.srcAttr val itm = et.dstAttr @@ -145,7 +146,7 @@ object Svdpp { val err = (et.attr - pred)*(et.attr - pred) Iterator((et.dstId, err)) } - def updateF3(vid: Vid, vd: VD, msg: Option[Double]) = { + def updateF3(vid: Vid, vd: VT, msg: Option[Double]) = { if (msg.isDefined && vid % 2 == 1) { // item sum up the errors vd.norm = msg.get } From ba538797dddeb940bac746e690955b2d08896e0f Mon Sep 17 00:00:00 2001 From: Wang Jianping J Date: Wed, 18 Dec 2013 06:42:19 +0800 Subject: [PATCH 311/531] Update Svdpp.scala --- .../main/scala/org/apache/spark/graph/algorithms/Svdpp.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala b/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala index ef266bb551ddd..efe407ac30601 100644 --- a/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala +++ b/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala @@ -65,7 +65,7 @@ object Svdpp { val u = rs / rc // global rating mean // make graph - var g = Graph.fromEdges(edges, defaultF(rank), RandomVertexCut).cache() + var g = Graph.fromEdges(edges, defaultF(rank)).cache() // calculate initial norm and bias val t0 = g.mapReduceTriplets(mapF0, reduceF0) From 06581b6a96713d61a61c4ad8eba34fa1e7ecff48 Mon Sep 17 00:00:00 2001 From: Wang Jianping J Date: Wed, 18 Dec 2013 06:51:37 +0800 Subject: [PATCH 312/531] Update Svdpp.scala --- .../src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala b/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala index efe407ac30601..4ddf0b1fd5ee4 100644 --- a/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala +++ b/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala @@ -96,7 +96,6 @@ object Svdpp { val itmBias = 0.0 val usrBias = 0.0 var pred = u + usr.bias + itm.bias + q.dotProduct(usr.v2) - println(pred) pred = math.max(pred, minVal) pred = math.min(pred, maxVal) val err = et.attr - pred From 0f137e8b75497e61f8d9fec98896cd912f27c3ed Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Mon, 16 Dec 2013 23:52:34 -0800 Subject: [PATCH 313/531] Reimplement Graph.mask using innerJoin --- .../org/apache/spark/graph/EdgeRDD.scala | 23 ++++++++++- .../scala/org/apache/spark/graph/Graph.scala | 2 +- .../spark/graph/impl/EdgePartition.scala | 34 ++++++++++++++++ .../apache/spark/graph/impl/GraphImpl.scala | 39 +++---------------- .../org/apache/spark/graph/GraphSuite.scala | 23 +++++++++-- 5 files changed, 83 insertions(+), 38 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala b/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala index ee368ebb41c4f..63858db2efdd9 100644 --- a/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala @@ -44,7 +44,7 @@ class EdgeRDD[@specialized ED: ClassManifest]( override def cache(): EdgeRDD[ED] = persist() def mapEdgePartitions[ED2: ClassManifest](f: EdgePartition[ED] => EdgePartition[ED2]) - : EdgeRDD[ED2]= { + : EdgeRDD[ED2] = { new EdgeRDD[ED2](partitionsRDD.mapPartitions({ iter => val (pid, ep) = iter.next() Iterator(Tuple2(pid, f(ep))) @@ -60,6 +60,27 @@ class EdgeRDD[@specialized ED: ClassManifest]( } } + def zipEdgePartitions[ED2: ClassManifest, ED3: ClassManifest] + (other: EdgeRDD[ED2]) + (f: (EdgePartition[ED], EdgePartition[ED2]) => EdgePartition[ED3]): EdgeRDD[ED3] = { + new EdgeRDD[ED3](partitionsRDD.zipPartitions(other.partitionsRDD, preservesPartitioning = true) { + (thisIter, otherIter) => + val (pid, thisEPart) = thisIter.next() + val (_, otherEPart) = otherIter.next() + Iterator(Tuple2(pid, f(thisEPart, otherEPart))) + }) + } + + def innerJoin[ED2: ClassManifest, ED3: ClassManifest] + (other: EdgeRDD[ED2]) + (f: (Vid, Vid, ED, ED2) => ED3): EdgeRDD[ED3] = { + val ed2Manifest = classManifest[ED2] + val ed3Manifest = classManifest[ED3] + zipEdgePartitions(other) { (thisEPart, otherEPart) => + thisEPart.innerJoin(otherEPart)(f)(ed2Manifest, ed3Manifest) + } + } + def collectVids(): RDD[Vid] = { partitionsRDD.flatMap { case (_, p) => Array.concat(p.srcIds, p.dstIds) } } diff --git a/graph/src/main/scala/org/apache/spark/graph/Graph.scala b/graph/src/main/scala/org/apache/spark/graph/Graph.scala index e5446509636f7..e8fa8e611c9cb 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Graph.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Graph.scala @@ -48,7 +48,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * along with their vertex data. * */ - val edges: RDD[Edge[ED]] + val edges: EdgeRDD[ED] /** * Get the edges with the vertex data associated with the adjacent diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala b/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala index bfdafcc54248b..e97522feaeaf2 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala @@ -98,6 +98,40 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) builder.toEdgePartition } + /** + * Apply `f` to all edges present in both `this` and `other` and return a new EdgePartition + * containing the resulting edges. + * + * If there are multiple edges with the same src and dst in `this`, `f` will be invoked once for + * each edge, but each time it may be invoked on any corresponding edge in `other`. + * + * If there are multiple edges with the same src and dst in `other`, `f` will only be invoked + * once. + */ + def innerJoin[ED2: ClassManifest, ED3: ClassManifest] + (other: EdgePartition[ED2]) + (f: (Vid, Vid, ED, ED2) => ED3): EdgePartition[ED3] = { + val builder = new EdgePartitionBuilder[ED3] + var i = 0 + var j = 0 + // For i = index of each edge in `this`... + while (i < size && j < other.size) { + val srcId = this.srcIds(i) + val dstId = this.dstIds(i) + // ... forward j to the index of the corresponding edge in `other`, and... + while (j < other.size && other.srcIds(j) < srcId) { j += 1 } + if (j < other.size && other.srcIds(j) == srcId) { + while (j < other.size && other.srcIds(j) == srcId && other.dstIds(j) < dstId) { j += 1 } + if (j < other.size && other.srcIds(j) == srcId && other.dstIds(j) == dstId) { + // ... run `f` on the matching edge + builder.add(srcId, dstId, f(srcId, dstId, this.data(i), other.data(j))) + } + } + i += 1 + } + builder.toEdgePartition + } + /** * The number of edges in this partition * diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index 2fe02718e9817..e7f975253a6c9 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -216,7 +216,12 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( } // end of subgraph override def mask[VD2: ClassManifest, ED2: ClassManifest] ( - other: Graph[VD2, ED2]) : Graph[VD, ED] = GraphImpl.mask(this, other) + other: Graph[VD2, ED2]): Graph[VD, ED] = { + val newVerts = vertices.innerJoin(other.vertices) { (vid, v, w) => v } + val newEdges = edges.innerJoin(other.edges) { (src, dst, v, w) => v } + new GraphImpl(newVerts, newEdges) + + } override def groupEdges(merge: (ED, ED) => ED): Graph[VD, ED] = { ClosureCleaner.clean(merge) @@ -379,38 +384,6 @@ object GraphImpl { new EdgeRDD(edges) } - def mask[VD: ClassManifest, ED: ClassManifest, VD2: ClassManifest, ED2: ClassManifest] ( - thisGraph: Graph[VD, ED], otherGraph: Graph[VD2, ED2]) : Graph[VD, ED] = { - // basically vertices.join(other.vertices) - // written this way to take advantage of fast join in VertexSetRDDs - val newVTable = VertexSetRDD( - thisGraph.vertices.leftJoin(otherGraph.vertices)((vid, v, w) => if (w.isEmpty) None else Some(v)) - .filter{case (vid, opt) => !opt.isEmpty} - .map{case (vid, opt) => (vid, opt.get)} - ) - - // TODO(amatsukawa): safer way to downcast? case matching perhaps? - val thisImpl = thisGraph.asInstanceOf[GraphImpl[VD, ED]] - val otherImpl = otherGraph.asInstanceOf[GraphImpl[VD2, ED2]] - val newETable = thisImpl.eTable.zipPartitions(otherImpl.eTable) { - // extract two edge partitions, keep all edges in in this partition that is - // also in the other partition - (thisIter, otherIter) => - val (_, otherEPart) = otherIter.next() - val otherEdges = otherEPart.iterator.map(e => (e.srcId, e.dstId)).toSet - val (pid, thisEPart) = thisIter.next() - val newEPartBuilder = new EdgePartitionBuilder[ED] - thisEPart.foreach { e => - if (otherEdges.contains((e.srcId, e.dstId))) - newEPartBuilder.add(e.srcId, e.dstId, e.attr) - } - Iterator((pid, newEPartBuilder.toEdgePartition)) - }.partitionBy(thisImpl.eTable.partitioner.get) - - val newVertexPlacement = new VertexPlacement(newETable, newVTable) - new GraphImpl(newVTable, newETable, newVertexPlacement, thisImpl.partitioner) - } - private def fromEdgeRDD[VD: ClassManifest, ED: ClassManifest]( edges: EdgeRDD[ED], defaultVertexAttr: VD): GraphImpl[VD, ED] = { diff --git a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala index fae6eb55250a9..f6bb201a83b32 100644 --- a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala @@ -5,7 +5,9 @@ import scala.util.Random import org.scalatest.FunSuite import org.apache.spark.SparkContext +import org.apache.spark.graph.Graph._ import org.apache.spark.graph.LocalSparkContext._ +import org.apache.spark.graph.impl.EdgePartition import org.apache.spark.graph.impl.EdgePartitionBuilder import org.apache.spark.rdd._ @@ -183,7 +185,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { } } - test("projectGraph") { + test("mask") { withSpark(new SparkContext("local", "test")) { sc => val n = 5 val vertices = sc.parallelize((0 to n).map(x => (x:Vid, x))) @@ -207,7 +209,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { } } - test ("filterGraph") { + test ("filter") { withSpark(new SparkContext("local", "test")) { sc => val n = 5 val vertices = sc.parallelize((0 to n).map(x => (x:Vid, x))) @@ -215,7 +217,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { val graph: Graph[Int, Int] = Graph(vertices, edges) val filteredGraph = graph.filter( graph => { - val degrees: VertexSetRDD[Int] = graph.outDegrees + val degrees: VertexRDD[Int] = graph.outDegrees graph.outerJoinVertices(degrees) {(vid, data, deg) => deg.getOrElse(0)} }, vpred = (vid: Vid, deg:Int) => deg > 0 @@ -278,4 +280,19 @@ class GraphSuite extends FunSuite with LocalSparkContext { assert(edgePartition.indexIterator(_ == 0).map(_.copy()).toList === edgesFrom0) assert(edgePartition.indexIterator(_ == 1).map(_.copy()).toList === edgesFrom1) } + + test("EdgePartition.innerJoin") { + def makeEdgePartition[A: ClassManifest](xs: Iterable[(Int, Int, A)]): EdgePartition[A] = { + val builder = new EdgePartitionBuilder[A] + for ((src, dst, attr) <- xs) { builder.add(src: Vid, dst: Vid, attr) } + builder.toEdgePartition + } + val aList = List((0, 1, 0), (1, 0, 0), (1, 2, 0), (5, 4, 0), (5, 5, 0)) + val bList = List((0, 1, 0), (1, 0, 0), (1, 1, 0), (3, 4, 0), (5, 5, 0)) + val a = makeEdgePartition(aList) + val b = makeEdgePartition(bList) + + assert(a.innerJoin(b) { (src, dst, a, b) => a }.iterator.map(_.copy()).toList === + List(Edge(0, 1, 0), Edge(1, 0, 0), Edge(5, 5, 0))) + } } From b1db1b6c84cfaa0c45cd99386c4500dc6ccf0bf6 Mon Sep 17 00:00:00 2001 From: Akihiro Matsukawa Date: Wed, 18 Dec 2013 18:22:43 -0800 Subject: [PATCH 314/531] scc algorithm --- .../StronglyConnectedComponents.scala | 87 +++++++++++++++++++ 1 file changed, 87 insertions(+) create mode 100644 graph/src/main/scala/org/apache/spark/graph/algorithms/StronglyConnectedComponents.scala diff --git a/graph/src/main/scala/org/apache/spark/graph/algorithms/StronglyConnectedComponents.scala b/graph/src/main/scala/org/apache/spark/graph/algorithms/StronglyConnectedComponents.scala new file mode 100644 index 0000000000000..c324c984d7692 --- /dev/null +++ b/graph/src/main/scala/org/apache/spark/graph/algorithms/StronglyConnectedComponents.scala @@ -0,0 +1,87 @@ +package org.apache.spark.graph.algorithms + +import org.apache.spark.graph._ + +object StronglyConnectedComponents { + + /** + * Compute the strongly connected component (SCC) of each vertex and return an RDD with the vertex + * value containing the lowest vertex id in the SCC containing that vertex. + * + * @tparam VD the vertex attribute type (discarded in the computation) + * @tparam ED the edge attribute type (preserved in the computation) + * + * @param graph the graph for which to compute the SCC + * + * @return a graph with vertex attributes containing the smallest vertex id in each SCC + */ + def run[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], numIter: Int): Graph[Vid, ED] = { + + // the graph we update with final SCC ids, and the graph we return at the end + var sccGraph = graph.mapVertices { case (vid, _) => vid } + // graph we are going to work with in our iterations + var sccWorkGraph = graph.mapVertices { case (vid, _) => (vid, false) } + + var numVertices = sccWorkGraph.numVertices + var iter = 0 + while (sccWorkGraph.numVertices > 0 && iter < numIter) { + iter += 1 + do { + numVertices = sccWorkGraph.numVertices + sccWorkGraph = sccWorkGraph.outerJoinVertices(sccWorkGraph.outDegrees) { + (vid, data, degreeOpt) => if (degreeOpt.isDefined) data else (vid, true) + } + sccWorkGraph = sccWorkGraph.outerJoinVertices(sccWorkGraph.inDegrees) { + (vid, data, degreeOpt) => if (degreeOpt.isDefined) data else (vid, true) + } + + // get all vertices to be removed + val finalVertices = sccWorkGraph.vertices + .filter { case (vid, (scc, isFinal)) => isFinal} + .mapValues { (vid, data) => data._1} + + // write values to sccGraph + sccGraph = sccGraph.outerJoinVertices(finalVertices) { + (vid, scc, opt) => opt.getOrElse(scc) + } + // only keep vertices that are not final + sccWorkGraph = sccWorkGraph.subgraph(vpred = (vid, data) => !data._2) + } while (sccWorkGraph.numVertices < numVertices) + + sccWorkGraph = sccWorkGraph.mapVertices{ case (vid, (color, isFinal)) => (vid, isFinal) } + + // 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 = GraphLab[(Vid, Boolean), ED, Vid](sccWorkGraph, Integer.MAX_VALUE)( + (vid, e) => e.otherVertexAttr(vid)._1, + (vid1, vid2) => math.min(vid1, vid2), + (vid, scc, optScc) => + (math.min(scc._1, optScc.getOrElse(scc._1)), scc._2), + (vid, e) => e.vertexAttr(vid)._1 < e.otherVertexAttr(vid)._1 + ) + + // start at root of SCCs. Traverse values in reverse, notify all my neighbors + // do not propagate if colors do not match! + sccWorkGraph = GraphLab[(Vid, Boolean), ED, Boolean]( + sccWorkGraph, + Integer.MAX_VALUE, + EdgeDirection.Out, + EdgeDirection.In + )( + // vertex is final if it is the root of a color + // or it has the same color as a neighbor that is final + (vid, e) => (vid == e.vertexAttr(vid)._1) || (e.vertexAttr(vid)._1 == e.otherVertexAttr(vid)._1), + (final1, final2) => final1 || final2, + (vid, scc, optFinal) => + (scc._1, scc._2 || optFinal.getOrElse(false)), + // activate neighbor if they are not final, you are, and you have the same color + (vid, e) => e.vertexAttr(vid)._2 && + !e.otherVertexAttr(vid)._2 && (e.vertexAttr(vid)._1 == e.otherVertexAttr(vid)._1), + // start at root of colors + (vid, data) => vid == data._1 + ) + } + sccGraph + } + +} From 989f22b365c8f51fa9c262f29bb07ed608d967a1 Mon Sep 17 00:00:00 2001 From: Akihiro Matsukawa Date: Wed, 18 Dec 2013 18:22:53 -0800 Subject: [PATCH 315/531] added scc tests --- .../apache/spark/graph/AnalyticsSuite.scala | 43 +++++++++++++++++++ 1 file changed, 43 insertions(+) diff --git a/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala b/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala index 05ebe2b84d772..d36339b65b711 100644 --- a/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala @@ -199,6 +199,49 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { } } // end of reverse chain connected components + test("Island Strongly Connected Components") { + withSpark(new SparkContext("local", "test")) { sc => + val vertices = sc.parallelize((1L to 5L).map(x => (x, -1))) + val edges = sc.parallelize(Seq.empty[Edge[Int]]) + val graph = Graph(vertices, edges) + val sccGraph = StronglyConnectedComponents.run(graph, 5) + for ((id, scc) <- sccGraph.vertices.collect) { + assert(id == scc) + } + } + } + + test("Cycle Strongly Connected Components") { + withSpark(new SparkContext("local", "test")) { sc => + val rawEdges = sc.parallelize((0L to 6L).map(x => (x, (x + 1) % 7))) + val graph = Graph.fromEdgeTuples(rawEdges, -1) + val sccGraph = StronglyConnectedComponents.run(graph, 20) + for ((id, scc) <- sccGraph.vertices.collect) { + assert(0L == scc) + } + } + } + + test("2 Cycle Strongly Connected Components") { + withSpark(new SparkContext("local", "test")) { sc => + val edges = + Array(0L -> 1L, 1L -> 2L, 2L -> 0L) ++ + Array(3L -> 4L, 4L -> 5L, 5L -> 3L) ++ + Array(6L -> 0L, 5L -> 7L) + val rawEdges = sc.parallelize(edges) + val graph = Graph.fromEdgeTuples(rawEdges, -1) + val sccGraph = StronglyConnectedComponents.run(graph, 20) + for ((id, scc) <- sccGraph.vertices.collect) { + if (id < 3) + assert(0L == scc) + else if (id < 6) + assert(3L == scc) + else + assert(id == scc) + } + } + } + test("Count a single triangle") { withSpark(new SparkContext("local", "test")) { sc => val rawEdges = sc.parallelize(Array( 0L->1L, 1L->2L, 2L->0L ), 2) From da9f5e3fc093a91e0e91bc9311d5f5d085dbc929 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Thu, 19 Dec 2013 19:49:07 -0800 Subject: [PATCH 316/531] Split GraphSuite; simplify LocalSparkContext --- .../apache/spark/graph/AnalyticsSuite.scala | 39 +- .../apache/spark/graph/GraphOpsSuite.scala | 94 +++++ .../org/apache/spark/graph/GraphSuite.scala | 342 ++++++++---------- .../spark/graph/LocalSparkContext.scala | 44 +-- .../org/apache/spark/graph/PregelSuite.scala | 8 +- .../apache/spark/graph/SerializerSuite.scala | 6 +- .../apache/spark/graph/VertexRDDSuite.scala | 32 ++ .../spark/graph/impl/EdgePartitionSuite.scala | 43 +++ 8 files changed, 344 insertions(+), 264 deletions(-) create mode 100644 graph/src/test/scala/org/apache/spark/graph/GraphOpsSuite.scala create mode 100644 graph/src/test/scala/org/apache/spark/graph/VertexRDDSuite.scala create mode 100644 graph/src/test/scala/org/apache/spark/graph/impl/EdgePartitionSuite.scala diff --git a/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala b/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala index d36339b65b711..1e6d8ec7cf22d 100644 --- a/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala @@ -7,8 +7,6 @@ import org.apache.spark.SparkContext._ import org.apache.spark.graph.algorithms._ import org.apache.spark.rdd._ -import org.apache.spark.graph.LocalSparkContext._ - import org.apache.spark.graph.util.GraphGenerators @@ -48,16 +46,13 @@ object GridPageRank { class AnalyticsSuite extends FunSuite with LocalSparkContext { - System.setProperty("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - System.setProperty("spark.kryo.registrator", "org.apache.spark.graph.GraphKryoRegistrator") - def compareRanks(a: VertexRDD[Double], b: VertexRDD[Double]): Double = { a.leftJoin(b) { case (id, a, bOpt) => (a - bOpt.getOrElse(0.0)) * (a - bOpt.getOrElse(0.0)) } .map { case (id, error) => error }.sum } test("Star PageRank") { - withSpark(new SparkContext("local", "test")) { sc => + withSpark { sc => val nVertices = 100 val starGraph = GraphGenerators.starGraph(sc, nVertices).cache() val resetProb = 0.15 @@ -89,7 +84,7 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { test("Grid PageRank") { - withSpark(new SparkContext("local", "test")) { sc => + withSpark { sc => val rows = 10 val cols = 10 val resetProb = 0.15 @@ -111,7 +106,7 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { test("Chain PageRank") { - withSpark(new SparkContext("local", "test")) { sc => + withSpark { sc => val chain1 = (0 until 9).map(x => (x, x+1) ) val rawEdges = sc.parallelize(chain1, 1).map { case (s,d) => (s.toLong, d.toLong) } val chain = Graph.fromEdgeTuples(rawEdges, 1.0).cache() @@ -131,7 +126,7 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { test("Grid Connected Components") { - withSpark(new SparkContext("local", "test")) { sc => + withSpark { sc => val gridGraph = GraphGenerators.gridGraph(sc, 10, 10).cache() val ccGraph = ConnectedComponents.run(gridGraph).cache() val maxCCid = ccGraph.vertices.map { case (vid, ccId) => ccId }.sum @@ -141,7 +136,7 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { test("Reverse Grid Connected Components") { - withSpark(new SparkContext("local", "test")) { sc => + withSpark { sc => val gridGraph = GraphGenerators.gridGraph(sc, 10, 10).reverse.cache() val ccGraph = ConnectedComponents.run(gridGraph).cache() val maxCCid = ccGraph.vertices.map { case (vid, ccId) => ccId }.sum @@ -151,7 +146,7 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { test("Chain Connected Components") { - withSpark(new SparkContext("local", "test")) { sc => + withSpark { sc => val chain1 = (0 until 9).map(x => (x, x+1) ) val chain2 = (10 until 20).map(x => (x, x+1) ) val rawEdges = sc.parallelize(chain1 ++ chain2, 3).map { case (s,d) => (s.toLong, d.toLong) } @@ -174,7 +169,7 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { } // end of chain connected components test("Reverse Chain Connected Components") { - withSpark(new SparkContext("local", "test")) { sc => + withSpark { sc => val chain1 = (0 until 9).map(x => (x, x+1) ) val chain2 = (10 until 20).map(x => (x, x+1) ) val rawEdges = sc.parallelize(chain1 ++ chain2, 3).map { case (s,d) => (s.toLong, d.toLong) } @@ -200,7 +195,7 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { } // end of reverse chain connected components test("Island Strongly Connected Components") { - withSpark(new SparkContext("local", "test")) { sc => + withSpark { sc => val vertices = sc.parallelize((1L to 5L).map(x => (x, -1))) val edges = sc.parallelize(Seq.empty[Edge[Int]]) val graph = Graph(vertices, edges) @@ -212,7 +207,7 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { } test("Cycle Strongly Connected Components") { - withSpark(new SparkContext("local", "test")) { sc => + withSpark { sc => val rawEdges = sc.parallelize((0L to 6L).map(x => (x, (x + 1) % 7))) val graph = Graph.fromEdgeTuples(rawEdges, -1) val sccGraph = StronglyConnectedComponents.run(graph, 20) @@ -223,7 +218,7 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { } test("2 Cycle Strongly Connected Components") { - withSpark(new SparkContext("local", "test")) { sc => + withSpark { sc => val edges = Array(0L -> 1L, 1L -> 2L, 2L -> 0L) ++ Array(3L -> 4L, 4L -> 5L, 5L -> 3L) ++ @@ -243,7 +238,7 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { } test("Count a single triangle") { - withSpark(new SparkContext("local", "test")) { sc => + withSpark { sc => val rawEdges = sc.parallelize(Array( 0L->1L, 1L->2L, 2L->0L ), 2) val graph = Graph.fromEdgeTuples(rawEdges, true).cache() val triangleCount = TriangleCount.run(graph) @@ -253,7 +248,7 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { } test("Count two triangles") { - withSpark(new SparkContext("local", "test")) { sc => + withSpark { sc => val triangles = Array(0L -> 1L, 1L -> 2L, 2L -> 0L) ++ Array(0L -> -1L, -1L -> -2L, -2L -> 0L) val rawEdges = sc.parallelize(triangles, 2) @@ -271,7 +266,7 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { } test("Count two triangles with bi-directed edges") { - withSpark(new SparkContext("local", "test")) { sc => + withSpark { sc => val triangles = Array(0L -> 1L, 1L -> 2L, 2L -> 0L) ++ Array(0L -> -1L, -1L -> -2L, -2L -> 0L) @@ -291,7 +286,7 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { } test("Count a single triangle with duplicate edges") { - withSpark(new SparkContext("local", "test")) { sc => + withSpark { sc => val rawEdges = sc.parallelize(Array(0L -> 1L, 1L -> 2L, 2L -> 0L) ++ Array(0L -> 1L, 1L -> 2L, 2L -> 0L), 2) val graph = Graph.fromEdgeTuples(rawEdges, true, uniqueEdges = Some(RandomVertexCut)).cache() @@ -302,14 +297,14 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { } test("Test SVD++ with mean square error on training set") { - withSpark(new SparkContext("local", "test")) { sc => + withSpark { sc => val SvdppErr = 0.01 - val edges = sc.textFile("mllib/data/als/test.data").map { line => + val edges = sc.textFile("mllib/data/als/test.data").map { line => val fields = line.split(",") Edge(fields(0).toLong * 2, fields(1).toLong * 2 + 1, fields(2).toDouble) } val graph = Svdpp.run(edges) - val err = graph.vertices.collect.map{ case (vid, vd) => + val err = graph.vertices.collect.map{ case (vid, vd) => if (vid % 2 == 1) { vd.norm } else { 0.0 } }.reduce(_ + _) / graph.triplets.collect.size assert(err < SvdppErr) diff --git a/graph/src/test/scala/org/apache/spark/graph/GraphOpsSuite.scala b/graph/src/test/scala/org/apache/spark/graph/GraphOpsSuite.scala new file mode 100644 index 0000000000000..c055e461b74f3 --- /dev/null +++ b/graph/src/test/scala/org/apache/spark/graph/GraphOpsSuite.scala @@ -0,0 +1,94 @@ +package org.apache.spark.graph + +import scala.util.Random + +import org.scalatest.FunSuite + +import org.apache.spark.SparkContext +import org.apache.spark.graph.Graph._ +import org.apache.spark.graph.impl.EdgePartition +import org.apache.spark.graph.impl.EdgePartitionBuilder +import org.apache.spark.rdd._ + +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: Vid, x: Vid))), 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: Vid, edge: EdgeTriplet[Int, Int]) => None, + (a: Int, b: Int) => throw new Exception("reduceFunc called unexpectedly"), + EdgeDirection.In) + assert(noVertexValues.collect().toSet === Set.empty[(Vid, Int)]) + } + } + + test("joinVertices") { + withSpark { sc => + val vertices = sc.parallelize(Seq[(Vid, String)]((1, "one"), (2, "two"), (3, "three")), 2) + val edges = sc.parallelize((Seq(Edge(1, 2, "onetwo")))) + val g: Graph[String, String] = Graph(vertices, edges) + + val tbl = sc.parallelize(Seq[(Vid, Int)]((1, 10), (2, 20))) + val g1 = g.joinVertices(tbl) { (vid: Vid, attr: String, u: Int) => attr + u } + + val v = g1.vertices.collect().toSet + assert(v === Set((1, "one10"), (2, "two20"), (3, "three"))) + } + } + + test("collectNeighborIds") { + withSpark { sc => + 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) + val nbrs = graph.collectNeighborIds(EdgeDirection.Both) + assert(nbrs.count === chain.size) + assert(graph.numVertices === nbrs.count) + nbrs.collect.foreach { case (vid, nbrs) => assert(nbrs.size === 2) } + nbrs.collect.foreach { case (vid, nbrs) => + val s = nbrs.toSet + assert(s.contains((vid + 1) % 100)) + assert(s.contains(if (vid > 0) vid - 1 else 99 )) + } + } + } + + test ("filter") { + withSpark { sc => + val n = 5 + val vertices = sc.parallelize((0 to n).map(x => (x:Vid, x))) + val edges = sc.parallelize((1 to n).map(x => Edge(0, x, x))) + val graph: Graph[Int, Int] = Graph(vertices, edges) + val filteredGraph = graph.filter( + graph => { + val degrees: VertexRDD[Int] = graph.outDegrees + graph.outerJoinVertices(degrees) {(vid, data, deg) => deg.getOrElse(0)} + }, + vpred = (vid: Vid, deg:Int) => deg > 0 + ) + + val v = filteredGraph.vertices.collect().toSet + assert(v === Set((0,0))) + + // the map is necessary because of object-reuse in the edge iterator + val e = filteredGraph.edges.map(e => Edge(e.srcId, e.dstId, e.attr)).collect().toSet + assert(e.isEmpty) + } + } + +} diff --git a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala index f6bb201a83b32..09da1023501e7 100644 --- a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala @@ -6,32 +6,47 @@ import org.scalatest.FunSuite import org.apache.spark.SparkContext import org.apache.spark.graph.Graph._ -import org.apache.spark.graph.LocalSparkContext._ import org.apache.spark.graph.impl.EdgePartition import org.apache.spark.graph.impl.EdgePartitionBuilder import org.apache.spark.rdd._ class GraphSuite extends FunSuite with LocalSparkContext { - System.setProperty("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - System.setProperty("spark.kryo.registrator", "org.apache.spark.graph.GraphKryoRegistrator") + def starGraph(sc: SparkContext, n: Int): Graph[String, Int] = { + Graph.fromEdgeTuples(sc.parallelize((1 to n).map(x => (0: Vid, x: Vid)), 3), "v") + } - test("Graph Creation") { - withSpark(new SparkContext("local", "test")) { sc => - val rawEdges = (0L to 100L).zip((1L to 99L) :+ 0L) - val edges = sc.parallelize(rawEdges) - val graph = Graph.fromEdgeTuples(edges, 1.0F) - assert(graph.edges.count() === rawEdges.size) + test("Graph.fromEdgeTuples") { + withSpark { sc => + val ring = (0L to 100L).zip((1L to 99L) :+ 0L) + val doubleRing = ring ++ ring + val graph = Graph.fromEdgeTuples(sc.parallelize(doubleRing), 1) + assert(graph.edges.count() === doubleRing.size) + assert(graph.edges.collect.forall(e => e.attr == 1)) + + // uniqueEdges option should uniquify edges and store duplicate count in edge attributes + val uniqueGraph = Graph.fromEdgeTuples(sc.parallelize(doubleRing), 1, Some(RandomVertexCut)) + assert(uniqueGraph.edges.count() === ring.size) + assert(uniqueGraph.edges.collect.forall(e => e.attr == 2)) } } - test("Graph Creation with invalid vertices") { - withSpark(new SparkContext("local", "test")) { sc => + test("Graph.fromEdges") { + withSpark { sc => + val ring = (0L to 100L).zip((1L to 99L) :+ 0L).map { case (a, b) => Edge(a, b, 1) } + val graph = Graph.fromEdges(sc.parallelize(ring), 1.0F) + assert(graph.edges.count() === ring.size) + } + } + + test("Graph.apply") { + withSpark { sc => val rawEdges = (0L to 98L).zip((1L to 99L) :+ 0L) val edges: RDD[Edge[Int]] = sc.parallelize(rawEdges).map { case (s, t) => Edge(s, t, 1) } val vertices: RDD[(Vid, Boolean)] = sc.parallelize((0L until 10L).map(id => (id, true))) val graph = Graph(vertices, edges, false) assert( graph.edges.count() === rawEdges.size ) + // Vertices not explicitly provided but referenced by edges should be created automatically assert( graph.vertices.count() === 100) graph.triplets.map { et => assert((et.srcId < 10 && et.srcAttr) || (et.srcId >= 10 && !et.srcAttr)) @@ -40,153 +55,97 @@ class GraphSuite extends FunSuite with LocalSparkContext { } } - test("core operations") { - withSpark(new SparkContext("local", "test")) { sc => + test("triplets") { + withSpark { sc => val n = 5 - val star = Graph.fromEdgeTuples( - sc.parallelize((1 to n).map(x => (0: Vid, x: Vid)), 3), "v") - // triplets + val star = starGraph(sc, n) assert(star.triplets.map(et => (et.srcId, et.dstId, et.srcAttr, et.dstAttr)).collect.toSet === (1 to n).map(x => (0: Vid, x: Vid, "v", "v")).toSet) - // reverse - val reverseStar = star.reverse - assert(reverseStar.outDegrees.collect.toSet === (1 to n).map(x => (x: Vid, 1)).toSet) - // outerJoinVertices - val reverseStarDegrees = - reverseStar.outerJoinVertices(reverseStar.outDegrees) { (vid, a, bOpt) => bOpt.getOrElse(0) } - val neighborDegreeSums = reverseStarDegrees.mapReduceTriplets( - et => Iterator((et.srcId, et.dstAttr), (et.dstId, et.srcAttr)), - (a: Int, b: Int) => a + b).collect.toSet - assert(neighborDegreeSums === Set((0: Vid, n)) ++ (1 to n).map(x => (x: Vid, 0))) + } + } + + test("partitionBy") { + withSpark { sc => + def mkGraph(edges: List[(Long, Long)]) = Graph.fromEdgeTuples(sc.parallelize(edges, 2), 0) + def nonemptyParts(graph: Graph[Int, Int]) = { + graph.edges.partitionsRDD.mapPartitions { iter => + Iterator(iter.next()._2.iterator.toList) + }.filter(_.nonEmpty) + } + val identicalEdges = List((0L, 1L), (0L, 1L)) + val canonicalEdges = List((0L, 1L), (1L, 0L)) + val sameSrcEdges = List((0L, 1L), (0L, 2L)) + + // The two edges start out in different partitions + for (edges <- List(identicalEdges, canonicalEdges, sameSrcEdges)) { + assert(nonemptyParts(mkGraph(edges)).count === 2) + } + // partitionBy(RandomVertexCut) puts identical edges in the same partition + assert(nonemptyParts(mkGraph(identicalEdges).partitionBy(RandomVertexCut)).count === 1) + // partitionBy(EdgePartition1D) puts same-source edges in the same partition + assert(nonemptyParts(mkGraph(sameSrcEdges).partitionBy(EdgePartition1D)).count === 1) + // partitionBy(CanonicalRandomVertexCut) puts edges that are identical modulo direction into + // the same partition + assert(nonemptyParts(mkGraph(canonicalEdges).partitionBy(CanonicalRandomVertexCut)).count === 1) + // TODO(ankurdave): Test EdgePartition2D by checking the 2 * sqrt(p) bound on vertex + // replication + } + } + + test("mapVertices") { + withSpark { sc => + val n = 5 + val star = starGraph(sc, n) // mapVertices preserving type - val mappedVAttrs = reverseStar.mapVertices((vid, attr) => attr + "2") + val mappedVAttrs = star.mapVertices((vid, attr) => attr + "2") assert(mappedVAttrs.vertices.collect.toSet === (0 to n).map(x => (x: Vid, "v2")).toSet) // mapVertices changing type - val mappedVAttrs2 = reverseStar.mapVertices((vid, attr) => attr.length) + val mappedVAttrs2 = star.mapVertices((vid, attr) => attr.length) assert(mappedVAttrs2.vertices.collect.toSet === (0 to n).map(x => (x: Vid, 1)).toSet) - // groupEdges - val doubleStar = Graph.fromEdgeTuples( - sc.parallelize((1 to n).flatMap(x => List((0: Vid, x: Vid), (0: Vid, x: Vid))), 1), "v") - val star2 = doubleStar.groupEdges { (a, b) => a} - assert(star2.edges.collect.toArray.sorted(Edge.lexicographicOrdering[Int]) === - star.edges.collect.toArray.sorted(Edge.lexicographicOrdering[Int])) - assert(star2.vertices.collect.toSet === star.vertices.collect.toSet) } } test("mapEdges") { - withSpark(new SparkContext("local", "test")) { sc => + withSpark { sc => val n = 3 - val star = Graph.fromEdgeTuples( - sc.parallelize((1 to n).map(x => (0: Vid, x: Vid))), "v") + val star = starGraph(sc, n) val starWithEdgeAttrs = star.mapEdges(e => e.dstId) - // map(_.copy()) is a workaround for https://github.com/amplab/graphx/issues/25 - val edges = starWithEdgeAttrs.edges.map(_.copy()).collect() + val edges = starWithEdgeAttrs.edges.collect() assert(edges.size === n) assert(edges.toSet === (1 to n).map(x => Edge(0, x, x)).toSet) } } - test("mapReduceTriplets") { - withSpark(new SparkContext("local", "test")) { sc => - val n = 5 - val star = Graph.fromEdgeTuples(sc.parallelize((1 to n).map(x => (0: Vid, x: Vid))), 0) - val starDeg = star.joinVertices(star.degrees){ (vid, oldV, deg) => deg } - val neighborDegreeSums = starDeg.mapReduceTriplets( - edge => Iterator((edge.srcId, edge.dstAttr), (edge.dstId, edge.srcAttr)), - (a: Int, b: Int) => a + b) - assert(neighborDegreeSums.collect().toSet === (0 to n).map(x => (x, n)).toSet) - - // activeSetOpt - val allPairs = for (x <- 1 to n; y <- 1 to n) yield (x: Vid, y: Vid) - val complete = Graph.fromEdgeTuples(sc.parallelize(allPairs, 3), 0) - val vids = complete.mapVertices((vid, attr) => vid).cache() - val active = vids.vertices.filter { case (vid, attr) => attr % 2 == 0 } - val numEvenNeighbors = vids.mapReduceTriplets(et => { - // Map function should only run on edges with destination in the active set - if (et.dstId % 2 != 0) { - throw new Exception("map ran on edge with dst vid %d, which is odd".format(et.dstId)) - } - Iterator((et.srcId, 1)) - }, (a: Int, b: Int) => a + b, Some((active, EdgeDirection.In))).collect.toSet - assert(numEvenNeighbors === (1 to n).map(x => (x: Vid, n / 2)).toSet) - - // outerJoinVertices followed by mapReduceTriplets(activeSetOpt) - val ring = Graph.fromEdgeTuples(sc.parallelize((0 until n).map(x => (x: Vid, (x+1) % n: Vid)), 3), 0) - .mapVertices((vid, attr) => vid).cache() - val changed = ring.vertices.filter { case (vid, attr) => attr % 2 == 1 }.mapValues(-_) - val changedGraph = ring.outerJoinVertices(changed) { (vid, old, newOpt) => newOpt.getOrElse(old) } - val numOddNeighbors = changedGraph.mapReduceTriplets(et => { - // Map function should only run on edges with source in the active set - if (et.srcId % 2 != 1) { - throw new Exception("map ran on edge with src vid %d, which is even".format(et.dstId)) - } - Iterator((et.dstId, 1)) - }, (a: Int, b: Int) => a + b, Some(changed, EdgeDirection.Out)).collect.toSet - assert(numOddNeighbors === (2 to n by 2).map(x => (x: Vid, 1)).toSet) - - } + test("mapTriplets") { } - test("aggregateNeighbors") { - withSpark(new SparkContext("local", "test")) { sc => - val n = 3 - val star = Graph.fromEdgeTuples(sc.parallelize((1 to n).map(x => (0: Vid, x: Vid))), 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: Vid, edge: EdgeTriplet[Int, Int]) => None, - (a: Int, b: Int) => throw new Exception("reduceFunc called unexpectedly"), - EdgeDirection.In) - assert(noVertexValues.collect().toSet === Set.empty[(Vid, Int)]) + test("reverse") { + withSpark { sc => + val n = 5 + val star = starGraph(sc, n) + assert(star.reverse.outDegrees.collect.toSet === (1 to n).map(x => (x: Vid, 1)).toSet) } } - test("joinVertices") { - withSpark(new SparkContext("local", "test")) { sc => - val vertices = sc.parallelize(Seq[(Vid, String)]((1, "one"), (2, "two"), (3, "three")), 2) - val edges = sc.parallelize((Seq(Edge(1, 2, "onetwo")))) - val g: Graph[String, String] = Graph(vertices, edges) - - val tbl = sc.parallelize(Seq[(Vid, Int)]((1, 10), (2, 20))) - val g1 = g.joinVertices(tbl) { (vid: Vid, attr: String, u: Int) => attr + u } + test("subgraph") { + withSpark { sc => + // Create a star graph of 10 veritces. + val n = 10 + val star = starGraph(sc, n) + // Take only vertices whose vids are even + val subgraph = star.subgraph(vpred = (vid, attr) => vid % 2 == 0) - val v = g1.vertices.collect().toSet - assert(v === Set((1, "one10"), (2, "two20"), (3, "three"))) - } - } + // We should have 5 vertices. + assert(subgraph.vertices.collect().toSet === (0 to n by 2).map(x => (x, "v")).toSet) - test("collectNeighborIds") { - withSpark(new SparkContext("local", "test")) { sc => - 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) - val nbrs = graph.collectNeighborIds(EdgeDirection.Both) - assert(nbrs.count === chain.size) - assert(graph.numVertices === nbrs.count) - nbrs.collect.foreach { case (vid, nbrs) => assert(nbrs.size === 2) } - nbrs.collect.foreach { case (vid, nbrs) => - val s = nbrs.toSet - assert(s.contains((vid + 1) % 100)) - assert(s.contains(if (vid > 0) vid - 1 else 99 )) - } + // And 4 edges. + assert(subgraph.edges.map(_.copy()).collect().toSet === (2 to n by 2).map(x => Edge(0, x, 1)).toSet) } } test("mask") { - withSpark(new SparkContext("local", "test")) { sc => + withSpark { sc => val n = 5 val vertices = sc.parallelize((0 to n).map(x => (x:Vid, x))) val edges = sc.parallelize((1 to n).map(x => Edge(0, x, x))) @@ -209,90 +168,71 @@ class GraphSuite extends FunSuite with LocalSparkContext { } } - test ("filter") { - withSpark(new SparkContext("local", "test")) { sc => + test("groupEdges") { + withSpark { sc => val n = 5 - val vertices = sc.parallelize((0 to n).map(x => (x:Vid, x))) - val edges = sc.parallelize((1 to n).map(x => Edge(0, x, x))) - val graph: Graph[Int, Int] = Graph(vertices, edges) - val filteredGraph = graph.filter( - graph => { - val degrees: VertexRDD[Int] = graph.outDegrees - graph.outerJoinVertices(degrees) {(vid, data, deg) => deg.getOrElse(0)} - }, - vpred = (vid: Vid, deg:Int) => deg > 0 - ) - - val v = filteredGraph.vertices.collect().toSet - assert(v === Set((0,0))) - - // the map is necessary because of object-reuse in the edge iterator - val e = filteredGraph.edges.map(e => Edge(e.srcId, e.dstId, e.attr)).collect().toSet - assert(e.isEmpty) + val star = starGraph(sc, n) + val doubleStar = Graph.fromEdgeTuples( + sc.parallelize((1 to n).flatMap(x => List((0: Vid, x: Vid), (0: Vid, x: Vid))), 1), "v") + val star2 = doubleStar.groupEdges { (a, b) => a} + assert(star2.edges.collect.toArray.sorted(Edge.lexicographicOrdering[Int]) === + star.edges.collect.toArray.sorted(Edge.lexicographicOrdering[Int])) + assert(star2.vertices.collect.toSet === star.vertices.collect.toSet) } } - test("VertexSetRDD") { - withSpark(new SparkContext("local", "test")) { sc => - val n = 100 - val a = sc.parallelize((0 to n).map(x => (x.toLong, x.toLong)), 5) - val b = VertexRDD(a).mapValues(x => -x).cache() // Allow joining b with a derived RDD of b - assert(b.count === n + 1) - assert(b.leftJoin(a){ (id, a, bOpt) => a + bOpt.get }.map(x=> x._2).reduce(_+_) === 0) - val c = b.aggregateUsingIndex[Long](a, (x, y) => x) - assert(b.leftJoin(c){ (id, b, cOpt) => b + cOpt.get }.map(x=> x._2).reduce(_+_) === 0) - val d = c.filter(q => ((q._2 % 2) == 0)) - val e = a.filter(q => ((q._2 % 2) == 0)) - assert(d.count === e.count) - assert(b.zipJoin(c)((id, b, c) => b + c).map(x => x._2).reduce(_+_) === 0) - val f = b.mapValues(x => if (x % 2 == 0) -x else x) - assert(b.diff(f).collect().toSet === (2 to n by 2).map(x => (x.toLong, x.toLong)).toSet) - } - } + test("mapReduceTriplets") { + withSpark { sc => + val n = 5 + val star = starGraph(sc, n).mapVertices { (_, _) => 0 } + val starDeg = star.joinVertices(star.degrees){ (vid, oldV, deg) => deg } + val neighborDegreeSums = starDeg.mapReduceTriplets( + edge => Iterator((edge.srcId, edge.dstAttr), (edge.dstId, edge.srcAttr)), + (a: Int, b: Int) => a + b) + assert(neighborDegreeSums.collect().toSet === (0 to n).map(x => (x, n)).toSet) - test("subgraph") { - withSpark(new SparkContext("local", "test")) { sc => - // Create a star graph of 10 veritces. - val n = 10 - val star = Graph.fromEdgeTuples(sc.parallelize((1 to n).map(x => (0: Vid, x: Vid))), "v") - // Take only vertices whose vids are even - val subgraph = star.subgraph(vpred = (vid, attr) => vid % 2 == 0) + // activeSetOpt + val allPairs = for (x <- 1 to n; y <- 1 to n) yield (x: Vid, y: Vid) + val complete = Graph.fromEdgeTuples(sc.parallelize(allPairs, 3), 0) + val vids = complete.mapVertices((vid, attr) => vid).cache() + val active = vids.vertices.filter { case (vid, attr) => attr % 2 == 0 } + val numEvenNeighbors = vids.mapReduceTriplets(et => { + // Map function should only run on edges with destination in the active set + if (et.dstId % 2 != 0) { + throw new Exception("map ran on edge with dst vid %d, which is odd".format(et.dstId)) + } + Iterator((et.srcId, 1)) + }, (a: Int, b: Int) => a + b, Some((active, EdgeDirection.In))).collect.toSet + assert(numEvenNeighbors === (1 to n).map(x => (x: Vid, n / 2)).toSet) - // We should have 5 vertices. - assert(subgraph.vertices.collect().toSet === (0 to n by 2).map(x => (x, "v")).toSet) + // outerJoinVertices followed by mapReduceTriplets(activeSetOpt) + val ring = Graph.fromEdgeTuples(sc.parallelize((0 until n).map(x => (x: Vid, (x+1) % n: Vid)), 3), 0) + .mapVertices((vid, attr) => vid).cache() + val changed = ring.vertices.filter { case (vid, attr) => attr % 2 == 1 }.mapValues(-_) + val changedGraph = ring.outerJoinVertices(changed) { (vid, old, newOpt) => newOpt.getOrElse(old) } + val numOddNeighbors = changedGraph.mapReduceTriplets(et => { + // Map function should only run on edges with source in the active set + if (et.srcId % 2 != 1) { + throw new Exception("map ran on edge with src vid %d, which is even".format(et.dstId)) + } + Iterator((et.dstId, 1)) + }, (a: Int, b: Int) => a + b, Some(changed, EdgeDirection.Out)).collect.toSet + assert(numOddNeighbors === (2 to n by 2).map(x => (x: Vid, 1)).toSet) - // And 4 edges. - assert(subgraph.edges.map(_.copy()).collect().toSet === (2 to n by 2).map(x => Edge(0, x, 1)).toSet) } } - test("EdgePartition.sort") { - val edgesFrom0 = List(Edge(0, 1, 0)) - val edgesFrom1 = List(Edge(1, 0, 0), Edge(1, 2, 0)) - val sortedEdges = edgesFrom0 ++ edgesFrom1 - val builder = new EdgePartitionBuilder[Int] - for (e <- Random.shuffle(sortedEdges)) { - builder.add(e.srcId, e.dstId, e.attr) + test("outerJoinVertices") { + withSpark { sc => + val n = 5 + val reverseStar = starGraph(sc, n).reverse + val reverseStarDegrees = + reverseStar.outerJoinVertices(reverseStar.outDegrees) { (vid, a, bOpt) => bOpt.getOrElse(0) } + val neighborDegreeSums = reverseStarDegrees.mapReduceTriplets( + et => Iterator((et.srcId, et.dstAttr), (et.dstId, et.srcAttr)), + (a: Int, b: Int) => a + b).collect.toSet + assert(neighborDegreeSums === Set((0: Vid, n)) ++ (1 to n).map(x => (x: Vid, 0))) } - - val edgePartition = builder.toEdgePartition - assert(edgePartition.iterator.map(_.copy()).toList === sortedEdges) - assert(edgePartition.indexIterator(_ == 0).map(_.copy()).toList === edgesFrom0) - assert(edgePartition.indexIterator(_ == 1).map(_.copy()).toList === edgesFrom1) } - test("EdgePartition.innerJoin") { - def makeEdgePartition[A: ClassManifest](xs: Iterable[(Int, Int, A)]): EdgePartition[A] = { - val builder = new EdgePartitionBuilder[A] - for ((src, dst, attr) <- xs) { builder.add(src: Vid, dst: Vid, attr) } - builder.toEdgePartition - } - val aList = List((0, 1, 0), (1, 0, 0), (1, 2, 0), (5, 4, 0), (5, 5, 0)) - val bList = List((0, 1, 0), (1, 0, 0), (1, 1, 0), (3, 4, 0), (5, 5, 0)) - val a = makeEdgePartition(aList) - val b = makeEdgePartition(bList) - - assert(a.innerJoin(b) { (src, dst, a, b) => a }.iterator.map(_.copy()).toList === - List(Edge(0, 1, 0), Edge(1, 0, 0), Edge(5, 5, 0))) - } } diff --git a/graph/src/test/scala/org/apache/spark/graph/LocalSparkContext.scala b/graph/src/test/scala/org/apache/spark/graph/LocalSparkContext.scala index 4a0155b6bd281..5c20d559aa973 100644 --- a/graph/src/test/scala/org/apache/spark/graph/LocalSparkContext.scala +++ b/graph/src/test/scala/org/apache/spark/graph/LocalSparkContext.scala @@ -6,39 +6,23 @@ import org.scalatest.BeforeAndAfterEach import org.apache.spark.SparkContext -/** Manages a local `sc` {@link SparkContext} variable, correctly stopping it after each test. */ -trait LocalSparkContext extends BeforeAndAfterEach { self: Suite => - - @transient var sc: SparkContext = _ - - override def afterEach() { - resetSparkContext() - super.afterEach() - } - - def resetSparkContext() = { - if (sc != null) { - LocalSparkContext.stop(sc) - sc = null - } - } - -} - -object LocalSparkContext { - def stop(sc: SparkContext) { - sc.stop() - // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown - System.clearProperty("spark.driver.port") - } - - /** Runs `f` by passing in `sc` and ensures that `sc` is stopped. */ - def withSpark[T](sc: SparkContext)(f: SparkContext => T) = { +/** + * Provides a method to run tests against a {@link SparkContext} variable that is correctly stopped + * after each test. +*/ +trait LocalSparkContext { + System.setProperty("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + System.setProperty("spark.kryo.registrator", "org.apache.spark.graph.GraphKryoRegistrator") + + /** Runs `f` on a new SparkContext and ensures that it is stopped afterwards. */ + def withSpark[T](f: SparkContext => T) = { + val sc = new SparkContext("local", "test") try { f(sc) } finally { - stop(sc) + sc.stop() + // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown + System.clearProperty("spark.driver.port") } } - } diff --git a/graph/src/test/scala/org/apache/spark/graph/PregelSuite.scala b/graph/src/test/scala/org/apache/spark/graph/PregelSuite.scala index 0897d9783eaf8..44182e85eeafe 100644 --- a/graph/src/test/scala/org/apache/spark/graph/PregelSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/PregelSuite.scala @@ -3,16 +3,12 @@ package org.apache.spark.graph import org.scalatest.FunSuite import org.apache.spark.SparkContext -import org.apache.spark.graph.LocalSparkContext._ import org.apache.spark.rdd._ class PregelSuite extends FunSuite with LocalSparkContext { - System.setProperty("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - System.setProperty("spark.kryo.registrator", "org.apache.spark.graph.GraphKryoRegistrator") - test("1 iteration") { - withSpark(new SparkContext("local", "test")) { sc => + withSpark { sc => val n = 5 val star = Graph.fromEdgeTuples(sc.parallelize((1 to n).map(x => (0: Vid, x: Vid)), 3), "v") val result = Pregel(star, 0)( @@ -24,7 +20,7 @@ class PregelSuite extends FunSuite with LocalSparkContext { } test("chain propagation") { - withSpark(new SparkContext("local", "test")) { sc => + withSpark { sc => val n = 5 val chain = Graph.fromEdgeTuples( sc.parallelize((1 until n).map(x => (x: Vid, x + 1: Vid)), 3), diff --git a/graph/src/test/scala/org/apache/spark/graph/SerializerSuite.scala b/graph/src/test/scala/org/apache/spark/graph/SerializerSuite.scala index 6b86f9b25d00c..80075f3437fb8 100644 --- a/graph/src/test/scala/org/apache/spark/graph/SerializerSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/SerializerSuite.scala @@ -7,7 +7,6 @@ import scala.util.Random import org.scalatest.FunSuite import org.apache.spark._ -import org.apache.spark.graph.LocalSparkContext._ import org.apache.spark.graph.impl._ import org.apache.spark.graph.impl.MsgRDDFunctions._ import org.apache.spark.serializer.SerializationStream @@ -15,9 +14,6 @@ import org.apache.spark.serializer.SerializationStream class SerializerSuite extends FunSuite with LocalSparkContext { - System.setProperty("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - System.setProperty("spark.kryo.registrator", "org.apache.spark.graph.GraphKryoRegistrator") - test("IntVertexBroadcastMsgSerializer") { val outMsg = new VertexBroadcastMsg[Int](3, 4, 5) val bout = new ByteArrayOutputStream @@ -139,7 +135,7 @@ class SerializerSuite extends FunSuite with LocalSparkContext { } test("TestShuffleVertexBroadcastMsg") { - withSpark(new SparkContext("local[2]", "test")) { sc => + withSpark { sc => val bmsgs = sc.parallelize(0 until 100, 10).map { pid => new VertexBroadcastMsg[Int](pid, pid, pid) } diff --git a/graph/src/test/scala/org/apache/spark/graph/VertexRDDSuite.scala b/graph/src/test/scala/org/apache/spark/graph/VertexRDDSuite.scala new file mode 100644 index 0000000000000..316968bbf0474 --- /dev/null +++ b/graph/src/test/scala/org/apache/spark/graph/VertexRDDSuite.scala @@ -0,0 +1,32 @@ +package org.apache.spark.graph + +import scala.util.Random + +import org.scalatest.FunSuite + +import org.apache.spark.SparkContext +import org.apache.spark.graph.Graph._ +import org.apache.spark.graph.impl.EdgePartition +import org.apache.spark.graph.impl.EdgePartitionBuilder +import org.apache.spark.rdd._ + +class VertexRDDSuite extends FunSuite with LocalSparkContext { + + test("VertexRDD") { + withSpark { sc => + val n = 100 + val a = sc.parallelize((0 to n).map(x => (x.toLong, x.toLong)), 5) + val b = VertexRDD(a).mapValues(x => -x).cache() // Allow joining b with a derived RDD of b + assert(b.count === n + 1) + assert(b.leftJoin(a){ (id, a, bOpt) => a + bOpt.get }.map(x=> x._2).reduce(_+_) === 0) + val c = b.aggregateUsingIndex[Long](a, (x, y) => x) + assert(b.leftJoin(c){ (id, b, cOpt) => b + cOpt.get }.map(x=> x._2).reduce(_+_) === 0) + val d = c.filter(q => ((q._2 % 2) == 0)) + val e = a.filter(q => ((q._2 % 2) == 0)) + assert(d.count === e.count) + assert(b.zipJoin(c)((id, b, c) => b + c).map(x => x._2).reduce(_+_) === 0) + val f = b.mapValues(x => if (x % 2 == 0) -x else x) + assert(b.diff(f).collect().toSet === (2 to n by 2).map(x => (x.toLong, x.toLong)).toSet) + } + } +} diff --git a/graph/src/test/scala/org/apache/spark/graph/impl/EdgePartitionSuite.scala b/graph/src/test/scala/org/apache/spark/graph/impl/EdgePartitionSuite.scala new file mode 100644 index 0000000000000..2bce90120d6c7 --- /dev/null +++ b/graph/src/test/scala/org/apache/spark/graph/impl/EdgePartitionSuite.scala @@ -0,0 +1,43 @@ +package org.apache.spark.graph.impl + +import scala.util.Random + +import org.scalatest.FunSuite + +import org.apache.spark.SparkContext +import org.apache.spark.graph.Graph._ +import org.apache.spark.graph._ +import org.apache.spark.rdd._ + +class EdgePartitionSuite extends FunSuite { + + test("EdgePartition.sort") { + val edgesFrom0 = List(Edge(0, 1, 0)) + val edgesFrom1 = List(Edge(1, 0, 0), Edge(1, 2, 0)) + val sortedEdges = edgesFrom0 ++ edgesFrom1 + val builder = new EdgePartitionBuilder[Int] + for (e <- Random.shuffle(sortedEdges)) { + builder.add(e.srcId, e.dstId, e.attr) + } + + val edgePartition = builder.toEdgePartition + assert(edgePartition.iterator.map(_.copy()).toList === sortedEdges) + assert(edgePartition.indexIterator(_ == 0).map(_.copy()).toList === edgesFrom0) + assert(edgePartition.indexIterator(_ == 1).map(_.copy()).toList === edgesFrom1) + } + + test("EdgePartition.innerJoin") { + def makeEdgePartition[A: ClassManifest](xs: Iterable[(Int, Int, A)]): EdgePartition[A] = { + val builder = new EdgePartitionBuilder[A] + for ((src, dst, attr) <- xs) { builder.add(src: Vid, dst: Vid, attr) } + builder.toEdgePartition + } + val aList = List((0, 1, 0), (1, 0, 0), (1, 2, 0), (5, 4, 0), (5, 5, 0)) + val bList = List((0, 1, 0), (1, 0, 0), (1, 1, 0), (3, 4, 0), (5, 5, 0)) + val a = makeEdgePartition(aList) + val b = makeEdgePartition(bList) + + assert(a.innerJoin(b) { (src, dst, a, b) => a }.iterator.map(_.copy()).toList === + List(Edge(0, 1, 0), Edge(1, 0, 0), Edge(5, 5, 0))) + } +} From a69465b1fa7250d036e1585543c225b6340e4790 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Thu, 19 Dec 2013 20:32:30 -0800 Subject: [PATCH 317/531] Split VertexRDD tests; fix #114 --- .../org/apache/spark/graph/VertexRDD.scala | 26 +----- .../spark/graph/impl/VertexPartition.scala | 38 ++------ .../apache/spark/graph/AnalyticsSuite.scala | 2 +- .../org/apache/spark/graph/GraphSuite.scala | 5 +- .../apache/spark/graph/VertexRDDSuite.scala | 89 +++++++++++++++---- 5 files changed, 86 insertions(+), 74 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala b/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala index 90ac6dc61dc3a..fe0f0ae491277 100644 --- a/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala @@ -187,28 +187,6 @@ class VertexRDD[@specialized VD: ClassManifest]( } } - /** - * Inner join this VertexSet with another VertexSet which has the - * same Index. This function will fail if both VertexSets do not - * share the same index. The resulting vertex set will only contain - * vertices that are in both this and the other vertex set. - * - * @tparam VD2 the attribute type of the other VertexSet - * @tparam VD3 the attribute type of the resulting VertexSet - * - * @param other the other VertexSet with which to join. - * @param f the function mapping a vertex id and its attributes in - * this and the other vertex set to a new vertex attribute. - * @return a VertexRDD containing only the vertices in both this - * and the other VertexSet and with tuple attributes. - */ - def zipJoin[VD2: ClassManifest, VD3: ClassManifest] - (other: VertexRDD[VD2])(f: (Vid, VD, VD2) => VD3): VertexRDD[VD3] = { - this.zipVertexPartitions(other) { (thisPart, otherPart) => - thisPart.join(otherPart)(f) - } - } - /** * Left join this VertexSet with another VertexSet which has the * same Index. This function will fail if both VertexSets do not @@ -309,6 +287,10 @@ class VertexRDD[@specialized VD: ClassManifest]( } } + /** + * Aggregate messages with the same ids using `reduceFunc`, returning a VertexRDD that is + * co-indexed with this one. + */ def aggregateUsingIndex[VD2: ClassManifest]( messages: RDD[(Vid, VD2)], reduceFunc: (VD2, VD2) => VD2): VertexRDD[VD2] = { diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala index ccbc83c512fc6..7710d6eadafc9 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala @@ -125,27 +125,6 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassManifest]( } } - /** Inner join another VertexPartition. */ - def join[VD2: ClassManifest, VD3: ClassManifest] - (other: VertexPartition[VD2]) - (f: (Vid, VD, VD2) => VD3): VertexPartition[VD3] = - { - if (index != other.index) { - logWarning("Joining two VertexPartitions with different indexes is slow.") - join(createUsingIndex(other.iterator))(f) - } else { - val newValues = new Array[VD3](capacity) - val newMask = mask & other.mask - - var i = newMask.nextSetBit(0) - while (i >= 0) { - newValues(i) = f(index.getValue(i), values(i), other.values(i)) - i = mask.nextSetBit(i + 1) - } - new VertexPartition(index, newValues, newMask) - } - } - /** Left outer join another VertexPartition. */ def leftJoin[VD2: ClassManifest, VD3: ClassManifest] (other: VertexPartition[VD2]) @@ -179,15 +158,16 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassManifest]( if (index != other.index) { logWarning("Joining two VertexPartitions with different indexes is slow.") innerJoin(createUsingIndex(other.iterator))(f) + } else { + val newMask = mask & other.mask + val newValues = new Array[VD2](capacity) + var i = newMask.nextSetBit(0) + while (i >= 0) { + newValues(i) = f(index.getValue(i), values(i), other.values(i)) + i = newMask.nextSetBit(i + 1) + } + new VertexPartition(index, newValues, newMask) } - val newMask = mask & other.mask - val newValues = new Array[VD2](capacity) - var i = newMask.nextSetBit(0) - while (i >= 0) { - newValues(i) = f(index.getValue(i), values(i), other.values(i)) - i = newMask.nextSetBit(i + 1) - } - new VertexPartition(index, newValues, newMask) } /** diff --git a/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala b/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala index 1e6d8ec7cf22d..77a193a9acc37 100644 --- a/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala @@ -62,7 +62,7 @@ class AnalyticsSuite extends FunSuite with LocalSparkContext { val staticRanks2 = PageRank.run(starGraph, numIter = 2, resetProb).vertices.cache() // Static PageRank should only take 2 iterations to converge - val notMatching = staticRanks1.zipJoin(staticRanks2) { (vid, pr1, pr2) => + val notMatching = staticRanks1.innerZipJoin(staticRanks2) { (vid, pr1, pr2) => if (pr1 != pr2) 1 else 0 }.map { case (vid, test) => test }.sum assert(notMatching === 0) diff --git a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala index 09da1023501e7..487d949e1f7c4 100644 --- a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala @@ -1,13 +1,9 @@ package org.apache.spark.graph -import scala.util.Random - import org.scalatest.FunSuite import org.apache.spark.SparkContext import org.apache.spark.graph.Graph._ -import org.apache.spark.graph.impl.EdgePartition -import org.apache.spark.graph.impl.EdgePartitionBuilder import org.apache.spark.rdd._ class GraphSuite extends FunSuite with LocalSparkContext { @@ -118,6 +114,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { } test("mapTriplets") { + // TODO(ankurdave): Write the test } test("reverse") { diff --git a/graph/src/test/scala/org/apache/spark/graph/VertexRDDSuite.scala b/graph/src/test/scala/org/apache/spark/graph/VertexRDDSuite.scala index 316968bbf0474..e876b8e4e8bfe 100644 --- a/graph/src/test/scala/org/apache/spark/graph/VertexRDDSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/VertexRDDSuite.scala @@ -1,32 +1,85 @@ package org.apache.spark.graph -import scala.util.Random - -import org.scalatest.FunSuite - import org.apache.spark.SparkContext import org.apache.spark.graph.Graph._ import org.apache.spark.graph.impl.EdgePartition -import org.apache.spark.graph.impl.EdgePartitionBuilder import org.apache.spark.rdd._ +import org.scalatest.FunSuite class VertexRDDSuite extends FunSuite with LocalSparkContext { - test("VertexRDD") { + def vertices(sc: SparkContext, n: Int) = { + VertexRDD(sc.parallelize((0 to n).map(x => (x.toLong, x)), 5)) + } + + test("filter") { + withSpark { sc => + val n = 100 + val verts = vertices(sc, n) + val evens = verts.filter(q => ((q._2 % 2) == 0)) + assert(evens.count === (0 to n).filter(_ % 2 == 0).size) + } + } + + test("mapValues") { + withSpark { sc => + val n = 100 + val verts = vertices(sc, n) + val negatives = verts.mapValues(x => -x).cache() // Allow joining b with a derived RDD of b + assert(negatives.count === n + 1) + } + } + + test("diff") { withSpark { sc => val n = 100 - val a = sc.parallelize((0 to n).map(x => (x.toLong, x.toLong)), 5) - val b = VertexRDD(a).mapValues(x => -x).cache() // Allow joining b with a derived RDD of b - assert(b.count === n + 1) - assert(b.leftJoin(a){ (id, a, bOpt) => a + bOpt.get }.map(x=> x._2).reduce(_+_) === 0) - val c = b.aggregateUsingIndex[Long](a, (x, y) => x) - assert(b.leftJoin(c){ (id, b, cOpt) => b + cOpt.get }.map(x=> x._2).reduce(_+_) === 0) - val d = c.filter(q => ((q._2 % 2) == 0)) - val e = a.filter(q => ((q._2 % 2) == 0)) - assert(d.count === e.count) - assert(b.zipJoin(c)((id, b, c) => b + c).map(x => x._2).reduce(_+_) === 0) - val f = b.mapValues(x => if (x % 2 == 0) -x else x) - assert(b.diff(f).collect().toSet === (2 to n by 2).map(x => (x.toLong, x.toLong)).toSet) + val verts = vertices(sc, n) + val flipEvens = verts.mapValues(x => if (x % 2 == 0) -x else x) + // diff should keep only the changed vertices + assert(verts.diff(flipEvens).map(_._2).collect().toSet === (2 to n by 2).map(-_).toSet) + // diff should keep the vertex values from `other` + assert(flipEvens.diff(verts).map(_._2).collect().toSet === (2 to n by 2).toSet) } } + + test("leftJoin") { + withSpark { sc => + val n = 100 + val verts = vertices(sc, n) + val evens = verts.filter(q => ((q._2 % 2) == 0)) + // leftJoin with another VertexRDD + assert(verts.leftJoin(evens) { (id, a, bOpt) => a - bOpt.getOrElse(0) }.collect.toSet === + (0 to n by 2).map(x => (x.toLong, 0)).toSet ++ (1 to n by 2).map(x => (x.toLong, x)).toSet) + // leftJoin with an RDD + val evensRDD = evens.map(identity) + assert(verts.leftJoin(evensRDD) { (id, a, bOpt) => a - bOpt.getOrElse(0) }.collect.toSet === + (0 to n by 2).map(x => (x.toLong, 0)).toSet ++ (1 to n by 2).map(x => (x.toLong, x)).toSet) + } + } + + test("innerJoin") { + withSpark { sc => + val n = 100 + val verts = vertices(sc, n) + val evens = verts.filter(q => ((q._2 % 2) == 0)) + // innerJoin with another VertexRDD + assert(verts.innerJoin(evens) { (id, a, b) => a - b }.collect.toSet === + (0 to n by 2).map(x => (x.toLong, 0)).toSet) + // innerJoin with an RDD + val evensRDD = evens.map(identity) + assert(verts.innerJoin(evensRDD) { (id, a, b) => a - b }.collect.toSet === + (0 to n by 2).map(x => (x.toLong, 0)).toSet) } + } + + test("aggregateUsingIndex") { + withSpark { sc => + val n = 100 + val verts = vertices(sc, n) + val messageTargets = (0 to n) ++ (0 to n by 2) + val messages = sc.parallelize(messageTargets.map(x => (x.toLong, 1))) + assert(verts.aggregateUsingIndex[Int](messages, _ + _).collect.toSet === + (0 to n).map(x => (x.toLong, if (x % 2 == 0) 2 else 1)).toSet) + } + } + } From 95381dfb9461af65f31cc2f18a125b3a82abbc3f Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Thu, 19 Dec 2013 20:46:03 -0800 Subject: [PATCH 318/531] Split AnalyticsSuite into algorithms suites --- .../org/apache/spark/graph/GraphLab.scala | 5 +- .../apache/spark/graph/AnalyticsSuite.scala | 313 ------------------ .../apache/spark/graph/GraphOpsSuite.scala | 6 +- .../algorithms/ConnectedComponentsSuite.scala | 83 +++++ .../graph/algorithms/PageRankSuite.scala | 126 +++++++ .../StronglyConnectedComponentsSuite.scala | 57 ++++ .../spark/graph/algorithms/SvdppSuite.scala | 29 ++ .../graph/algorithms/TriangleCountSuite.scala | 73 ++++ .../spark/graph/impl/EdgePartitionSuite.scala | 5 +- 9 files changed, 375 insertions(+), 322 deletions(-) delete mode 100644 graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala create mode 100644 graph/src/test/scala/org/apache/spark/graph/algorithms/ConnectedComponentsSuite.scala create mode 100644 graph/src/test/scala/org/apache/spark/graph/algorithms/PageRankSuite.scala create mode 100644 graph/src/test/scala/org/apache/spark/graph/algorithms/StronglyConnectedComponentsSuite.scala create mode 100644 graph/src/test/scala/org/apache/spark/graph/algorithms/SvdppSuite.scala create mode 100644 graph/src/test/scala/org/apache/spark/graph/algorithms/TriangleCountSuite.scala diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala b/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala index 5618ce6272adb..5d2f0f4bda6f1 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala @@ -1,12 +1,13 @@ package org.apache.spark.graph +import org.apache.spark.Logging import scala.collection.JavaConversions._ import org.apache.spark.rdd.RDD /** * This object implements the GraphLab gather-apply-scatter api. */ -object GraphLab { +object GraphLab extends Logging { /** * Execute the GraphLab Gather-Apply-Scatter API @@ -119,7 +120,7 @@ object GraphLab { numActive = activeGraph.vertices.map{ case (vid, data) => if (data._1) 1 else 0 }.reduce(_ + _) - println("Number active vertices: " + numActive) + logInfo("Number active vertices: " + numActive) i += 1 } diff --git a/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala b/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala deleted file mode 100644 index 77a193a9acc37..0000000000000 --- a/graph/src/test/scala/org/apache/spark/graph/AnalyticsSuite.scala +++ /dev/null @@ -1,313 +0,0 @@ -package org.apache.spark.graph - -import org.scalatest.FunSuite - -import org.apache.spark.SparkContext -import org.apache.spark.SparkContext._ -import org.apache.spark.graph.algorithms._ -import org.apache.spark.rdd._ - -import org.apache.spark.graph.util.GraphGenerators - - -object GridPageRank { - def apply(nRows: Int, nCols: Int, nIter: Int, resetProb: Double) = { - val inNbrs = Array.fill(nRows * nCols)(collection.mutable.MutableList.empty[Int]) - val outDegree = Array.fill(nRows * nCols)(0) - // Convert row column address into vertex ids (row major order) - def sub2ind(r: Int, c: Int): Int = r * nCols + c - // Make the grid graph - for (r <- 0 until nRows; c <- 0 until nCols) { - val ind = sub2ind(r,c) - if (r+1 < nRows) { - outDegree(ind) += 1 - inNbrs(sub2ind(r+1,c)) += ind - } - if (c+1 < nCols) { - outDegree(ind) += 1 - inNbrs(sub2ind(r,c+1)) += ind - } - } - // compute the pagerank - var pr = Array.fill(nRows * nCols)(resetProb) - for (iter <- 0 until nIter) { - val oldPr = pr - pr = new Array[Double](nRows * nCols) - for (ind <- 0 until (nRows * nCols)) { - pr(ind) = resetProb + (1.0 - resetProb) * - inNbrs(ind).map( nbr => oldPr(nbr) / outDegree(nbr)).sum - } - } - (0L until (nRows * nCols)).zip(pr) - } - -} - - -class AnalyticsSuite extends FunSuite with LocalSparkContext { - - def compareRanks(a: VertexRDD[Double], b: VertexRDD[Double]): Double = { - a.leftJoin(b) { case (id, a, bOpt) => (a - bOpt.getOrElse(0.0)) * (a - bOpt.getOrElse(0.0)) } - .map { case (id, error) => error }.sum - } - - test("Star PageRank") { - withSpark { sc => - val nVertices = 100 - val starGraph = GraphGenerators.starGraph(sc, nVertices).cache() - val resetProb = 0.15 - val errorTol = 1.0e-5 - - val staticRanks1 = PageRank.run(starGraph, numIter = 1, resetProb).vertices.cache() - val staticRanks2 = PageRank.run(starGraph, numIter = 2, resetProb).vertices.cache() - - // Static PageRank should only take 2 iterations to converge - val notMatching = staticRanks1.innerZipJoin(staticRanks2) { (vid, pr1, pr2) => - if (pr1 != pr2) 1 else 0 - }.map { case (vid, test) => test }.sum - assert(notMatching === 0) - - val staticErrors = staticRanks2.map { case (vid, pr) => - val correct = (vid > 0 && pr == resetProb) || - (vid == 0 && math.abs(pr - (resetProb + (1.0 - resetProb) * (resetProb * (nVertices - 1)) )) < 1.0E-5) - if (!correct) 1 else 0 - } - assert(staticErrors.sum === 0) - - val dynamicRanks = PageRank.runUntillConvergence(starGraph, 0, resetProb).vertices.cache() - val standaloneRanks = PageRank.runStandalone(starGraph, 0, resetProb).cache() - assert(compareRanks(staticRanks2, dynamicRanks) < errorTol) - assert(compareRanks(staticRanks2, standaloneRanks) < errorTol) - } - } // end of test Star PageRank - - - - test("Grid PageRank") { - withSpark { sc => - val rows = 10 - val cols = 10 - val resetProb = 0.15 - val tol = 0.0001 - val numIter = 50 - val errorTol = 1.0e-5 - val gridGraph = GraphGenerators.gridGraph(sc, rows, cols).cache() - - val staticRanks = PageRank.run(gridGraph, numIter, resetProb).vertices.cache() - val dynamicRanks = PageRank.runUntillConvergence(gridGraph, tol, resetProb).vertices.cache() - val standaloneRanks = PageRank.runStandalone(gridGraph, tol, resetProb).cache() - val referenceRanks = VertexRDD(sc.parallelize(GridPageRank(rows, cols, numIter, resetProb))) - - assert(compareRanks(staticRanks, referenceRanks) < errorTol) - assert(compareRanks(dynamicRanks, referenceRanks) < errorTol) - assert(compareRanks(standaloneRanks, referenceRanks) < errorTol) - } - } // end of Grid PageRank - - - test("Chain PageRank") { - withSpark { sc => - val chain1 = (0 until 9).map(x => (x, x+1) ) - val rawEdges = sc.parallelize(chain1, 1).map { case (s,d) => (s.toLong, d.toLong) } - val chain = Graph.fromEdgeTuples(rawEdges, 1.0).cache() - val resetProb = 0.15 - val tol = 0.0001 - val numIter = 10 - val errorTol = 1.0e-5 - - val staticRanks = PageRank.run(chain, numIter, resetProb).vertices.cache() - val dynamicRanks = PageRank.runUntillConvergence(chain, tol, resetProb).vertices.cache() - val standaloneRanks = PageRank.runStandalone(chain, tol, resetProb).cache() - - assert(compareRanks(staticRanks, dynamicRanks) < errorTol) - assert(compareRanks(dynamicRanks, standaloneRanks) < errorTol) - } - } - - - test("Grid Connected Components") { - withSpark { sc => - val gridGraph = GraphGenerators.gridGraph(sc, 10, 10).cache() - val ccGraph = ConnectedComponents.run(gridGraph).cache() - val maxCCid = ccGraph.vertices.map { case (vid, ccId) => ccId }.sum - assert(maxCCid === 0) - } - } // end of Grid connected components - - - test("Reverse Grid Connected Components") { - withSpark { sc => - val gridGraph = GraphGenerators.gridGraph(sc, 10, 10).reverse.cache() - val ccGraph = ConnectedComponents.run(gridGraph).cache() - val maxCCid = ccGraph.vertices.map { case (vid, ccId) => ccId }.sum - assert(maxCCid === 0) - } - } // end of Grid connected components - - - test("Chain Connected Components") { - withSpark { sc => - val chain1 = (0 until 9).map(x => (x, x+1) ) - val chain2 = (10 until 20).map(x => (x, x+1) ) - val rawEdges = sc.parallelize(chain1 ++ chain2, 3).map { case (s,d) => (s.toLong, d.toLong) } - val twoChains = Graph.fromEdgeTuples(rawEdges, 1.0).cache() - val ccGraph = ConnectedComponents.run(twoChains).cache() - val vertices = ccGraph.vertices.collect() - for ( (id, cc) <- vertices ) { - if(id < 10) { assert(cc === 0) } - else { assert(cc === 10) } - } - val ccMap = vertices.toMap - for (id <- 0 until 20) { - if (id < 10) { - assert(ccMap(id) === 0) - } else { - assert(ccMap(id) === 10) - } - } - } - } // end of chain connected components - - test("Reverse Chain Connected Components") { - withSpark { sc => - val chain1 = (0 until 9).map(x => (x, x+1) ) - val chain2 = (10 until 20).map(x => (x, x+1) ) - val rawEdges = sc.parallelize(chain1 ++ chain2, 3).map { case (s,d) => (s.toLong, d.toLong) } - val twoChains = Graph.fromEdgeTuples(rawEdges, true).reverse.cache() - val ccGraph = ConnectedComponents.run(twoChains).cache() - val vertices = ccGraph.vertices.collect - for ( (id, cc) <- vertices ) { - if (id < 10) { - assert(cc === 0) - } else { - assert(cc === 10) - } - } - val ccMap = vertices.toMap - for ( id <- 0 until 20 ) { - if (id < 10) { - assert(ccMap(id) === 0) - } else { - assert(ccMap(id) === 10) - } - } - } - } // end of reverse chain connected components - - test("Island Strongly Connected Components") { - withSpark { sc => - val vertices = sc.parallelize((1L to 5L).map(x => (x, -1))) - val edges = sc.parallelize(Seq.empty[Edge[Int]]) - val graph = Graph(vertices, edges) - val sccGraph = StronglyConnectedComponents.run(graph, 5) - for ((id, scc) <- sccGraph.vertices.collect) { - assert(id == scc) - } - } - } - - test("Cycle Strongly Connected Components") { - withSpark { sc => - val rawEdges = sc.parallelize((0L to 6L).map(x => (x, (x + 1) % 7))) - val graph = Graph.fromEdgeTuples(rawEdges, -1) - val sccGraph = StronglyConnectedComponents.run(graph, 20) - for ((id, scc) <- sccGraph.vertices.collect) { - assert(0L == scc) - } - } - } - - test("2 Cycle Strongly Connected Components") { - withSpark { sc => - val edges = - Array(0L -> 1L, 1L -> 2L, 2L -> 0L) ++ - Array(3L -> 4L, 4L -> 5L, 5L -> 3L) ++ - Array(6L -> 0L, 5L -> 7L) - val rawEdges = sc.parallelize(edges) - val graph = Graph.fromEdgeTuples(rawEdges, -1) - val sccGraph = StronglyConnectedComponents.run(graph, 20) - for ((id, scc) <- sccGraph.vertices.collect) { - if (id < 3) - assert(0L == scc) - else if (id < 6) - assert(3L == scc) - else - assert(id == scc) - } - } - } - - test("Count a single triangle") { - withSpark { sc => - val rawEdges = sc.parallelize(Array( 0L->1L, 1L->2L, 2L->0L ), 2) - val graph = Graph.fromEdgeTuples(rawEdges, true).cache() - val triangleCount = TriangleCount.run(graph) - val verts = triangleCount.vertices - verts.collect.foreach { case (vid, count) => assert(count === 1) } - } - } - - test("Count two triangles") { - withSpark { sc => - val triangles = Array(0L -> 1L, 1L -> 2L, 2L -> 0L) ++ - Array(0L -> -1L, -1L -> -2L, -2L -> 0L) - val rawEdges = sc.parallelize(triangles, 2) - val graph = Graph.fromEdgeTuples(rawEdges, true).cache() - val triangleCount = TriangleCount.run(graph) - val verts = triangleCount.vertices - verts.collect().foreach { case (vid, count) => - if (vid == 0) { - assert(count === 2) - } else { - assert(count === 1) - } - } - } - } - - test("Count two triangles with bi-directed edges") { - withSpark { sc => - val triangles = - Array(0L -> 1L, 1L -> 2L, 2L -> 0L) ++ - Array(0L -> -1L, -1L -> -2L, -2L -> 0L) - val revTriangles = triangles.map { case (a,b) => (b,a) } - val rawEdges = sc.parallelize(triangles ++ revTriangles, 2) - val graph = Graph.fromEdgeTuples(rawEdges, true).cache() - val triangleCount = TriangleCount.run(graph) - val verts = triangleCount.vertices - verts.collect().foreach { case (vid, count) => - if (vid == 0) { - assert(count === 4) - } else { - assert(count === 2) - } - } - } - } - - test("Count a single triangle with duplicate edges") { - withSpark { sc => - val rawEdges = sc.parallelize(Array(0L -> 1L, 1L -> 2L, 2L -> 0L) ++ - Array(0L -> 1L, 1L -> 2L, 2L -> 0L), 2) - val graph = Graph.fromEdgeTuples(rawEdges, true, uniqueEdges = Some(RandomVertexCut)).cache() - val triangleCount = TriangleCount.run(graph) - val verts = triangleCount.vertices - verts.collect.foreach { case (vid, count) => assert(count === 1) } - } - } - - test("Test SVD++ with mean square error on training set") { - withSpark { sc => - val SvdppErr = 0.01 - val edges = sc.textFile("mllib/data/als/test.data").map { line => - val fields = line.split(",") - Edge(fields(0).toLong * 2, fields(1).toLong * 2 + 1, fields(2).toDouble) - } - val graph = Svdpp.run(edges) - val err = graph.vertices.collect.map{ case (vid, vd) => - if (vid % 2 == 1) { vd.norm } else { 0.0 } - }.reduce(_ + _) / graph.triplets.collect.size - assert(err < SvdppErr) - } - } -} // end of AnalyticsSuite diff --git a/graph/src/test/scala/org/apache/spark/graph/GraphOpsSuite.scala b/graph/src/test/scala/org/apache/spark/graph/GraphOpsSuite.scala index c055e461b74f3..9e9213631ff63 100644 --- a/graph/src/test/scala/org/apache/spark/graph/GraphOpsSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/GraphOpsSuite.scala @@ -1,14 +1,10 @@ package org.apache.spark.graph -import scala.util.Random - -import org.scalatest.FunSuite - import org.apache.spark.SparkContext import org.apache.spark.graph.Graph._ import org.apache.spark.graph.impl.EdgePartition -import org.apache.spark.graph.impl.EdgePartitionBuilder import org.apache.spark.rdd._ +import org.scalatest.FunSuite class GraphOpsSuite extends FunSuite with LocalSparkContext { diff --git a/graph/src/test/scala/org/apache/spark/graph/algorithms/ConnectedComponentsSuite.scala b/graph/src/test/scala/org/apache/spark/graph/algorithms/ConnectedComponentsSuite.scala new file mode 100644 index 0000000000000..81a1b7337f2a4 --- /dev/null +++ b/graph/src/test/scala/org/apache/spark/graph/algorithms/ConnectedComponentsSuite.scala @@ -0,0 +1,83 @@ +package org.apache.spark.graph.algorithms + +import org.scalatest.FunSuite + +import org.apache.spark.SparkContext +import org.apache.spark.SparkContext._ +import org.apache.spark.graph._ +import org.apache.spark.graph.util.GraphGenerators +import org.apache.spark.rdd._ + + +class ConnectedComponentsSuite extends FunSuite with LocalSparkContext { + + test("Grid Connected Components") { + withSpark { sc => + val gridGraph = GraphGenerators.gridGraph(sc, 10, 10).cache() + val ccGraph = ConnectedComponents.run(gridGraph).cache() + val maxCCid = ccGraph.vertices.map { case (vid, ccId) => ccId }.sum + assert(maxCCid === 0) + } + } // end of Grid connected components + + + test("Reverse Grid Connected Components") { + withSpark { sc => + val gridGraph = GraphGenerators.gridGraph(sc, 10, 10).reverse.cache() + val ccGraph = ConnectedComponents.run(gridGraph).cache() + val maxCCid = ccGraph.vertices.map { case (vid, ccId) => ccId }.sum + assert(maxCCid === 0) + } + } // end of Grid connected components + + + test("Chain Connected Components") { + withSpark { sc => + val chain1 = (0 until 9).map(x => (x, x+1) ) + val chain2 = (10 until 20).map(x => (x, x+1) ) + val rawEdges = sc.parallelize(chain1 ++ chain2, 3).map { case (s,d) => (s.toLong, d.toLong) } + val twoChains = Graph.fromEdgeTuples(rawEdges, 1.0).cache() + val ccGraph = ConnectedComponents.run(twoChains).cache() + val vertices = ccGraph.vertices.collect() + for ( (id, cc) <- vertices ) { + if(id < 10) { assert(cc === 0) } + else { assert(cc === 10) } + } + val ccMap = vertices.toMap + for (id <- 0 until 20) { + if (id < 10) { + assert(ccMap(id) === 0) + } else { + assert(ccMap(id) === 10) + } + } + } + } // end of chain connected components + + test("Reverse Chain Connected Components") { + withSpark { sc => + val chain1 = (0 until 9).map(x => (x, x+1) ) + val chain2 = (10 until 20).map(x => (x, x+1) ) + val rawEdges = sc.parallelize(chain1 ++ chain2, 3).map { case (s,d) => (s.toLong, d.toLong) } + val twoChains = Graph.fromEdgeTuples(rawEdges, true).reverse.cache() + val ccGraph = ConnectedComponents.run(twoChains).cache() + val vertices = ccGraph.vertices.collect + for ( (id, cc) <- vertices ) { + if (id < 10) { + assert(cc === 0) + } else { + assert(cc === 10) + } + } + val ccMap = vertices.toMap + for ( id <- 0 until 20 ) { + if (id < 10) { + assert(ccMap(id) === 0) + } else { + assert(ccMap(id) === 10) + } + } + } + } // end of reverse chain connected components + +} diff --git a/graph/src/test/scala/org/apache/spark/graph/algorithms/PageRankSuite.scala b/graph/src/test/scala/org/apache/spark/graph/algorithms/PageRankSuite.scala new file mode 100644 index 0000000000000..81d82a5a6b719 --- /dev/null +++ b/graph/src/test/scala/org/apache/spark/graph/algorithms/PageRankSuite.scala @@ -0,0 +1,126 @@ +package org.apache.spark.graph.algorithms + +import org.scalatest.FunSuite + +import org.apache.spark.graph._ +import org.apache.spark.SparkContext +import org.apache.spark.SparkContext._ +import org.apache.spark.rdd._ + +import org.apache.spark.graph.util.GraphGenerators + + +object GridPageRank { + def apply(nRows: Int, nCols: Int, nIter: Int, resetProb: Double) = { + val inNbrs = Array.fill(nRows * nCols)(collection.mutable.MutableList.empty[Int]) + val outDegree = Array.fill(nRows * nCols)(0) + // Convert row column address into vertex ids (row major order) + def sub2ind(r: Int, c: Int): Int = r * nCols + c + // Make the grid graph + for (r <- 0 until nRows; c <- 0 until nCols) { + val ind = sub2ind(r,c) + if (r+1 < nRows) { + outDegree(ind) += 1 + inNbrs(sub2ind(r+1,c)) += ind + } + if (c+1 < nCols) { + outDegree(ind) += 1 + inNbrs(sub2ind(r,c+1)) += ind + } + } + // compute the pagerank + var pr = Array.fill(nRows * nCols)(resetProb) + for (iter <- 0 until nIter) { + val oldPr = pr + pr = new Array[Double](nRows * nCols) + for (ind <- 0 until (nRows * nCols)) { + pr(ind) = resetProb + (1.0 - resetProb) * + inNbrs(ind).map( nbr => oldPr(nbr) / outDegree(nbr)).sum + } + } + (0L until (nRows * nCols)).zip(pr) + } + +} + + +class PageRankSuite extends FunSuite with LocalSparkContext { + + def compareRanks(a: VertexRDD[Double], b: VertexRDD[Double]): Double = { + a.leftJoin(b) { case (id, a, bOpt) => (a - bOpt.getOrElse(0.0)) * (a - bOpt.getOrElse(0.0)) } + .map { case (id, error) => error }.sum + } + + test("Star PageRank") { + withSpark { sc => + val nVertices = 100 + val starGraph = GraphGenerators.starGraph(sc, nVertices).cache() + val resetProb = 0.15 + val errorTol = 1.0e-5 + + val staticRanks1 = PageRank.run(starGraph, numIter = 1, resetProb).vertices.cache() + val staticRanks2 = PageRank.run(starGraph, numIter = 2, resetProb).vertices.cache() + + // Static PageRank should only take 2 iterations to converge + val notMatching = staticRanks1.innerZipJoin(staticRanks2) { (vid, pr1, pr2) => + if (pr1 != pr2) 1 else 0 + }.map { case (vid, test) => test }.sum + assert(notMatching === 0) + + val staticErrors = staticRanks2.map { case (vid, pr) => + val correct = (vid > 0 && pr == resetProb) || + (vid == 0 && math.abs(pr - (resetProb + (1.0 - resetProb) * (resetProb * (nVertices - 1)) )) < 1.0E-5) + if (!correct) 1 else 0 + } + assert(staticErrors.sum === 0) + + val dynamicRanks = PageRank.runUntillConvergence(starGraph, 0, resetProb).vertices.cache() + val standaloneRanks = PageRank.runStandalone(starGraph, 0, resetProb).cache() + assert(compareRanks(staticRanks2, dynamicRanks) < errorTol) + assert(compareRanks(staticRanks2, standaloneRanks) < errorTol) + } + } // end of test Star PageRank + + + + test("Grid PageRank") { + withSpark { sc => + val rows = 10 + val cols = 10 + val resetProb = 0.15 + val tol = 0.0001 + val numIter = 50 + val errorTol = 1.0e-5 + val gridGraph = GraphGenerators.gridGraph(sc, rows, cols).cache() + + val staticRanks = PageRank.run(gridGraph, numIter, resetProb).vertices.cache() + val dynamicRanks = PageRank.runUntillConvergence(gridGraph, tol, resetProb).vertices.cache() + val standaloneRanks = PageRank.runStandalone(gridGraph, tol, resetProb).cache() + val referenceRanks = VertexRDD(sc.parallelize(GridPageRank(rows, cols, numIter, resetProb))) + + assert(compareRanks(staticRanks, referenceRanks) < errorTol) + assert(compareRanks(dynamicRanks, referenceRanks) < errorTol) + assert(compareRanks(standaloneRanks, referenceRanks) < errorTol) + } + } // end of Grid PageRank + + + test("Chain PageRank") { + withSpark { sc => + val chain1 = (0 until 9).map(x => (x, x+1) ) + val rawEdges = sc.parallelize(chain1, 1).map { case (s,d) => (s.toLong, d.toLong) } + val chain = Graph.fromEdgeTuples(rawEdges, 1.0).cache() + val resetProb = 0.15 + val tol = 0.0001 + val numIter = 10 + val errorTol = 1.0e-5 + + val staticRanks = PageRank.run(chain, numIter, resetProb).vertices.cache() + val dynamicRanks = PageRank.runUntillConvergence(chain, tol, resetProb).vertices.cache() + val standaloneRanks = PageRank.runStandalone(chain, tol, resetProb).cache() + + assert(compareRanks(staticRanks, dynamicRanks) < errorTol) + assert(compareRanks(dynamicRanks, standaloneRanks) < errorTol) + } + } +} diff --git a/graph/src/test/scala/org/apache/spark/graph/algorithms/StronglyConnectedComponentsSuite.scala b/graph/src/test/scala/org/apache/spark/graph/algorithms/StronglyConnectedComponentsSuite.scala new file mode 100644 index 0000000000000..4afb158a68ed4 --- /dev/null +++ b/graph/src/test/scala/org/apache/spark/graph/algorithms/StronglyConnectedComponentsSuite.scala @@ -0,0 +1,57 @@ +package org.apache.spark.graph.algorithms + +import org.scalatest.FunSuite + +import org.apache.spark.SparkContext +import org.apache.spark.SparkContext._ +import org.apache.spark.graph._ +import org.apache.spark.graph.util.GraphGenerators +import org.apache.spark.rdd._ + + +class StronglyConnectedComponentsSuite extends FunSuite with LocalSparkContext { + + test("Island Strongly Connected Components") { + withSpark { sc => + val vertices = sc.parallelize((1L to 5L).map(x => (x, -1))) + val edges = sc.parallelize(Seq.empty[Edge[Int]]) + val graph = Graph(vertices, edges) + val sccGraph = StronglyConnectedComponents.run(graph, 5) + for ((id, scc) <- sccGraph.vertices.collect) { + assert(id == scc) + } + } + } + + test("Cycle Strongly Connected Components") { + withSpark { sc => + val rawEdges = sc.parallelize((0L to 6L).map(x => (x, (x + 1) % 7))) + val graph = Graph.fromEdgeTuples(rawEdges, -1) + val sccGraph = StronglyConnectedComponents.run(graph, 20) + for ((id, scc) <- sccGraph.vertices.collect) { + assert(0L == scc) + } + } + } + + test("2 Cycle Strongly Connected Components") { + withSpark { sc => + val edges = + Array(0L -> 1L, 1L -> 2L, 2L -> 0L) ++ + Array(3L -> 4L, 4L -> 5L, 5L -> 3L) ++ + Array(6L -> 0L, 5L -> 7L) + val rawEdges = sc.parallelize(edges) + val graph = Graph.fromEdgeTuples(rawEdges, -1) + val sccGraph = StronglyConnectedComponents.run(graph, 20) + for ((id, scc) <- sccGraph.vertices.collect) { + if (id < 3) + assert(0L == scc) + else if (id < 6) + assert(3L == scc) + else + assert(id == scc) + } + } + } + +} diff --git a/graph/src/test/scala/org/apache/spark/graph/algorithms/SvdppSuite.scala b/graph/src/test/scala/org/apache/spark/graph/algorithms/SvdppSuite.scala new file mode 100644 index 0000000000000..4ea675c2dc1ba --- /dev/null +++ b/graph/src/test/scala/org/apache/spark/graph/algorithms/SvdppSuite.scala @@ -0,0 +1,29 @@ +package org.apache.spark.graph.algorithms + +import org.scalatest.FunSuite + +import org.apache.spark.SparkContext +import org.apache.spark.SparkContext._ +import org.apache.spark.graph._ +import org.apache.spark.graph.util.GraphGenerators +import org.apache.spark.rdd._ + + +class SvdppSuite extends FunSuite with LocalSparkContext { + + test("Test SVD++ with mean square error on training set") { + withSpark { sc => + val SvdppErr = 0.01 + val edges = sc.textFile("mllib/data/als/test.data").map { line => + val fields = line.split(",") + Edge(fields(0).toLong * 2, fields(1).toLong * 2 + 1, fields(2).toDouble) + } + val graph = Svdpp.run(edges) + val err = graph.vertices.collect.map{ case (vid, vd) => + if (vid % 2 == 1) { vd.norm } else { 0.0 } + }.reduce(_ + _) / graph.triplets.collect.size + assert(err < SvdppErr) + } + } + +} diff --git a/graph/src/test/scala/org/apache/spark/graph/algorithms/TriangleCountSuite.scala b/graph/src/test/scala/org/apache/spark/graph/algorithms/TriangleCountSuite.scala new file mode 100644 index 0000000000000..274ab11f0cbd1 --- /dev/null +++ b/graph/src/test/scala/org/apache/spark/graph/algorithms/TriangleCountSuite.scala @@ -0,0 +1,73 @@ +package org.apache.spark.graph.algorithms + +import org.scalatest.FunSuite + +import org.apache.spark.SparkContext +import org.apache.spark.SparkContext._ +import org.apache.spark.graph._ +import org.apache.spark.graph.util.GraphGenerators +import org.apache.spark.rdd._ + + +class TriangleCountSuite extends FunSuite with LocalSparkContext { + + test("Count a single triangle") { + withSpark { sc => + val rawEdges = sc.parallelize(Array( 0L->1L, 1L->2L, 2L->0L ), 2) + val graph = Graph.fromEdgeTuples(rawEdges, true).cache() + val triangleCount = TriangleCount.run(graph) + val verts = triangleCount.vertices + verts.collect.foreach { case (vid, count) => assert(count === 1) } + } + } + + test("Count two triangles") { + withSpark { sc => + val triangles = Array(0L -> 1L, 1L -> 2L, 2L -> 0L) ++ + Array(0L -> -1L, -1L -> -2L, -2L -> 0L) + val rawEdges = sc.parallelize(triangles, 2) + val graph = Graph.fromEdgeTuples(rawEdges, true).cache() + val triangleCount = TriangleCount.run(graph) + val verts = triangleCount.vertices + verts.collect().foreach { case (vid, count) => + if (vid == 0) { + assert(count === 2) + } else { + assert(count === 1) + } + } + } + } + + test("Count two triangles with bi-directed edges") { + withSpark { sc => + val triangles = + Array(0L -> 1L, 1L -> 2L, 2L -> 0L) ++ + Array(0L -> -1L, -1L -> -2L, -2L -> 0L) + val revTriangles = triangles.map { case (a,b) => (b,a) } + val rawEdges = sc.parallelize(triangles ++ revTriangles, 2) + val graph = Graph.fromEdgeTuples(rawEdges, true).cache() + val triangleCount = TriangleCount.run(graph) + val verts = triangleCount.vertices + verts.collect().foreach { case (vid, count) => + if (vid == 0) { + assert(count === 4) + } else { + assert(count === 2) + } + } + } + } + + test("Count a single triangle with duplicate edges") { + withSpark { sc => + val rawEdges = sc.parallelize(Array(0L -> 1L, 1L -> 2L, 2L -> 0L) ++ + Array(0L -> 1L, 1L -> 2L, 2L -> 0L), 2) + val graph = Graph.fromEdgeTuples(rawEdges, true, uniqueEdges = Some(RandomVertexCut)).cache() + val triangleCount = TriangleCount.run(graph) + val verts = triangleCount.vertices + verts.collect.foreach { case (vid, count) => assert(count === 1) } + } + } + +} diff --git a/graph/src/test/scala/org/apache/spark/graph/impl/EdgePartitionSuite.scala b/graph/src/test/scala/org/apache/spark/graph/impl/EdgePartitionSuite.scala index 2bce90120d6c7..caedb55ea2a8b 100644 --- a/graph/src/test/scala/org/apache/spark/graph/impl/EdgePartitionSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/impl/EdgePartitionSuite.scala @@ -9,9 +9,10 @@ import org.apache.spark.graph.Graph._ import org.apache.spark.graph._ import org.apache.spark.rdd._ + class EdgePartitionSuite extends FunSuite { - test("EdgePartition.sort") { + test("sort") { val edgesFrom0 = List(Edge(0, 1, 0)) val edgesFrom1 = List(Edge(1, 0, 0), Edge(1, 2, 0)) val sortedEdges = edgesFrom0 ++ edgesFrom1 @@ -26,7 +27,7 @@ class EdgePartitionSuite extends FunSuite { assert(edgePartition.indexIterator(_ == 1).map(_.copy()).toList === edgesFrom1) } - test("EdgePartition.innerJoin") { + test("innerJoin") { def makeEdgePartition[A: ClassManifest](xs: Iterable[(Int, Int, A)]): EdgePartition[A] = { val builder = new EdgePartitionBuilder[A] for ((src, dst, attr) <- xs) { builder.add(src: Vid, dst: Vid, attr) } From c3ec91a462c1e9582b9cb08a231f2aad10e4e52e Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 20 Dec 2013 00:30:21 -0800 Subject: [PATCH 319/531] Write mapTriplets test --- .../scala/org/apache/spark/graph/GraphSuite.scala | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) diff --git a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala index 487d949e1f7c4..6494ef890003a 100644 --- a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala @@ -114,7 +114,12 @@ class GraphSuite extends FunSuite with LocalSparkContext { } test("mapTriplets") { - // TODO(ankurdave): Write the test + withSpark { sc => + val n = 5 + val star = starGraph(sc, n) + assert(star.mapTriplets(et => et.srcAttr + et.dstAttr).edges.collect.toSet === + (1L to n).map(x => Edge(0, x, "vv")).toSet) + } } test("reverse") { @@ -223,12 +228,19 @@ class GraphSuite extends FunSuite with LocalSparkContext { withSpark { sc => val n = 5 val reverseStar = starGraph(sc, n).reverse + // outerJoinVertices changing type val reverseStarDegrees = reverseStar.outerJoinVertices(reverseStar.outDegrees) { (vid, a, bOpt) => bOpt.getOrElse(0) } val neighborDegreeSums = reverseStarDegrees.mapReduceTriplets( et => Iterator((et.srcId, et.dstAttr), (et.dstId, et.srcAttr)), (a: Int, b: Int) => a + b).collect.toSet assert(neighborDegreeSums === Set((0: Vid, n)) ++ (1 to n).map(x => (x: Vid, 0))) + // outerJoinVertices preserving type + val messages = reverseStar.vertices.mapValues { (vid, attr) => vid.toString } + val newReverseStar = + reverseStar.outerJoinVertices(messages) { (vid, a, bOpt) => a + bOpt.getOrElse("") } + assert(newReverseStar.vertices.map(_._2).collect.toSet === + (0 to n).map(x => "v%d".format(x)).toSet) } } From 343d8977aa7d53f381b014778fb60106f9cbcabb Mon Sep 17 00:00:00 2001 From: Wang Jianping J Date: Fri, 20 Dec 2013 16:57:24 +0800 Subject: [PATCH 320/531] remove unused variable and fix a bug --- .../org/apache/spark/graph/algorithms/Svdpp.scala | 15 ++++++--------- 1 file changed, 6 insertions(+), 9 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala b/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala index 4ddf0b1fd5ee4..ffd0ddba7ee8c 100644 --- a/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala +++ b/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala @@ -21,6 +21,7 @@ class Msg ( // message object Svdpp { // implement SVD++ based on http://public.research.att.com/~volinsky/netflix/kdd08koren.pdf + // model (15) on page 6 def run(edges: RDD[Edge[Double]]): Graph[VT, Double] = { // defalut parameters @@ -91,15 +92,13 @@ object Svdpp { assert(et.srcAttr != null && et.dstAttr != null) val usr = et.srcAttr val itm = et.dstAttr - var p = usr.v1 - var q = itm.v1 - val itmBias = 0.0 - val usrBias = 0.0 + val p = usr.v1 + val q = itm.v1 var pred = u + usr.bias + itm.bias + q.dotProduct(usr.v2) pred = math.max(pred, minVal) pred = math.min(pred, maxVal) val err = et.attr - pred - val y = (q.mapMultiply(err*usr.norm)).subtract((usr.v2).mapMultiply(gamma7)) + val y = (q.mapMultiply(err*usr.norm)).subtract((itm.v2).mapMultiply(gamma7)) val newP = (q.mapMultiply(err)).subtract(p.mapMultiply(gamma7)) // for each connected item q val newQ = (usr.v2.mapMultiply(err)).subtract(q.mapMultiply(gamma7)) Iterator((et.srcId, new Msg(newP, y, err - gamma6*usr.bias)), (et.dstId, new Msg(newQ, y, err - gamma6*itm.bias))) @@ -135,10 +134,8 @@ object Svdpp { assert(et.srcAttr != null && et.dstAttr != null) val usr = et.srcAttr val itm = et.dstAttr - var p = usr.v1 - var q = itm.v1 - val itmBias = 0.0 - val usrBias = 0.0 + val p = usr.v1 + val q = itm.v1 var pred = u + usr.bias + itm.bias + q.dotProduct(usr.v2) pred = math.max(pred, minVal) pred = math.min(pred, maxVal) From 752c0106e8d935875a06ec21b05de2c1f3dbc56a Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 20 Dec 2013 01:06:49 -0800 Subject: [PATCH 321/531] Test EdgePartition2D --- .../org/apache/spark/graph/GraphSuite.scala | 28 +++++++++++++++++-- 1 file changed, 26 insertions(+), 2 deletions(-) diff --git a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala index 6494ef890003a..e6c19dbc40358 100644 --- a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala @@ -83,8 +83,32 @@ class GraphSuite extends FunSuite with LocalSparkContext { // partitionBy(CanonicalRandomVertexCut) puts edges that are identical modulo direction into // the same partition assert(nonemptyParts(mkGraph(canonicalEdges).partitionBy(CanonicalRandomVertexCut)).count === 1) - // TODO(ankurdave): Test EdgePartition2D by checking the 2 * sqrt(p) bound on vertex - // replication + // partitionBy(EdgePartition2D) puts identical edges in the same partition + assert(nonemptyParts(mkGraph(identicalEdges).partitionBy(EdgePartition2D)).count === 1) + + // partitionBy(EdgePartition2D) ensures that vertices need only be replicated to 2 * sqrt(p) + // partitions + val n = 100 + val p = 100 + val verts = 1 to n + val graph = Graph.fromEdgeTuples(sc.parallelize(verts.flatMap(x => + verts.filter(y => y % x == 0).map(y => (x: Vid, y: Vid))), p), 0) + assert(graph.edges.partitions.length === p) + val partitionedGraph = graph.partitionBy(EdgePartition2D) + assert(graph.edges.partitions.length === p) + val bound = 2 * math.sqrt(p) + // Each vertex should be replicated to at most 2 * sqrt(p) partitions + val partitionSets = partitionedGraph.edges.partitionsRDD.mapPartitions { iter => + val part = iter.next()._2 + Iterator((part.srcIds ++ part.dstIds).toSet) + }.collect + assert(verts.forall(id => partitionSets.count(_.contains(id)) <= bound)) + // This should not be true for the default hash partitioning + val partitionSetsUnpartitioned = graph.edges.partitionsRDD.mapPartitions { iter => + val part = iter.next()._2 + Iterator((part.srcIds ++ part.dstIds).toSet) + }.collect + assert(verts.exists(id => partitionSetsUnpartitioned.count(_.contains(id)) > bound)) } } From efc765cf1a287c398e3321c374263a740200fe89 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 20 Dec 2013 01:30:33 -0800 Subject: [PATCH 322/531] Test all methods on EdgePartition --- .../spark/graph/impl/EdgePartitionSuite.scala | 41 ++++++++++++++++--- 1 file changed, 36 insertions(+), 5 deletions(-) diff --git a/graph/src/test/scala/org/apache/spark/graph/impl/EdgePartitionSuite.scala b/graph/src/test/scala/org/apache/spark/graph/impl/EdgePartitionSuite.scala index caedb55ea2a8b..a52a5653e2cf6 100644 --- a/graph/src/test/scala/org/apache/spark/graph/impl/EdgePartitionSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/impl/EdgePartitionSuite.scala @@ -4,15 +4,46 @@ import scala.util.Random import org.scalatest.FunSuite -import org.apache.spark.SparkContext -import org.apache.spark.graph.Graph._ import org.apache.spark.graph._ -import org.apache.spark.rdd._ - class EdgePartitionSuite extends FunSuite { - test("sort") { + test("reverse") { + val edges = List(Edge(0, 1, 0), Edge(1, 2, 0), Edge(2, 0, 0)) + val reversedEdges = List(Edge(0, 2, 0), Edge(1, 0, 0), Edge(2, 1, 0)) + val builder = new EdgePartitionBuilder[Int] + for (e <- edges) { + builder.add(e.srcId, e.dstId, e.attr) + } + val edgePartition = builder.toEdgePartition + assert(edgePartition.reverse.iterator.map(_.copy()).toList === reversedEdges) + assert(edgePartition.reverse.reverse.iterator.map(_.copy()).toList === edges) + } + + test("map") { + val edges = List(Edge(0, 1, 0), Edge(1, 2, 0), Edge(2, 0, 0)) + val builder = new EdgePartitionBuilder[Int] + for (e <- edges) { + builder.add(e.srcId, e.dstId, e.attr) + } + val edgePartition = builder.toEdgePartition + assert(edgePartition.map(e => e.srcId + e.dstId).iterator.map(_.copy()).toList === + edges.map(e => e.copy(attr = e.srcId + e.dstId))) + } + + test("groupEdges") { + val edges = List( + Edge(0, 1, 1), Edge(1, 2, 2), Edge(2, 0, 4), Edge(0, 1, 8), Edge(1, 2, 16), Edge(2, 0, 32)) + val groupedEdges = List(Edge(0, 1, 9), Edge(1, 2, 18), Edge(2, 0, 36)) + val builder = new EdgePartitionBuilder[Int] + for (e <- edges) { + builder.add(e.srcId, e.dstId, e.attr) + } + val edgePartition = builder.toEdgePartition + assert(edgePartition.groupEdges(_ + _).iterator.map(_.copy()).toList === groupedEdges) + } + + test("indexIterator") { val edgesFrom0 = List(Edge(0, 1, 0)) val edgesFrom1 = List(Edge(1, 0, 0), Edge(1, 2, 0)) val sortedEdges = edgesFrom0 ++ edgesFrom1 From 32508e20d468fcb72fb89e6ae23c9fdd6475f0c8 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 20 Dec 2013 12:59:07 -0800 Subject: [PATCH 323/531] Test VertexPartition and fix bugs --- .../spark/graph/impl/VertexPartition.scala | 24 ++-- .../graph/impl/VertexPartitionSuite.scala | 113 ++++++++++++++++++ 2 files changed, 128 insertions(+), 9 deletions(-) create mode 100644 graph/src/test/scala/org/apache/spark/graph/impl/VertexPartitionSuite.scala diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala index 7710d6eadafc9..9b2d66999cc5a 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala @@ -188,8 +188,10 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassManifest]( val newValues = new Array[VD2](capacity) iter.foreach { case (vid, vdata) => val pos = index.getPos(vid) - newMask.set(pos) - newValues(pos) = vdata + if (pos >= 0) { + newMask.set(pos) + newValues(pos) = vdata + } } new VertexPartition[VD2](index, newValues, newMask) } @@ -204,8 +206,10 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassManifest]( System.arraycopy(values, 0, newValues, 0, newValues.length) iter.foreach { case (vid, vdata) => val pos = index.getPos(vid) - newMask.set(pos) - newValues(pos) = vdata + if (pos >= 0) { + newMask.set(pos) + newValues(pos) = vdata + } } new VertexPartition(index, newValues, newMask) } @@ -219,11 +223,13 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassManifest]( val vid = product._1 val vdata = product._2 val pos = index.getPos(vid) - if (newMask.get(pos)) { - newValues(pos) = reduceFunc(newValues(pos), vdata) - } else { // otherwise just store the new value - newMask.set(pos) - newValues(pos) = vdata + if (pos >= 0) { + if (newMask.get(pos)) { + newValues(pos) = reduceFunc(newValues(pos), vdata) + } else { // otherwise just store the new value + newMask.set(pos) + newValues(pos) = vdata + } } } new VertexPartition[VD2](index, newValues, newMask) diff --git a/graph/src/test/scala/org/apache/spark/graph/impl/VertexPartitionSuite.scala b/graph/src/test/scala/org/apache/spark/graph/impl/VertexPartitionSuite.scala new file mode 100644 index 0000000000000..72579a48c2885 --- /dev/null +++ b/graph/src/test/scala/org/apache/spark/graph/impl/VertexPartitionSuite.scala @@ -0,0 +1,113 @@ +package org.apache.spark.graph.impl + +import org.apache.spark.graph._ +import org.scalatest.FunSuite + +class VertexPartitionSuite extends FunSuite { + + test("isDefined, filter") { + val vp = VertexPartition(Iterator((0L, 1), (1L, 1))).filter { (vid, attr) => vid == 0 } + assert(vp.isDefined(0)) + assert(!vp.isDefined(1)) + assert(!vp.isDefined(2)) + assert(!vp.isDefined(-1)) + } + + test("isActive, numActives, replaceActives") { + val vp = VertexPartition(Iterator((0L, 1), (1L, 1))) + .filter { (vid, attr) => vid == 0 } + .replaceActives(Iterator(0, 2, 0)) + assert(vp.isActive(0)) + assert(!vp.isActive(1)) + assert(vp.isActive(2)) + assert(!vp.isActive(-1)) + assert(vp.numActives == Some(2)) + } + + test("map") { + val vp = VertexPartition(Iterator((0L, 1), (1L, 1))).map { (vid, attr) => 2 } + assert(vp(0) === 2) + } + + test("diff") { + val vp = VertexPartition(Iterator((0L, 1), (1L, 1), (2L, 1))) + val vp2 = vp.filter { (vid, attr) => vid <= 1 } + val vp3a = vp.map { (vid, attr) => 2 } + val vp3b = VertexPartition(vp3a.iterator) + // diff with same index + val diff1 = vp2.diff(vp3a) + assert(diff1(0) === 2) + assert(diff1(1) === 2) + assert(diff1(2) === 2) + assert(!diff1.isDefined(2)) + // diff with different indexes + val diff2 = vp2.diff(vp3b) + assert(diff2(0) === 2) + assert(diff2(1) === 2) + assert(diff2(2) === 2) + assert(!diff2.isDefined(2)) + } + + test("leftJoin") { + val vp = VertexPartition(Iterator((0L, 1), (1L, 1), (2L, 1))) + val vp2a = vp.filter { (vid, attr) => vid <= 1 }.map { (vid, attr) => 2 } + val vp2b = VertexPartition(vp2a.iterator) + // leftJoin with same index + val join1 = vp.leftJoin(vp2a) { (vid, a, bOpt) => bOpt.getOrElse(a) } + assert(join1.iterator.toSet === Set((0L, 2), (1L, 2), (2L, 1))) + // leftJoin with different indexes + val join2 = vp.leftJoin(vp2b) { (vid, a, bOpt) => bOpt.getOrElse(a) } + assert(join2.iterator.toSet === Set((0L, 2), (1L, 2), (2L, 1))) + // leftJoin an iterator + val join3 = vp.leftJoin(vp2a.iterator) { (vid, a, bOpt) => bOpt.getOrElse(a) } + assert(join3.iterator.toSet === Set((0L, 2), (1L, 2), (2L, 1))) + } + + test("innerJoin") { + val vp = VertexPartition(Iterator((0L, 1), (1L, 1), (2L, 1))) + val vp2a = vp.filter { (vid, attr) => vid <= 1 }.map { (vid, attr) => 2 } + val vp2b = VertexPartition(vp2a.iterator) + // innerJoin with same index + val join1 = vp.innerJoin(vp2a) { (vid, a, b) => b } + assert(join1.iterator.toSet === Set((0L, 2), (1L, 2))) + // innerJoin with different indexes + val join2 = vp.innerJoin(vp2b) { (vid, a, b) => b } + assert(join2.iterator.toSet === Set((0L, 2), (1L, 2))) + // innerJoin an iterator + val join3 = vp.innerJoin(vp2a.iterator) { (vid, a, b) => b } + assert(join3.iterator.toSet === Set((0L, 2), (1L, 2))) + } + + test("createUsingIndex") { + val vp = VertexPartition(Iterator((0L, 1), (1L, 1), (2L, 1))) + val elems = List((0L, 2), (2L, 2), (3L, 2)) + val vp2 = vp.createUsingIndex(elems.iterator) + assert(vp2.iterator.toSet === Set((0L, 2), (2L, 2))) + assert(vp.index === vp2.index) + } + + test("innerJoinKeepLeft") { + val vp = VertexPartition(Iterator((0L, 1), (1L, 1), (2L, 1))) + val elems = List((0L, 2), (2L, 2), (3L, 2)) + val vp2 = vp.innerJoinKeepLeft(elems.iterator) + assert(vp2.iterator.toSet === Set((0L, 2), (2L, 2))) + assert(vp2(1) === 1) + } + + test("aggregateUsingIndex") { + val vp = VertexPartition(Iterator((0L, 1), (1L, 1), (2L, 1))) + val messages = List((0L, "a"), (2L, "b"), (0L, "c"), (3L, "d")) + val vp2 = vp.aggregateUsingIndex[String](messages.iterator, _ + _) + assert(vp2.iterator.toSet === Set((0L, "ac"), (2L, "b"))) + } + + test("reindex") { + val vp = VertexPartition(Iterator((0L, 1), (1L, 1), (2L, 1))) + val vp2 = vp.filter { (vid, attr) => vid <= 1 } + val vp3 = vp2.reindex() + assert(vp2.iterator.toSet === vp3.iterator.toSet) + assert(vp2(2) === 1) + assert(vp3.index.getPos(2) === -1) + } + +} From 6bb077cd3de5ce959576ac21b0ae917452802cbc Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 20 Dec 2013 13:36:14 -0800 Subject: [PATCH 324/531] Reuse VTableReplicated in GraphImpl.subgraph --- .../apache/spark/graph/impl/EdgePartition.scala | 10 ++++++++++ .../org/apache/spark/graph/impl/GraphImpl.scala | 14 +++++++------- .../spark/graph/impl/EdgePartitionSuite.scala | 11 +++++++++++ 3 files changed, 28 insertions(+), 7 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala b/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala index e97522feaeaf2..3430ffdfc48c9 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala @@ -56,6 +56,16 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) new EdgePartition(srcIds, dstIds, newData, index) } + def filter(pred: Edge[ED] => Boolean): EdgePartition[ED] = { + val builder = new EdgePartitionBuilder[ED] + iterator.foreach { e => + if (pred(e)) { + builder.add(e.srcId, e.dstId, e.attr) + } + } + builder.toEdgePartition + } + /** * Apply the function f to all edges in this partition. * diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index e7f975253a6c9..9e44f49113de0 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -196,14 +196,12 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( override def subgraph( epred: EdgeTriplet[VD, ED] => Boolean = x => true, vpred: (Vid, VD) => Boolean = (a, b) => true): Graph[VD, ED] = { + // Filter the vertices, reusing the partitioner and the index from this graph + val newVerts = vertices.mapVertexPartitions(_.filter(vpred)) - // Filter the vertices, reusing the partitioner (but not the index) from - // this graph - val newVTable = vertices.mapVertexPartitions(_.filter(vpred).reindex()) - + // Filter the edges val edManifest = classManifest[ED] - - val newETable = new EdgeRDD[ED](triplets.filter { et => + val newEdges = new EdgeRDD[ED](triplets.filter { et => vpred(et.srcId, et.srcAttr) && vpred(et.dstId, et.dstAttr) && epred(et) }.mapPartitionsWithIndex( { (pid, iter) => val builder = new EdgePartitionBuilder[ED]()(edManifest) @@ -212,7 +210,9 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( Iterator((pid, edgePartition)) }, preservesPartitioning = true)).cache() - new GraphImpl(newVTable, newETable) + // Reuse the previous VTableReplicated unmodified. It will contain extra vertices, which is + // fine. + new GraphImpl(newVerts, newEdges, new VertexPlacement(newEdges, newVerts), vTableReplicated) } // end of subgraph override def mask[VD2: ClassManifest, ED2: ClassManifest] ( diff --git a/graph/src/test/scala/org/apache/spark/graph/impl/EdgePartitionSuite.scala b/graph/src/test/scala/org/apache/spark/graph/impl/EdgePartitionSuite.scala index a52a5653e2cf6..2991533e89f15 100644 --- a/graph/src/test/scala/org/apache/spark/graph/impl/EdgePartitionSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/impl/EdgePartitionSuite.scala @@ -31,6 +31,17 @@ class EdgePartitionSuite extends FunSuite { edges.map(e => e.copy(attr = e.srcId + e.dstId))) } + test("filter") { + val edges = List(Edge(0, 1, 0), Edge(1, 2, 0), Edge(2, 0, 0)) + val builder = new EdgePartitionBuilder[Int] + for (e <- edges) { + builder.add(e.srcId, e.dstId, e.attr) + } + val edgePartition = builder.toEdgePartition + assert(edgePartition.filter(e => e.srcId <= 1).iterator.map(_.copy()).toList === + edges.filter(e => e.srcId <= 1)) + } + test("groupEdges") { val edges = List( Edge(0, 1, 1), Edge(1, 2, 2), Edge(2, 0, 4), Edge(0, 1, 8), Edge(1, 2, 16), Edge(2, 0, 32)) From cd01539d6f5231f80877add68b9f02147dd616cf Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 20 Dec 2013 13:37:14 -0800 Subject: [PATCH 325/531] Remove unused EdgePartition.filter --- .../org/apache/spark/graph/impl/EdgePartition.scala | 10 ---------- .../apache/spark/graph/impl/EdgePartitionSuite.scala | 11 ----------- 2 files changed, 21 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala b/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala index 3430ffdfc48c9..e97522feaeaf2 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala @@ -56,16 +56,6 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) new EdgePartition(srcIds, dstIds, newData, index) } - def filter(pred: Edge[ED] => Boolean): EdgePartition[ED] = { - val builder = new EdgePartitionBuilder[ED] - iterator.foreach { e => - if (pred(e)) { - builder.add(e.srcId, e.dstId, e.attr) - } - } - builder.toEdgePartition - } - /** * Apply the function f to all edges in this partition. * diff --git a/graph/src/test/scala/org/apache/spark/graph/impl/EdgePartitionSuite.scala b/graph/src/test/scala/org/apache/spark/graph/impl/EdgePartitionSuite.scala index 2991533e89f15..a52a5653e2cf6 100644 --- a/graph/src/test/scala/org/apache/spark/graph/impl/EdgePartitionSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/impl/EdgePartitionSuite.scala @@ -31,17 +31,6 @@ class EdgePartitionSuite extends FunSuite { edges.map(e => e.copy(attr = e.srcId + e.dstId))) } - test("filter") { - val edges = List(Edge(0, 1, 0), Edge(1, 2, 0), Edge(2, 0, 0)) - val builder = new EdgePartitionBuilder[Int] - for (e <- edges) { - builder.add(e.srcId, e.dstId, e.attr) - } - val edgePartition = builder.toEdgePartition - assert(edgePartition.filter(e => e.srcId <= 1).iterator.map(_.copy()).toList === - edges.filter(e => e.srcId <= 1)) - } - test("groupEdges") { val edges = List( Edge(0, 1, 1), Edge(1, 2, 2), Edge(2, 0, 4), Edge(0, 1, 8), Edge(1, 2, 16), Edge(2, 0, 32)) From 32f957f3317bd62768b415da0c0cd9114f59782c Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 20 Dec 2013 15:37:33 -0800 Subject: [PATCH 326/531] Create VertexPlacement on demand in VTableReplicated --- .../apache/spark/graph/impl/GraphImpl.scala | 59 ++++++----------- .../spark/graph/impl/VTableReplicated.scala | 65 +++++++++++++++++-- .../spark/graph/impl/VertexPlacement.scala | 64 ------------------ 3 files changed, 79 insertions(+), 109 deletions(-) delete mode 100644 graph/src/main/scala/org/apache/spark/graph/impl/VertexPlacement.scala diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index 9e44f49113de0..ebeebd4c65fab 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -30,21 +30,13 @@ import org.apache.spark.util.ClosureCleaner class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( @transient val vertices: VertexRDD[VD], @transient val edges: EdgeRDD[ED], - @transient val vertexPlacement: VertexPlacement, @transient val vTableReplicated: VTableReplicated[VD]) extends Graph[VD, ED] { - def this( - vertices: VertexRDD[VD], - edges: EdgeRDD[ED], - vertexPlacement: VertexPlacement) = { - this(vertices, edges, vertexPlacement, new VTableReplicated(vertices, edges, vertexPlacement)) - } - def this( vertices: VertexRDD[VD], edges: EdgeRDD[ED]) = { - this(vertices, edges, new VertexPlacement(edges, vertices)) + this(vertices, edges, new VTableReplicated(vertices, edges)) } /** Return a RDD that brings edges together with their source and destination vertices. */ @@ -89,16 +81,8 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( } 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[Vid]]]): Double = { - vid2pids.map(_.map(_.size).sum.toLong).reduce(_ + _).toDouble - } - val numVertices = this.ops.numVertices val numEdges = this.ops.numEdges - val replicationRatioBoth = numReplicatedVertices(vertexPlacement.bothAttrs) / numVertices - val replicationRatioSrcOnly = numReplicatedVertices(vertexPlacement.srcAttrOnly) / numVertices - val replicationRatioDstOnly = numReplicatedVertices(vertexPlacement.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 @@ -106,9 +90,6 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( 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) @@ -145,16 +126,13 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( println("\n\nvTable ------------------------------------------") traverseLineage(vertices, " ", visited) visited += (vertices.id -> "vTable") - println("\n\nvertexPlacement.bothAttrs -------------------------------") - traverseLineage(vertexPlacement.bothAttrs, " ", visited) - visited += (vertexPlacement.bothAttrs.id -> "vertexPlacement.bothAttrs") println("\n\ntriplets ----------------------------------------") traverseLineage(triplets, " ", visited) println(visited) } // end of printLineage override def reverse: Graph[VD, ED] = - new GraphImpl(vertices, edges.mapEdgePartitions(_.reverse), vertexPlacement, vTableReplicated) + new GraphImpl(vertices, edges.mapEdgePartitions(_.reverse), vTableReplicated) override def mapVertices[VD2: ClassManifest](f: (Vid, VD) => VD2): Graph[VD2, ED] = { if (classManifest[VD] equals classManifest[VD2]) { @@ -162,17 +140,16 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( val newVerts = vertices.mapVertexPartitions(_.map(f)) val changedVerts = vertices.asInstanceOf[VertexRDD[VD2]].diff(newVerts) val newVTableReplicated = new VTableReplicated[VD2]( - changedVerts, edges, vertexPlacement, - Some(vTableReplicated.asInstanceOf[VTableReplicated[VD2]])) - new GraphImpl(newVerts, edges, vertexPlacement, newVTableReplicated) + changedVerts, edges, Some(vTableReplicated.asInstanceOf[VTableReplicated[VD2]])) + new GraphImpl(newVerts, edges, newVTableReplicated) } else { // The map does not preserve type, so we must re-replicate all vertices - new GraphImpl(vertices.mapVertexPartitions(_.map(f)), edges, vertexPlacement) + new GraphImpl(vertices.mapVertexPartitions(_.map(f)), edges) } } override def mapEdges[ED2: ClassManifest](f: Edge[ED] => ED2): Graph[VD, ED2] = - new GraphImpl(vertices, edges.mapEdgePartitions(_.map(f)), vertexPlacement, vTableReplicated) + new GraphImpl(vertices, edges.mapEdgePartitions(_.map(f)), vTableReplicated) override def mapTriplets[ED2: ClassManifest](f: EdgeTriplet[VD, ED] => ED2): Graph[VD, ED2] = { // Use an explicit manifest in PrimitiveKeyOpenHashMap init so we don't pull in the implicit @@ -190,7 +167,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( } Iterator((pid, newEdgePartition)) } - new GraphImpl(vertices, new EdgeRDD(newETable), vertexPlacement, vTableReplicated) + new GraphImpl(vertices, new EdgeRDD(newETable), vTableReplicated) } override def subgraph( @@ -210,23 +187,26 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( Iterator((pid, edgePartition)) }, preservesPartitioning = true)).cache() - // Reuse the previous VTableReplicated unmodified. It will contain extra vertices, which is - // fine. - new GraphImpl(newVerts, newEdges, new VertexPlacement(newEdges, newVerts), vTableReplicated) + // Reuse the previous VTableReplicated unmodified. The replicated vertices that have been + // removed will be ignored, since we only refer to replicated vertices when they are adjacent to + // an edge. + new GraphImpl(newVerts, newEdges, vTableReplicated) } // end of subgraph override def mask[VD2: ClassManifest, ED2: ClassManifest] ( other: Graph[VD2, ED2]): Graph[VD, ED] = { val newVerts = vertices.innerJoin(other.vertices) { (vid, v, w) => v } val newEdges = edges.innerJoin(other.edges) { (src, dst, v, w) => v } - new GraphImpl(newVerts, newEdges) - + // Reuse the previous VTableReplicated unmodified. The replicated vertices that have been + // removed will be ignored, since we only refer to replicated vertices when they are adjacent to + // an edge. + new GraphImpl(newVerts, newEdges, vTableReplicated) } override def groupEdges(merge: (ED, ED) => ED): Graph[VD, ED] = { ClosureCleaner.clean(merge) val newETable = edges.mapEdgePartitions(_.groupEdges(merge)) - new GraphImpl(vertices, newETable, vertexPlacement, vTableReplicated) + new GraphImpl(vertices, newETable, vTableReplicated) } ////////////////////////////////////////////////////////////////////////////////////////////////// @@ -304,13 +284,12 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( val newVerts = vertices.leftJoin(updates)(updateF) val changedVerts = vertices.asInstanceOf[VertexRDD[VD2]].diff(newVerts) val newVTableReplicated = new VTableReplicated[VD2]( - changedVerts, edges, vertexPlacement, - Some(vTableReplicated.asInstanceOf[VTableReplicated[VD2]])) - new GraphImpl(newVerts, edges, vertexPlacement, newVTableReplicated) + changedVerts, edges, Some(vTableReplicated.asInstanceOf[VTableReplicated[VD2]])) + new GraphImpl(newVerts, edges, newVTableReplicated) } else { // updateF does not preserve type, so we must re-replicate all vertices val newVerts = vertices.leftJoin(updates)(updateF) - new GraphImpl(newVerts, edges, vertexPlacement) + new GraphImpl(newVerts, edges) } } diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala b/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala index b9b2a4705bbe9..fc708da3d915c 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala @@ -1,11 +1,13 @@ package org.apache.spark.graph.impl +import org.apache.spark.Partitioner +import scala.collection.mutable + import org.apache.spark.SparkContext._ +import org.apache.spark.graph._ import org.apache.spark.rdd.RDD import org.apache.spark.util.collection.{PrimitiveVector, OpenHashSet} -import org.apache.spark.graph._ - /** * A view of the vertices after they are shipped to the join sites specified in * `vertexPlacement`. The resulting view is co-partitioned with `edges`. If `prevVTableReplicated` @@ -20,7 +22,6 @@ private[impl] class VTableReplicated[VD: ClassManifest]( updatedVerts: VertexRDD[VD], edges: EdgeRDD[_], - vertexPlacement: VertexPlacement, prevVTableReplicated: Option[VTableReplicated[VD]] = None) { /** @@ -49,6 +50,9 @@ class VTableReplicated[VD: ClassManifest]( private lazy val dstAttrOnly: RDD[(Pid, VertexPartition[VD])] = create(false, true) private lazy val noAttrs: RDD[(Pid, VertexPartition[VD])] = create(false, false) + private val routingTables: mutable.Map[(Boolean, Boolean), RDD[Array[Array[Vid]]]] = + new mutable.HashMap[(Boolean, Boolean), RDD[Array[Array[Vid]]]] + def get(includeSrc: Boolean, includeDst: Boolean): RDD[(Pid, VertexPartition[VD])] = { (includeSrc, includeDst) match { case (true, true) => bothAttrs @@ -67,7 +71,7 @@ class VTableReplicated[VD: ClassManifest]( // includeDst. These flags govern attribute shipping, but the activeness of a vertex must be // shipped to all edges mentioning that vertex, regardless of whether the vertex attribute is // also shipped there. - val shippedActives = vertexPlacement.get(true, true) + val shippedActives = getRoutingTable(true, true) .zipPartitions(actives.partitionsRDD)(VTableReplicated.buildActiveBuffer(_, _)) .partitionBy(edges.partitioner.get) // Update vTableReplicated with shippedActives, setting activeness flags in the resulting @@ -85,7 +89,7 @@ class VTableReplicated[VD: ClassManifest]( // Ship vertex attributes to edge partitions according to vertexPlacement val verts = updatedVerts.partitionsRDD - val shippedVerts = vertexPlacement.get(includeSrc, includeDst) + val shippedVerts = getRoutingTable(includeSrc, includeDst) .zipPartitions(verts)(VTableReplicated.buildBuffer(_, _)(vdManifest)) .partitionBy(edges.partitioner.get) // TODO: Consider using a specialized shuffler. @@ -125,6 +129,19 @@ class VTableReplicated[VD: ClassManifest]( }.cache().setName("VTableReplicated %s %s".format(includeSrc, includeDst)) } } + + /** + * Returns an RDD with the locations of edge-partition join sites for each vertex attribute in + * `vertices`; that is, the routing information for shipping vertex attributes to edge + * partitions. The routing information is stored as a compressed bitmap for each vertex partition. + */ + private def getRoutingTable( + includeSrc: Boolean, includeDst: Boolean): RDD[Array[Array[Vid]]] = { + routingTables.getOrElseUpdate( + (includeSrc, includeDst), + VTableReplicated.createRoutingTable( + edges, updatedVerts.partitioner.get, includeSrc, includeDst)) + } } object VTableReplicated { @@ -174,6 +191,44 @@ object VTableReplicated { (pid, actives.trim().array) } } + + private def createRoutingTable( + edges: EdgeRDD[_], + vertexPartitioner: Partitioner, + includeSrc: Boolean, + includeDst: Boolean): RDD[Array[Array[Vid]]] = { + // Determine which vertices each edge partition needs by creating a mapping from vid to pid. + val vid2pid: RDD[(Vid, Pid)] = edges.partitionsRDD.mapPartitions { iter => + val (pid: Pid, edgePartition: EdgePartition[_]) = iter.next() + val numEdges = edgePartition.size + val vSet = new VertexSet + if (includeSrc) { // Add src vertices to the set. + var i = 0 + while (i < numEdges) { + vSet.add(edgePartition.srcIds(i)) + i += 1 + } + } + if (includeDst) { // Add dst vertices to the set. + var i = 0 + while (i < numEdges) { + vSet.add(edgePartition.dstIds(i)) + i += 1 + } + } + vSet.iterator.map { vid => (vid, pid) } + } + + val numPartitions = vertexPartitioner.numPartitions + vid2pid.partitionBy(vertexPartitioner).mapPartitions { iter => + val pid2vid = Array.fill(numPartitions)(new PrimitiveVector[Vid]) + for ((vid, pid) <- iter) { + pid2vid(pid) += vid + } + + Iterator(pid2vid.map(_.trim().array)) + }.cache().setName("VertexPlacement %s %s".format(includeSrc, includeDst)) + } } class VertexAttributeBlock[VD: ClassManifest](val vids: Array[Vid], val attrs: Array[VD]) { diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPlacement.scala b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPlacement.scala deleted file mode 100644 index 44a0a05f74c62..0000000000000 --- a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPlacement.scala +++ /dev/null @@ -1,64 +0,0 @@ -package org.apache.spark.graph.impl - -import org.apache.spark.SparkContext._ -import org.apache.spark.graph._ -import org.apache.spark.rdd.RDD -import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.collection.PrimitiveVector - -/** - * Stores the locations of edge-partition join sites for each vertex attribute in `vTable`; that is, - * the routing information for shipping vertex attributes to edge partitions. This is always cached - * because it may be used multiple times in VTableReplicated -- once to ship the vertex attributes - * and (possibly) once to ship the active-set information. - */ -class VertexPlacement(eTable: EdgeRDD[_], vTable: VertexRDD[_]) { - - val bothAttrs: RDD[Array[Array[Vid]]] = createPid2Vid(true, true) - val srcAttrOnly: RDD[Array[Array[Vid]]] = createPid2Vid(true, false) - val dstAttrOnly: RDD[Array[Array[Vid]]] = createPid2Vid(false, true) - val noAttrs: RDD[Array[Array[Vid]]] = createPid2Vid(false, false) - - def get(includeSrcAttr: Boolean, includeDstAttr: Boolean): RDD[Array[Array[Vid]]] = - (includeSrcAttr, includeDstAttr) match { - case (true, true) => bothAttrs - case (true, false) => srcAttrOnly - case (false, true) => dstAttrOnly - case (false, false) => noAttrs - } - - private def createPid2Vid( - includeSrcAttr: Boolean, includeDstAttr: Boolean): RDD[Array[Array[Vid]]] = { - // Determine which vertices each edge partition needs by creating a mapping from vid to pid. - val vid2pid: RDD[(Vid, Pid)] = eTable.partitionsRDD.mapPartitions { iter => - val (pid: Pid, edgePartition: EdgePartition[_]) = iter.next() - val numEdges = edgePartition.size - val vSet = new VertexSet - if (includeSrcAttr) { // Add src vertices to the set. - var i = 0 - while (i < numEdges) { - vSet.add(edgePartition.srcIds(i)) - i += 1 - } - } - if (includeDstAttr) { // Add dst vertices to the set. - var i = 0 - while (i < numEdges) { - vSet.add(edgePartition.dstIds(i)) - i += 1 - } - } - vSet.iterator.map { vid => (vid, pid) } - } - - val numPartitions = vTable.partitions.size - vid2pid.partitionBy(vTable.partitioner.get).mapPartitions { iter => - val pid2vid = Array.fill(numPartitions)(new PrimitiveVector[Vid]) - for ((vid, pid) <- iter) { - pid2vid(pid) += vid - } - - Iterator(pid2vid.map(_.trim().array)) - }.cache().setName("VertexPlacement %s %s".format(includeSrcAttr, includeDstAttr)) - } -} From 17311359c44a99541e16489a8842f3013919e832 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 20 Dec 2013 15:58:40 -0800 Subject: [PATCH 327/531] Rename VTableReplicated -> ReplicatedVertexView --- .../apache/spark/graph/impl/GraphImpl.scala | 114 ++++++++---------- ...cated.scala => ReplicatedVertexView.scala} | 51 ++++---- 2 files changed, 77 insertions(+), 88 deletions(-) rename graph/src/main/scala/org/apache/spark/graph/impl/{VTableReplicated.scala => ReplicatedVertexView.scala} (84%) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index ebeebd4c65fab..e7c4b5db82603 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -16,27 +16,22 @@ import org.apache.spark.util.ClosureCleaner * A Graph RDD that supports computation on graphs. * * Graphs are represented using two classes of data: vertex-partitioned and - * edge-partitioned. `vTable` contains vertex attributes, which are - * vertex-partitioned. `eTable` contains edge attributes, which are - * edge-partitioned. For operations on vertex neighborhoods, vertex attributes - * are replicated to the edge partitions where they appear as sources or - * destinations. `vertexPlacement` specifies where each vertex will be - * replicated. `vTableReplicated` stores the replicated vertex attributes, which - * are co-partitioned with the relevant edges. - * - * mask in vertices means filter - * mask in vTableReplicated means active + * edge-partitioned. `vertices` contains vertex attributes, which are vertex-partitioned. `edges` + * contains edge attributes, which are edge-partitioned. For operations on vertex neighborhoods, + * vertex attributes are replicated to the edge partitions where they appear as sources or + * destinations. `replicatedVertexView` stores a view of the replicated vertex attributes, which are + * co-partitioned with the relevant edges. */ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( @transient val vertices: VertexRDD[VD], @transient val edges: EdgeRDD[ED], - @transient val vTableReplicated: VTableReplicated[VD]) + @transient val replicatedVertexView: ReplicatedVertexView[VD]) extends Graph[VD, ED] { def this( vertices: VertexRDD[VD], edges: EdgeRDD[ED]) = { - this(vertices, edges, new VTableReplicated(vertices, edges)) + this(vertices, edges, new ReplicatedVertexView(vertices, edges)) } /** Return a RDD that brings edges together with their source and destination vertices. */ @@ -44,7 +39,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( val vdManifest = classManifest[VD] val edManifest = classManifest[ED] - edges.zipEdgePartitions(vTableReplicated.get(true, true)) { (ePart, vPartIter) => + edges.zipEdgePartitions(replicatedVertexView.get(true, true)) { (ePart, vPartIter) => val (_, vPart) = vPartIter.next() new EdgeTripletIterator(vPart.index, vPart.values, ePart)(vdManifest, edManifest) } @@ -120,28 +115,28 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( deps.foreach(d => traverseLineage(d.rdd, indent + " | ", visited)) } } - println("eTable ------------------------------------------") + println("edges ------------------------------------------") traverseLineage(edges, " ") - var visited = Map(edges.id -> "eTable") - println("\n\nvTable ------------------------------------------") + var visited = Map(edges.id -> "edges") + println("\n\nvertices ------------------------------------------") traverseLineage(vertices, " ", visited) - visited += (vertices.id -> "vTable") + visited += (vertices.id -> "vertices") println("\n\ntriplets ----------------------------------------") traverseLineage(triplets, " ", visited) println(visited) } // end of printLineage override def reverse: Graph[VD, ED] = - new GraphImpl(vertices, edges.mapEdgePartitions(_.reverse), vTableReplicated) + new GraphImpl(vertices, edges.mapEdgePartitions(_.reverse), replicatedVertexView) override def mapVertices[VD2: ClassManifest](f: (Vid, VD) => VD2): Graph[VD2, ED] = { if (classManifest[VD] equals classManifest[VD2]) { // The map preserves type, so we can use incremental replication val newVerts = vertices.mapVertexPartitions(_.map(f)) val changedVerts = vertices.asInstanceOf[VertexRDD[VD2]].diff(newVerts) - val newVTableReplicated = new VTableReplicated[VD2]( - changedVerts, edges, Some(vTableReplicated.asInstanceOf[VTableReplicated[VD2]])) - new GraphImpl(newVerts, edges, newVTableReplicated) + val newReplicatedVertexView = new ReplicatedVertexView[VD2]( + changedVerts, edges, Some(replicatedVertexView.asInstanceOf[ReplicatedVertexView[VD2]])) + new GraphImpl(newVerts, edges, newReplicatedVertexView) } else { // The map does not preserve type, so we must re-replicate all vertices new GraphImpl(vertices.mapVertexPartitions(_.map(f)), edges) @@ -149,15 +144,15 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( } override def mapEdges[ED2: ClassManifest](f: Edge[ED] => ED2): Graph[VD, ED2] = - new GraphImpl(vertices, edges.mapEdgePartitions(_.map(f)), vTableReplicated) + new GraphImpl(vertices, edges.mapEdgePartitions(_.map(f)), replicatedVertexView) override def mapTriplets[ED2: ClassManifest](f: EdgeTriplet[VD, ED] => ED2): Graph[VD, ED2] = { // Use an explicit manifest in PrimitiveKeyOpenHashMap init so we don't pull in the implicit // manifest from GraphImpl (which would require serializing GraphImpl). val vdManifest = classManifest[VD] - val newETable = - edges.zipEdgePartitions(vTableReplicated.get(true, true)) { (edgePartition, vTableReplicatedIter) => - val (pid, vPart) = vTableReplicatedIter.next() + val newEdgePartitions = + edges.zipEdgePartitions(replicatedVertexView.get(true, true)) { (edgePartition, vPartIter) => + val (pid, vPart) = vPartIter.next() val et = new EdgeTriplet[VD, ED] val newEdgePartition = edgePartition.map { e => et.set(e) @@ -167,7 +162,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( } Iterator((pid, newEdgePartition)) } - new GraphImpl(vertices, new EdgeRDD(newETable), vTableReplicated) + new GraphImpl(vertices, new EdgeRDD(newEdgePartitions), replicatedVertexView) } override def subgraph( @@ -187,26 +182,26 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( Iterator((pid, edgePartition)) }, preservesPartitioning = true)).cache() - // Reuse the previous VTableReplicated unmodified. The replicated vertices that have been + // Reuse the previous ReplicatedVertexView unmodified. The replicated vertices that have been // removed will be ignored, since we only refer to replicated vertices when they are adjacent to // an edge. - new GraphImpl(newVerts, newEdges, vTableReplicated) + new GraphImpl(newVerts, newEdges, replicatedVertexView) } // end of subgraph override def mask[VD2: ClassManifest, ED2: ClassManifest] ( other: Graph[VD2, ED2]): Graph[VD, ED] = { val newVerts = vertices.innerJoin(other.vertices) { (vid, v, w) => v } val newEdges = edges.innerJoin(other.edges) { (src, dst, v, w) => v } - // Reuse the previous VTableReplicated unmodified. The replicated vertices that have been + // Reuse the previous ReplicatedVertexView unmodified. The replicated vertices that have been // removed will be ignored, since we only refer to replicated vertices when they are adjacent to // an edge. - new GraphImpl(newVerts, newEdges, vTableReplicated) + new GraphImpl(newVerts, newEdges, replicatedVertexView) } override def groupEdges(merge: (ED, ED) => ED): Graph[VD, ED] = { ClosureCleaner.clean(merge) - val newETable = edges.mapEdgePartitions(_.groupEdges(merge)) - new GraphImpl(vertices, newETable, vTableReplicated) + val newEdges = edges.mapEdgePartitions(_.groupEdges(merge)) + new GraphImpl(vertices, newEdges, replicatedVertexView) } ////////////////////////////////////////////////////////////////////////////////////////////////// @@ -226,14 +221,16 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( val mapUsesSrcAttr = accessesVertexAttr[VD, ED](mapFunc, "srcAttr") val mapUsesDstAttr = accessesVertexAttr[VD, ED](mapFunc, "dstAttr") val vs = activeSetOpt match { - case Some((activeSet, _)) => vTableReplicated.get(mapUsesSrcAttr, mapUsesDstAttr, activeSet) - case None => vTableReplicated.get(mapUsesSrcAttr, mapUsesDstAttr) + case Some((activeSet, _)) => + replicatedVertexView.get(mapUsesSrcAttr, mapUsesDstAttr, activeSet) + case None => + replicatedVertexView.get(mapUsesSrcAttr, mapUsesDstAttr) } val activeDirectionOpt = activeSetOpt.map(_._2) // Map and combine. - val preAgg = edges.zipEdgePartitions(vs) { (edgePartition, vTableReplicatedIter) => - val (_, vPart) = vTableReplicatedIter.next() + val preAgg = edges.zipEdgePartitions(vs) { (edgePartition, vPartIter) => + val (_, vPart) = vPartIter.next() // Choose scan method val activeFraction = vPart.numActives.getOrElse(0) / edgePartition.indexSize.toFloat @@ -283,9 +280,9 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( // updateF preserves type, so we can use incremental replication val newVerts = vertices.leftJoin(updates)(updateF) val changedVerts = vertices.asInstanceOf[VertexRDD[VD2]].diff(newVerts) - val newVTableReplicated = new VTableReplicated[VD2]( - changedVerts, edges, Some(vTableReplicated.asInstanceOf[VTableReplicated[VD2]])) - new GraphImpl(newVerts, edges, newVTableReplicated) + val newReplicatedVertexView = new ReplicatedVertexView[VD2]( + changedVerts, edges, Some(replicatedVertexView.asInstanceOf[ReplicatedVertexView[VD2]])) + new GraphImpl(newVerts, edges, newReplicatedVertexView) } else { // updateF does not preserve type, so we must re-replicate all vertices val newVerts = vertices.leftJoin(updates)(updateF) @@ -309,13 +306,13 @@ object GraphImpl { edges: RDD[Edge[ED]], defaultVertexAttr: VD): GraphImpl[VD, ED] = { - fromEdgeRDD(createETable(edges), defaultVertexAttr) + fromEdgeRDD(createEdgeRDD(edges), defaultVertexAttr) } def fromEdgePartitions[VD: ClassManifest, ED: ClassManifest]( - edges: RDD[(Pid, EdgePartition[ED])], + edgePartitions: RDD[(Pid, EdgePartition[ED])], defaultVertexAttr: VD): GraphImpl[VD, ED] = { - fromEdgeRDD(createETableFromEdgePartitions(edges), defaultVertexAttr) + fromEdgeRDD(new EdgeRDD(edgePartitions), defaultVertexAttr) } def apply[VD: ClassManifest, ED: ClassManifest]( @@ -323,44 +320,39 @@ object GraphImpl { edges: RDD[Edge[ED]], defaultVertexAttr: VD): GraphImpl[VD, ED] = { - val etable = createETable(edges).cache() + val edgeRDD = createEdgeRDD(edges).cache() // Get the set of all vids val partitioner = Partitioner.defaultPartitioner(vertices) val vPartitioned = vertices.partitionBy(partitioner) - val vidsFromEdges = collectVidsFromEdges(etable, partitioner) + val vidsFromEdges = collectVidsFromEdges(edgeRDD, partitioner) val vids = vPartitioned.zipPartitions(vidsFromEdges) { (vertexIter, vidsFromEdgesIter) => vertexIter.map(_._1) ++ vidsFromEdgesIter.map(_._1) } - val vtable = VertexRDD(vids, vPartitioned, defaultVertexAttr) + val vertexRDD = VertexRDD(vids, vPartitioned, defaultVertexAttr) - new GraphImpl(vtable, etable) + new GraphImpl(vertexRDD, edgeRDD) } /** - * Create the edge table RDD, which is much more efficient for Java heap storage than the - * normal edges data structure (RDD[(Vid, Vid, ED)]). + * Create the edge RDD, which is much more efficient for Java heap storage than the normal edges + * data structure (RDD[(Vid, Vid, ED)]). * - * The edge table contains multiple partitions, and each partition contains only one RDD - * key-value pair: the key is the partition id, and the value is an EdgePartition object - * containing all the edges in a partition. + * The edge RDD contains multiple partitions, and each partition contains only one RDD key-value + * pair: the key is the partition id, and the value is an EdgePartition object containing all the + * edges in a partition. */ - private def createETable[ED: ClassManifest]( + private def createEdgeRDD[ED: ClassManifest]( edges: RDD[Edge[ED]]): EdgeRDD[ED] = { - val eTable = edges.mapPartitionsWithIndex { (pid, iter) => + val edgePartitions = edges.mapPartitionsWithIndex { (pid, iter) => val builder = new EdgePartitionBuilder[ED] iter.foreach { e => builder.add(e.srcId, e.dstId, e.attr) } Iterator((pid, builder.toEdgePartition)) } - new EdgeRDD(eTable) - } - - private def createETableFromEdgePartitions[ED: ClassManifest]( - edges: RDD[(Pid, EdgePartition[ED])]): EdgeRDD[ED] = { - new EdgeRDD(edges) + new EdgeRDD(edgePartitions) } private def fromEdgeRDD[VD: ClassManifest, ED: ClassManifest]( @@ -370,8 +362,8 @@ object GraphImpl { // Get the set of all vids val vids = collectVidsFromEdges(edges, new HashPartitioner(edges.partitions.size)) // Create the VertexRDD. - val vtable = VertexRDD(vids.mapValues(x => defaultVertexAttr)) - new GraphImpl(vtable, edges) + val vertices = VertexRDD(vids.mapValues(x => defaultVertexAttr)) + new GraphImpl(vertices, edges) } /** Collects all vids mentioned in edges and partitions them by partitioner. */ diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala b/graph/src/main/scala/org/apache/spark/graph/impl/ReplicatedVertexView.scala similarity index 84% rename from graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala rename to graph/src/main/scala/org/apache/spark/graph/impl/ReplicatedVertexView.scala index fc708da3d915c..bef99810bd089 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/VTableReplicated.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/ReplicatedVertexView.scala @@ -10,19 +10,19 @@ import org.apache.spark.util.collection.{PrimitiveVector, OpenHashSet} /** * A view of the vertices after they are shipped to the join sites specified in - * `vertexPlacement`. The resulting view is co-partitioned with `edges`. If `prevVTableReplicated` - * is specified, `updatedVerts` are treated as incremental updates to the previous view. Otherwise, - * a fresh view is created. + * `vertexPlacement`. The resulting view is co-partitioned with `edges`. If `prevViewOpt` is + * specified, `updatedVerts` are treated as incremental updates to the previous view. Otherwise, a + * fresh view is created. * * The view is always cached (i.e., once it is created, it remains materialized). This avoids * constructing it twice if the user calls graph.triplets followed by graph.mapReduceTriplets, for * example. */ private[impl] -class VTableReplicated[VD: ClassManifest]( +class ReplicatedVertexView[VD: ClassManifest]( updatedVerts: VertexRDD[VD], edges: EdgeRDD[_], - prevVTableReplicated: Option[VTableReplicated[VD]] = None) { + prevViewOpt: Option[ReplicatedVertexView[VD]] = None) { /** * Within each edge partition, create a local map from vid to an index into the attribute @@ -30,9 +30,9 @@ class VTableReplicated[VD: ClassManifest]( * vids from both the source and destination of edges. It must always include both source and * destination vids because some operations, such as GraphImpl.mapReduceTriplets, rely on this. */ - private val localVidMap: RDD[(Int, VertexIdToIndexMap)] = prevVTableReplicated match { - case Some(prev) => - prev.localVidMap + private val localVidMap: RDD[(Int, VertexIdToIndexMap)] = prevViewOpt match { + case Some(prevView) => + prevView.localVidMap case None => edges.partitionsRDD.mapPartitions(_.map { case (pid, epart) => @@ -42,7 +42,7 @@ class VTableReplicated[VD: ClassManifest]( vidToIndex.add(e.dstId) } (pid, vidToIndex) - }, preservesPartitioning = true).cache().setName("VTableReplicated localVidMap") + }, preservesPartitioning = true).cache().setName("ReplicatedVertexView localVidMap") } private lazy val bothAttrs: RDD[(Pid, VertexPartition[VD])] = create(true, true) @@ -66,15 +66,14 @@ class VTableReplicated[VD: ClassManifest]( includeSrc: Boolean, includeDst: Boolean, actives: VertexRDD[_]): RDD[(Pid, VertexPartition[VD])] = { - // Ship active sets to edge partitions using vertexPlacement, but ignoring includeSrc and // includeDst. These flags govern attribute shipping, but the activeness of a vertex must be // shipped to all edges mentioning that vertex, regardless of whether the vertex attribute is // also shipped there. val shippedActives = getRoutingTable(true, true) - .zipPartitions(actives.partitionsRDD)(VTableReplicated.buildActiveBuffer(_, _)) + .zipPartitions(actives.partitionsRDD)(ReplicatedVertexView.buildActiveBuffer(_, _)) .partitionBy(edges.partitioner.get) - // Update vTableReplicated with shippedActives, setting activeness flags in the resulting + // Update the view with shippedActives, setting activeness flags in the resulting // VertexPartitions get(includeSrc, includeDst).zipPartitions(shippedActives) { (viewIter, shippedActivesIter) => val (pid, vPart) = viewIter.next() @@ -90,22 +89,20 @@ class VTableReplicated[VD: ClassManifest]( // Ship vertex attributes to edge partitions according to vertexPlacement val verts = updatedVerts.partitionsRDD val shippedVerts = getRoutingTable(includeSrc, includeDst) - .zipPartitions(verts)(VTableReplicated.buildBuffer(_, _)(vdManifest)) + .zipPartitions(verts)(ReplicatedVertexView.buildBuffer(_, _)(vdManifest)) .partitionBy(edges.partitioner.get) // TODO: Consider using a specialized shuffler. - prevVTableReplicated match { - case Some(vTableReplicated) => - val prevView: RDD[(Pid, VertexPartition[VD])] = - vTableReplicated.get(includeSrc, includeDst) - - // Update vTableReplicated with shippedVerts, setting staleness flags in the resulting + prevViewOpt match { + case Some(prevView) => + // Update prevView with shippedVerts, setting staleness flags in the resulting // VertexPartitions - prevView.zipPartitions(shippedVerts) { (prevViewIter, shippedVertsIter) => - val (pid, prevVPart) = prevViewIter.next() - val newVPart = prevVPart.innerJoinKeepLeft(shippedVertsIter.flatMap(_._2.iterator)) - Iterator((pid, newVPart)) - }.cache().setName("VTableReplicated delta %s %s".format(includeSrc, includeDst)) + prevView.get(includeSrc, includeDst).zipPartitions(shippedVerts) { + (prevViewIter, shippedVertsIter) => + val (pid, prevVPart) = prevViewIter.next() + val newVPart = prevVPart.innerJoinKeepLeft(shippedVertsIter.flatMap(_._2.iterator)) + Iterator((pid, newVPart)) + }.cache().setName("ReplicatedVertexView delta %s %s".format(includeSrc, includeDst)) case None => // Within each edge partition, place the shipped vertex attributes into the correct @@ -126,7 +123,7 @@ class VTableReplicated[VD: ClassManifest]( val newVPart = new VertexPartition( vidToIndex, vertexArray, vidToIndex.getBitSet)(vdManifest) Iterator((pid, newVPart)) - }.cache().setName("VTableReplicated %s %s".format(includeSrc, includeDst)) + }.cache().setName("ReplicatedVertexView %s %s".format(includeSrc, includeDst)) } } @@ -139,12 +136,12 @@ class VTableReplicated[VD: ClassManifest]( includeSrc: Boolean, includeDst: Boolean): RDD[Array[Array[Vid]]] = { routingTables.getOrElseUpdate( (includeSrc, includeDst), - VTableReplicated.createRoutingTable( + ReplicatedVertexView.createRoutingTable( edges, updatedVerts.partitioner.get, includeSrc, includeDst)) } } -object VTableReplicated { +object ReplicatedVertexView { protected def buildBuffer[VD: ClassManifest]( pid2vidIter: Iterator[Array[Array[Vid]]], vertexPartIter: Iterator[VertexPartition[VD]]) = { From 47eefd30edfb9c13d53729b743bd1a29935977cd Mon Sep 17 00:00:00 2001 From: Wang Jianping J Date: Sat, 21 Dec 2013 13:02:59 +0800 Subject: [PATCH 328/531] add javadoc --- .../apache/spark/graph/algorithms/Svdpp.scala | 38 +++++++++++-------- 1 file changed, 23 insertions(+), 15 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala b/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala index ffd0ddba7ee8c..28c717bfcbeb5 100644 --- a/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala +++ b/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala @@ -20,8 +20,15 @@ class Msg ( // message ) extends Serializable object Svdpp { - // implement SVD++ based on http://public.research.att.com/~volinsky/netflix/kdd08koren.pdf - // model (15) on page 6 + /** + * 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. + * + * @param edges edges for constructing the graph + * + * @return a graph with vertex attributes containing the trained model + */ def run(edges: RDD[Edge[Double]]): Graph[VT, Double] = { // defalut parameters @@ -33,7 +40,8 @@ object Svdpp { val gamma2 = 0.007 val gamma6 = 0.005 val gamma7 = 0.015 - + + // generate default vertex attribute def defaultF(rank: Int) = { val v1 = new ArrayRealVector(rank) val v2 = new ArrayRealVector(rank) @@ -70,8 +78,8 @@ object Svdpp { // calculate initial norm and bias val t0 = g.mapReduceTriplets(mapF0, reduceF0) - g.outerJoinVertices(t0) {updateF0} - + g.outerJoinVertices(t0) {updateF0} + // phase 1 def mapF1(et: EdgeTriplet[VT, Double]): Iterator[(Vid, RealVector)] = { assert(et.srcAttr != null && et.dstAttr != null) @@ -98,10 +106,10 @@ object Svdpp { pred = math.max(pred, minVal) pred = math.min(pred, maxVal) val err = et.attr - pred - val y = (q.mapMultiply(err*usr.norm)).subtract((itm.v2).mapMultiply(gamma7)) - val newP = (q.mapMultiply(err)).subtract(p.mapMultiply(gamma7)) // for each connected item q - val newQ = (usr.v2.mapMultiply(err)).subtract(q.mapMultiply(gamma7)) - Iterator((et.srcId, new Msg(newP, y, err - gamma6*usr.bias)), (et.dstId, new Msg(newQ, y, err - gamma6*itm.bias))) + val updateY = (q.mapMultiply(err*usr.norm)).subtract((itm.v2).mapMultiply(gamma7)) + val updateP = (q.mapMultiply(err)).subtract(p.mapMultiply(gamma7)) + val updateQ = (usr.v2.mapMultiply(err)).subtract(q.mapMultiply(gamma7)) + Iterator((et.srcId, new Msg(updateP, updateY, err - gamma6*usr.bias)), (et.dstId, new Msg(updateQ, updateY, err - gamma6*itm.bias))) } def reduceF2(g1: Msg, g2: Msg):Msg = { g1.v1 = g1.v1.add(g2.v1) @@ -112,7 +120,7 @@ object Svdpp { def updateF2(vid: Vid, vd: VT, msg: Option[Msg]) = { if (msg.isDefined) { vd.v1 = vd.v1.add(msg.get.v1.mapMultiply(gamma2)) - if (vid % 2 == 1) { // item node update y + if (vid % 2 == 1) { // item nodes update y vd.v2 = vd.v2.add(msg.get.v2.mapMultiply(gamma2)) } vd.bias += msg.get.bias*gamma1 @@ -121,14 +129,14 @@ object Svdpp { } for (i <- 0 until maxIters) { - // phase 1 + // phase 1, calculate v2 for user nodes val t1: VertexRDD[RealVector] = g.mapReduceTriplets(mapF1, reduceF1) - g.outerJoinVertices(t1) {updateF1} - // phase 2 + g.outerJoinVertices(t1) {updateF1} + // phase 2, update p for user nodes and q, y for item nodes val t2: VertexRDD[Msg] = g.mapReduceTriplets(mapF2, reduceF2) g.outerJoinVertices(t2) {updateF2} } - + // calculate error on training set def mapF3(et: EdgeTriplet[VT, Double]): Iterator[(Vid, Double)] = { assert(et.srcAttr != null && et.dstAttr != null) @@ -143,7 +151,7 @@ object Svdpp { Iterator((et.dstId, err)) } def updateF3(vid: Vid, vd: VT, msg: Option[Double]) = { - if (msg.isDefined && vid % 2 == 1) { // item sum up the errors + if (msg.isDefined && vid % 2 == 1) { // item nodes sum up the errors vd.norm = msg.get } vd From f986e4a13662724f0ff8a31a46133616aa2ca1e0 Mon Sep 17 00:00:00 2001 From: Wang Jianping J Date: Sat, 21 Dec 2013 13:18:11 +0800 Subject: [PATCH 329/531] Update Svdpp.scala --- .../main/scala/org/apache/spark/graph/algorithms/Svdpp.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala b/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala index 28c717bfcbeb5..fe0093c4a41ac 100644 --- a/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala +++ b/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala @@ -10,7 +10,7 @@ class VT ( // vertex type var v1: RealVector, // v1: p for user node, q for item node var v2: RealVector, // v2: pu + |N(u)|^(-0.5)*sum(y) for user node, y for item node var bias: Double, - var norm: Double // only for user node + var norm: Double // |N(u)|^(-0.5) for user node ) extends Serializable class Msg ( // message From 49eb0f1351fc73bcb44fffbd30d955a95bd88fbe Mon Sep 17 00:00:00 2001 From: Wang Jianping J Date: Sat, 21 Dec 2013 15:51:05 +0800 Subject: [PATCH 330/531] Update Svdpp.scala --- .../org/apache/spark/graph/algorithms/Svdpp.scala | 14 +++++--------- 1 file changed, 5 insertions(+), 9 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala b/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala index fe0093c4a41ac..ac20f150724a5 100644 --- a/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala +++ b/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala @@ -98,17 +98,15 @@ object Svdpp { // phase 2 def mapF2(et: EdgeTriplet[VT, Double]): Iterator[(Vid, Msg)] = { assert(et.srcAttr != null && et.dstAttr != null) - val usr = et.srcAttr - val itm = et.dstAttr - val p = usr.v1 - val q = itm.v1 + val (usr, itm) = (et.srcAttr, et.dstAttr) + val (p, q) = (usr.v1, itm.v1) var pred = u + usr.bias + itm.bias + q.dotProduct(usr.v2) pred = math.max(pred, minVal) pred = math.min(pred, maxVal) val err = et.attr - pred - val updateY = (q.mapMultiply(err*usr.norm)).subtract((itm.v2).mapMultiply(gamma7)) val updateP = (q.mapMultiply(err)).subtract(p.mapMultiply(gamma7)) val updateQ = (usr.v2.mapMultiply(err)).subtract(q.mapMultiply(gamma7)) + val updateY = (q.mapMultiply(err*usr.norm)).subtract((itm.v2).mapMultiply(gamma7)) Iterator((et.srcId, new Msg(updateP, updateY, err - gamma6*usr.bias)), (et.dstId, new Msg(updateQ, updateY, err - gamma6*itm.bias))) } def reduceF2(g1: Msg, g2: Msg):Msg = { @@ -140,10 +138,8 @@ object Svdpp { // calculate error on training set def mapF3(et: EdgeTriplet[VT, Double]): Iterator[(Vid, Double)] = { assert(et.srcAttr != null && et.dstAttr != null) - val usr = et.srcAttr - val itm = et.dstAttr - val p = usr.v1 - val q = itm.v1 + val (usr, item) = (et.srcAttr, et.dstAttr) + val (p, q) = (usr.v1, itm.v1) var pred = u + usr.bias + itm.bias + q.dotProduct(usr.v2) pred = math.max(pred, minVal) pred = math.min(pred, maxVal) From b4bce88a2d881d6514b6beb37e83c31afc01e53f Mon Sep 17 00:00:00 2001 From: Wang Jianping J Date: Sat, 21 Dec 2013 16:08:19 +0800 Subject: [PATCH 331/531] Update Svdpp.scala --- .../main/scala/org/apache/spark/graph/algorithms/Svdpp.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala b/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala index ac20f150724a5..74096eb80d5fe 100644 --- a/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala +++ b/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala @@ -138,7 +138,7 @@ object Svdpp { // calculate error on training set def mapF3(et: EdgeTriplet[VT, Double]): Iterator[(Vid, Double)] = { assert(et.srcAttr != null && et.dstAttr != null) - val (usr, item) = (et.srcAttr, et.dstAttr) + val (usr, itm) = (et.srcAttr, et.dstAttr) val (p, q) = (usr.v1, itm.v1) var pred = u + usr.bias + itm.bias + q.dotProduct(usr.v2) pred = math.max(pred, minVal) From e64a794a4417f614e1b74180a123f5f913a6db53 Mon Sep 17 00:00:00 2001 From: Wang Jianping J Date: Sun, 22 Dec 2013 18:20:49 +0800 Subject: [PATCH 332/531] Update Svdpp.scala --- .../org/apache/spark/graph/algorithms/Svdpp.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala b/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala index 74096eb80d5fe..26b999f4cffa4 100644 --- a/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala +++ b/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala @@ -1,6 +1,5 @@ package org.apache.spark.graph.algorithms -import org.apache.spark._ import org.apache.spark.rdd._ import org.apache.spark.graph._ import scala.util.Random @@ -53,7 +52,7 @@ object Svdpp { vd } - // calculate initial norm and bias + // calculate initial bias and norm def mapF0(et: EdgeTriplet[VT, Double]): Iterator[(Vid, (Long, Double))] = { assert(et.srcAttr != null && et.dstAttr != null) Iterator((et.srcId, (1L, et.attr)), (et.dstId, (1L, et.attr))) @@ -76,7 +75,7 @@ object Svdpp { // make graph var g = Graph.fromEdges(edges, defaultF(rank)).cache() - // calculate initial norm and bias + // calculate initial bias and norm val t0 = g.mapReduceTriplets(mapF0, reduceF0) g.outerJoinVertices(t0) {updateF0} @@ -107,8 +106,9 @@ object Svdpp { val updateP = (q.mapMultiply(err)).subtract(p.mapMultiply(gamma7)) val updateQ = (usr.v2.mapMultiply(err)).subtract(q.mapMultiply(gamma7)) val updateY = (q.mapMultiply(err*usr.norm)).subtract((itm.v2).mapMultiply(gamma7)) - Iterator((et.srcId, new Msg(updateP, updateY, err - gamma6*usr.bias)), (et.dstId, new Msg(updateQ, updateY, err - gamma6*itm.bias))) - } + Iterator((et.srcId, new Msg(updateP, updateY, err - gamma6*usr.bias)), + (et.dstId, new Msg(updateQ, updateY, err - gamma6*itm.bias))) + } def reduceF2(g1: Msg, g2: Msg):Msg = { g1.v1 = g1.v1.add(g2.v1) g1.v2 = g1.v2.add(g2.v2) From 29fe6bdaa29193c9dbf3a8fbd05094f3d812d4e5 Mon Sep 17 00:00:00 2001 From: Jianping J Wang Date: Mon, 30 Dec 2013 23:41:15 +0800 Subject: [PATCH 333/531] refactor and bug fix --- .../apache/spark/graph/algorithms/Svdpp.scala | 155 ++++++++---------- 1 file changed, 64 insertions(+), 91 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala b/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala index 26b999f4cffa4..cbbe240c90548 100644 --- a/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala +++ b/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala @@ -5,18 +5,27 @@ import org.apache.spark.graph._ import scala.util.Random import org.apache.commons.math.linear._ -class VT ( // vertex type +class VT( // vertex type var v1: RealVector, // v1: p for user node, q for item node var v2: RealVector, // v2: pu + |N(u)|^(-0.5)*sum(y) for user node, y for item node var bias: Double, var norm: Double // |N(u)|^(-0.5) for user node -) extends Serializable + ) extends Serializable -class Msg ( // message +class Msg( // message var v1: RealVector, var v2: RealVector, - var bias: Double -) extends Serializable + var bias: Double) extends Serializable + +class SvdppConf( // Svdpp parameters + 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 object Svdpp { /** @@ -24,21 +33,14 @@ object Svdpp { * 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. * - * @param edges edges for constructing the graph + * @param edges edges for constructing the graph + * + * @param conf Svdpp parameters * * @return a graph with vertex attributes containing the trained model */ - def run(edges: RDD[Edge[Double]]): Graph[VT, Double] = { - // defalut parameters - val rank = 10 - val maxIters = 20 - val minVal = 0.0 - val maxVal = 5.0 - val gamma1 = 0.007 - val gamma2 = 0.007 - val gamma6 = 0.005 - val gamma7 = 0.015 + def run(edges: RDD[Edge[Double]], conf: SvdppConf): Graph[VT, Double] = { // generate default vertex attribute def defaultF(rank: Int) = { @@ -52,108 +54,79 @@ object Svdpp { vd } - // calculate initial bias and norm - def mapF0(et: EdgeTriplet[VT, Double]): Iterator[(Vid, (Long, Double))] = { - assert(et.srcAttr != null && et.dstAttr != null) - Iterator((et.srcId, (1L, et.attr)), (et.dstId, (1L, et.attr))) - } - def reduceF0(g1: (Long, Double), g2: (Long, Double)) = { - (g1._1 + g2._1, g1._2 + g2._2) - } - def updateF0(vid: Vid, vd: VT, msg: Option[(Long, Double)]) = { - if (msg.isDefined) { - vd.bias = msg.get._2 / msg.get._1 - vd.norm = 1.0 / scala.math.sqrt(msg.get._1) - } - vd - } - // calculate global rating mean val (rs, rc) = edges.map(e => (e.attr, 1L)).reduce((a, b) => (a._1 + b._1, a._2 + b._2)) val u = rs / rc // global rating mean - // make graph - var g = Graph.fromEdges(edges, defaultF(rank)).cache() + // construct graph + var g = Graph.fromEdges(edges, defaultF(conf.rank)).cache() // calculate initial bias and norm - val t0 = g.mapReduceTriplets(mapF0, reduceF0) - g.outerJoinVertices(t0) {updateF0} - - // phase 1 - def mapF1(et: EdgeTriplet[VT, Double]): Iterator[(Vid, RealVector)] = { - assert(et.srcAttr != null && et.dstAttr != null) - Iterator((et.srcId, et.dstAttr.v2)) // sum up y of connected item nodes - } - def reduceF1(g1: RealVector, g2: RealVector) = { - g1.add(g2) - } - def updateF1(vid: Vid, vd: VT, msg: Option[RealVector]) = { - if (msg.isDefined) { - vd.v2 = vd.v1.add(msg.get.mapMultiply(vd.norm)) // pu + |N(u)|^(-0.5)*sum(y) - } - vd + var t0: VertexRDD[(Long, Double)] = 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: Vid, vd: VT, msg: Option[(Long, Double)]) => + vd.bias = msg.get._2 / msg.get._1; vd.norm = 1.0 / scala.math.sqrt(msg.get._1) + vd } - // phase 2 - def mapF2(et: EdgeTriplet[VT, Double]): Iterator[(Vid, Msg)] = { + def mapTrainF(conf: SvdppConf, u: Double)(et: EdgeTriplet[VT, Double]): Iterator[(Vid, Msg)] = { assert(et.srcAttr != null && et.dstAttr != null) val (usr, itm) = (et.srcAttr, et.dstAttr) val (p, q) = (usr.v1, itm.v1) var pred = u + usr.bias + itm.bias + q.dotProduct(usr.v2) - pred = math.max(pred, minVal) - pred = math.min(pred, maxVal) + 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(gamma7)) - val updateQ = (usr.v2.mapMultiply(err)).subtract(q.mapMultiply(gamma7)) - val updateY = (q.mapMultiply(err*usr.norm)).subtract((itm.v2).mapMultiply(gamma7)) - Iterator((et.srcId, new Msg(updateP, updateY, err - gamma6*usr.bias)), - (et.dstId, new Msg(updateQ, updateY, err - gamma6*itm.bias))) - } - def reduceF2(g1: Msg, g2: Msg):Msg = { - g1.v1 = g1.v1.add(g2.v1) - g1.v2 = g1.v2.add(g2.v2) - g1.bias += g2.bias - g1 - } - def updateF2(vid: Vid, vd: VT, msg: Option[Msg]) = { - if (msg.isDefined) { - vd.v1 = vd.v1.add(msg.get.v1.mapMultiply(gamma2)) - if (vid % 2 == 1) { // item nodes update y - vd.v2 = vd.v2.add(msg.get.v2.mapMultiply(gamma2)) - } - vd.bias += msg.get.bias*gamma1 - } - vd + val updateP = ((q.mapMultiply(err)).subtract(p.mapMultiply(conf.gamma7))).mapMultiply(conf.gamma2) + val updateQ = ((usr.v2.mapMultiply(err)).subtract(q.mapMultiply(conf.gamma7))).mapMultiply(conf.gamma2) + val updateY = ((q.mapMultiply(err * usr.norm)).subtract((itm.v2).mapMultiply(conf.gamma7))).mapMultiply(conf.gamma2) + Iterator((et.srcId, new Msg(updateP, updateY, (err - conf.gamma6 * usr.bias) * conf.gamma1)), + (et.dstId, new Msg(updateQ, updateY, (err - conf.gamma6 * itm.bias) * conf.gamma1))) } - for (i <- 0 until maxIters) { + for (i <- 0 until conf.maxIters) { // phase 1, calculate v2 for user nodes - val t1: VertexRDD[RealVector] = g.mapReduceTriplets(mapF1, reduceF1) - g.outerJoinVertices(t1) {updateF1} + var t1 = g.mapReduceTriplets(et => + Iterator((et.srcId, et.dstAttr.v2)), + (g1: RealVector, g2: RealVector) => g1.add(g2)) + g = g.outerJoinVertices(t1) { (vid: Vid, vd: VT, msg: Option[RealVector]) => + if (msg.isDefined) vd.v2 = vd.v1.add(msg.get.mapMultiply(vd.norm)) + vd + } // phase 2, update p for user nodes and q, y for item nodes - val t2: VertexRDD[Msg] = g.mapReduceTriplets(mapF2, reduceF2) - g.outerJoinVertices(t2) {updateF2} + val t2: VertexRDD[Msg] = g.mapReduceTriplets(mapTrainF(conf, u), (g1: Msg, g2: Msg) => { + g1.v1 = g1.v1.add(g2.v1) + g1.v2 = g1.v2.add(g2.v2) + g1.bias += g2.bias + g1 + }) + g = g.outerJoinVertices(t2) { (vid: Vid, vd: VT, msg: Option[Msg]) => + vd.v1 = vd.v1.add(msg.get.v1) + if (vid % 2 == 1) vd.v2 = vd.v2.add(msg.get.v2) + vd.bias += msg.get.bias + vd + } } // calculate error on training set - def mapF3(et: EdgeTriplet[VT, Double]): Iterator[(Vid, Double)] = { + def mapTestF(conf: SvdppConf, u: Double)(et: EdgeTriplet[VT, Double]): Iterator[(Vid, Double)] = { assert(et.srcAttr != null && et.dstAttr != null) val (usr, itm) = (et.srcAttr, et.dstAttr) val (p, q) = (usr.v1, itm.v1) var pred = u + usr.bias + itm.bias + q.dotProduct(usr.v2) - pred = math.max(pred, minVal) - pred = math.min(pred, maxVal) - val err = (et.attr - pred)*(et.attr - pred) + pred = math.max(pred, conf.minVal) + pred = math.min(pred, conf.maxVal) + val err = (et.attr - pred) * (et.attr - pred) Iterator((et.dstId, err)) } - def updateF3(vid: Vid, vd: VT, msg: Option[Double]) = { - if (msg.isDefined && vid % 2 == 1) { // item nodes sum up the errors - vd.norm = msg.get - } + val t3: VertexRDD[Double] = g.mapReduceTriplets(mapTestF(conf, u), _ + _) + g.outerJoinVertices(t3) { (vid: Vid, vd: VT, msg: Option[Double]) => + if (msg.isDefined && vid % 2 == 1) vd.norm = msg.get // item nodes sum up the errors vd } - val t3: VertexRDD[Double] = g.mapReduceTriplets(mapF3, _ + _) - g.outerJoinVertices(t3) {updateF3} - g + g } } From 600421d8bc258508d0e298131384d0abed4b87b7 Mon Sep 17 00:00:00 2001 From: Jianping J Wang Date: Mon, 30 Dec 2013 23:42:55 +0800 Subject: [PATCH 334/531] update --- .../scala/org/apache/spark/graph/algorithms/SvdppSuite.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/graph/src/test/scala/org/apache/spark/graph/algorithms/SvdppSuite.scala b/graph/src/test/scala/org/apache/spark/graph/algorithms/SvdppSuite.scala index 4ea675c2dc1ba..46dee4e9a07f6 100644 --- a/graph/src/test/scala/org/apache/spark/graph/algorithms/SvdppSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/algorithms/SvdppSuite.scala @@ -13,12 +13,13 @@ class SvdppSuite extends FunSuite with LocalSparkContext { test("Test SVD++ with mean square error on training set") { withSpark { sc => - val SvdppErr = 0.01 + val SvdppErr = 0.2 val edges = sc.textFile("mllib/data/als/test.data").map { line => val fields = line.split(",") Edge(fields(0).toLong * 2, fields(1).toLong * 2 + 1, fields(2).toDouble) } - val graph = Svdpp.run(edges) + val conf = new SvdppConf(10, 2, 0.0, 5.0, 0.007, 0.007, 0.005, 0.015) + val graph = Svdpp.run(edges, conf) val err = graph.vertices.collect.map{ case (vid, vd) => if (vid % 2 == 1) { vd.norm } else { 0.0 } }.reduce(_ + _) / graph.triplets.collect.size From 779c66ae4ee681f9cf8ab85cd48f4761ee49e031 Mon Sep 17 00:00:00 2001 From: Jianping J Wang Date: Tue, 31 Dec 2013 16:59:05 +0800 Subject: [PATCH 335/531] refactor and fix bugs --- .../apache/spark/graph/algorithms/Svdpp.scala | 87 +++++++------------ 1 file changed, 29 insertions(+), 58 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala b/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala index cbbe240c90548..7c3e0c83c9829 100644 --- a/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala +++ b/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala @@ -5,18 +5,6 @@ import org.apache.spark.graph._ import scala.util.Random import org.apache.commons.math.linear._ -class VT( // vertex type - var v1: RealVector, // v1: p for user node, q for item node - var v2: RealVector, // v2: pu + |N(u)|^(-0.5)*sum(y) for user node, y for item node - var bias: Double, - var norm: Double // |N(u)|^(-0.5) for user node - ) extends Serializable - -class Msg( // message - var v1: RealVector, - var v2: RealVector, - var bias: Double) extends Serializable - class SvdppConf( // Svdpp parameters var rank: Int, var maxIters: Int, @@ -40,92 +28,75 @@ object Svdpp { * @return a graph with vertex attributes containing the trained model */ - def run(edges: RDD[Edge[Double]], conf: SvdppConf): Graph[VT, Double] = { + def run(edges: RDD[Edge[Double]], conf: SvdppConf): Graph[(RealVector, RealVector, Double, Double), Double] = { // generate default vertex attribute - def defaultF(rank: Int) = { + 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) } - var vd = new VT(v1, v2, 0.0, 0.0) - vd + (v1, v2, 0.0, 0.0) } // calculate global rating mean val (rs, rc) = edges.map(e => (e.attr, 1L)).reduce((a, b) => (a._1 + b._1, a._2 + b._2)) - val u = rs / rc // global rating mean + val u = rs / rc // construct graph var g = Graph.fromEdges(edges, defaultF(conf.rank)).cache() // calculate initial bias and norm - var t0: VertexRDD[(Long, Double)] = 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: Vid, vd: VT, msg: Option[(Long, Double)]) => - vd.bias = msg.get._2 / msg.get._1; vd.norm = 1.0 / scala.math.sqrt(msg.get._1) - vd + 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: Vid, 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: SvdppConf, u: Double)(et: EdgeTriplet[VT, Double]): Iterator[(Vid, Msg)] = { - assert(et.srcAttr != null && et.dstAttr != null) + def mapTrainF(conf: SvdppConf, u: Double)(et: EdgeTriplet[(RealVector, RealVector, Double, Double), Double]) + : Iterator[(Vid, (RealVector, RealVector, Double))] = { val (usr, itm) = (et.srcAttr, et.dstAttr) - val (p, q) = (usr.v1, itm.v1) - var pred = u + usr.bias + itm.bias + q.dotProduct(usr.v2) + val (p, q) = (usr._1, itm._1) + var pred = u + usr._3 + itm._3 + q.dotProduct(usr._2) 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.v2.mapMultiply(err)).subtract(q.mapMultiply(conf.gamma7))).mapMultiply(conf.gamma2) - val updateY = ((q.mapMultiply(err * usr.norm)).subtract((itm.v2).mapMultiply(conf.gamma7))).mapMultiply(conf.gamma2) - Iterator((et.srcId, new Msg(updateP, updateY, (err - conf.gamma6 * usr.bias) * conf.gamma1)), - (et.dstId, new Msg(updateQ, updateY, (err - conf.gamma6 * itm.bias) * conf.gamma1))) + 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 v2 for user nodes - var t1 = g.mapReduceTriplets(et => - Iterator((et.srcId, et.dstAttr.v2)), - (g1: RealVector, g2: RealVector) => g1.add(g2)) - g = g.outerJoinVertices(t1) { (vid: Vid, vd: VT, msg: Option[RealVector]) => - if (msg.isDefined) vd.v2 = vd.v1.add(msg.get.mapMultiply(vd.norm)) - vd + var t1 = g.mapReduceTriplets(et => Iterator((et.srcId, et.dstAttr._2)), (g1: RealVector, g2: RealVector) => g1.add(g2)) + g = g.outerJoinVertices(t1) { (vid: Vid, 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 - val t2: VertexRDD[Msg] = g.mapReduceTriplets(mapTrainF(conf, u), (g1: Msg, g2: Msg) => { - g1.v1 = g1.v1.add(g2.v1) - g1.v2 = g1.v2.add(g2.v2) - g1.bias += g2.bias - g1 - }) - g = g.outerJoinVertices(t2) { (vid: Vid, vd: VT, msg: Option[Msg]) => - vd.v1 = vd.v1.add(msg.get.v1) - if (vid % 2 == 1) vd.v2 = vd.v2.add(msg.get.v2) - vd.bias += msg.get.bias - vd + 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: Vid, 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: SvdppConf, u: Double)(et: EdgeTriplet[VT, Double]): Iterator[(Vid, Double)] = { - assert(et.srcAttr != null && et.dstAttr != null) + def mapTestF(conf: SvdppConf, u: Double)(et: EdgeTriplet[(RealVector, RealVector, Double, Double), Double]): Iterator[(Vid, Double)] = { val (usr, itm) = (et.srcAttr, et.dstAttr) - val (p, q) = (usr.v1, itm.v1) - var pred = u + usr.bias + itm.bias + q.dotProduct(usr.v2) + val (p, q) = (usr._1, itm._1) + var pred = u + usr._3 + itm._3 + q.dotProduct(usr._2) pred = math.max(pred, conf.minVal) pred = math.min(pred, conf.maxVal) val err = (et.attr - pred) * (et.attr - pred) Iterator((et.dstId, err)) } - val t3: VertexRDD[Double] = g.mapReduceTriplets(mapTestF(conf, u), _ + _) - g.outerJoinVertices(t3) { (vid: Vid, vd: VT, msg: Option[Double]) => - if (msg.isDefined && vid % 2 == 1) vd.norm = msg.get // item nodes sum up the errors - vd + val t3 = g.mapReduceTriplets(mapTestF(conf, u), (g1: Double, g2: Double) => g1 + g2) + g.outerJoinVertices(t3) { (vid: Vid, vd: (RealVector, RealVector, Double, Double), msg: Option[Double]) => + if (msg.isDefined && vid % 2 == 1) (vd._1, vd._2, vd._3, msg.get) else vd } g } From 4a30f69b25239d0e7a4a27478be261e17c1dba1a Mon Sep 17 00:00:00 2001 From: Jianping J Wang Date: Tue, 31 Dec 2013 17:00:59 +0800 Subject: [PATCH 336/531] update svdpp test --- .../scala/org/apache/spark/graph/algorithms/SvdppSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/graph/src/test/scala/org/apache/spark/graph/algorithms/SvdppSuite.scala b/graph/src/test/scala/org/apache/spark/graph/algorithms/SvdppSuite.scala index 46dee4e9a07f6..c02a2d83986e6 100644 --- a/graph/src/test/scala/org/apache/spark/graph/algorithms/SvdppSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/algorithms/SvdppSuite.scala @@ -21,7 +21,7 @@ class SvdppSuite extends FunSuite with LocalSparkContext { val conf = new SvdppConf(10, 2, 0.0, 5.0, 0.007, 0.007, 0.005, 0.015) val graph = Svdpp.run(edges, conf) val err = graph.vertices.collect.map{ case (vid, vd) => - if (vid % 2 == 1) { vd.norm } else { 0.0 } + if (vid % 2 == 1) { vd._4 } else { 0.0 } }.reduce(_ + _) / graph.triplets.collect.size assert(err < SvdppErr) } From ab7b8ce13ec3a0a85f57ceede02df1e5eb9f6c23 Mon Sep 17 00:00:00 2001 From: Jianping J Wang Date: Tue, 31 Dec 2013 17:11:20 +0800 Subject: [PATCH 337/531] Update Svdpp.scala --- .../main/scala/org/apache/spark/graph/algorithms/Svdpp.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala b/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala index 7c3e0c83c9829..cb1a69e3185c9 100644 --- a/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala +++ b/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala @@ -71,7 +71,7 @@ object Svdpp { } for (i <- 0 until conf.maxIters) { - // phase 1, calculate v2 for user nodes + // phase 1, calculate pu + |N(u)|^(-0.5)*sum(y) for user nodes var t1 = g.mapReduceTriplets(et => Iterator((et.srcId, et.dstAttr._2)), (g1: RealVector, g2: RealVector) => g1.add(g2)) g = g.outerJoinVertices(t1) { (vid: Vid, 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 From 12c26d7fb912f07a9e4a9a3de532e0c53bc667a9 Mon Sep 17 00:00:00 2001 From: Jianping J Wang Date: Tue, 31 Dec 2013 17:18:15 +0800 Subject: [PATCH 338/531] Update Svdpp.scala --- .../main/scala/org/apache/spark/graph/algorithms/Svdpp.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala b/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala index cb1a69e3185c9..e7e8d00d9a505 100644 --- a/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala +++ b/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala @@ -96,7 +96,7 @@ object Svdpp { } val t3 = g.mapReduceTriplets(mapTestF(conf, u), (g1: Double, g2: Double) => g1 + g2) g.outerJoinVertices(t3) { (vid: Vid, vd: (RealVector, RealVector, Double, Double), msg: Option[Double]) => - if (msg.isDefined && vid % 2 == 1) (vd._1, vd._2, vd._3, msg.get) else vd + if (msg.isDefined) (vd._1, vd._2, vd._3, msg.get) else vd } g } From 61e6671f5abbbd0a96cc7359ea6302b84e6e9248 Mon Sep 17 00:00:00 2001 From: Jianping J Wang Date: Tue, 31 Dec 2013 22:01:02 +0800 Subject: [PATCH 339/531] fix test bug --- .../scala/org/apache/spark/graph/algorithms/Svdpp.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala b/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala index e7e8d00d9a505..18395bdc5f230 100644 --- a/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala +++ b/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala @@ -28,7 +28,7 @@ object Svdpp { * @return a graph with vertex attributes containing the trained model */ - def run(edges: RDD[Edge[Double]], conf: SvdppConf): Graph[(RealVector, RealVector, Double, Double), Double] = { + def run(edges: RDD[Edge[Double]], conf: SvdppConf): (Graph[(RealVector, RealVector, Double, Double), Double], Double) = { // generate default vertex attribute def defaultF(rank: Int): (RealVector, RealVector, Double, Double) = { @@ -95,9 +95,9 @@ object Svdpp { Iterator((et.dstId, err)) } val t3 = g.mapReduceTriplets(mapTestF(conf, u), (g1: Double, g2: Double) => g1 + g2) - g.outerJoinVertices(t3) { (vid: Vid, vd: (RealVector, RealVector, Double, Double), msg: Option[Double]) => + g = g.outerJoinVertices(t3) { (vid: Vid, vd: (RealVector, RealVector, Double, Double), msg: Option[Double]) => if (msg.isDefined) (vd._1, vd._2, vd._3, msg.get) else vd } - g + (g, u) } } From 6e50df6255dbe25c880dced3d15a77241eba803d Mon Sep 17 00:00:00 2001 From: Jianping J Wang Date: Tue, 31 Dec 2013 22:02:16 +0800 Subject: [PATCH 340/531] Update SvdppSuite.scala --- .../org/apache/spark/graph/algorithms/SvdppSuite.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/graph/src/test/scala/org/apache/spark/graph/algorithms/SvdppSuite.scala b/graph/src/test/scala/org/apache/spark/graph/algorithms/SvdppSuite.scala index c02a2d83986e6..411dd3d336c2a 100644 --- a/graph/src/test/scala/org/apache/spark/graph/algorithms/SvdppSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/algorithms/SvdppSuite.scala @@ -13,17 +13,17 @@ class SvdppSuite extends FunSuite with LocalSparkContext { test("Test SVD++ with mean square error on training set") { withSpark { sc => - val SvdppErr = 0.2 + val SvdppErr = 8.0 val edges = sc.textFile("mllib/data/als/test.data").map { line => val fields = line.split(",") Edge(fields(0).toLong * 2, fields(1).toLong * 2 + 1, fields(2).toDouble) } - val conf = new SvdppConf(10, 2, 0.0, 5.0, 0.007, 0.007, 0.005, 0.015) - val graph = Svdpp.run(edges, conf) + val conf = new SvdppConf(10, 2, 0.0, 5.0, 0.007, 0.007, 0.005, 0.015) // 2 iterations + var (graph, u) = Svdpp.run(edges, conf) val err = graph.vertices.collect.map{ case (vid, vd) => - if (vid % 2 == 1) { vd._4 } else { 0.0 } + if (vid % 2 == 1) vd._4 else 0.0 }.reduce(_ + _) / graph.triplets.collect.size - assert(err < SvdppErr) + assert(err <= SvdppErr) } } From 3d93d7339626bec62d514cd524e45084a22715ae Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Tue, 31 Dec 2013 15:19:01 -0800 Subject: [PATCH 341/531] Fixing the persist behavior of the Vertex and Edge RDDs to persist the current RDD and not the parent. --- graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala | 2 +- graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala b/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala index 63858db2efdd9..b1640bf9ceb02 100644 --- a/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala @@ -33,7 +33,7 @@ class EdgeRDD[@specialized ED: ClassManifest]( * Caching a VertexRDD causes the index and values to be cached separately. */ override def persist(newLevel: StorageLevel): EdgeRDD[ED] = { - partitionsRDD.persist(newLevel) + super.persist(newLevel) this } diff --git a/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala b/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala index fe0f0ae491277..671cf496f8a80 100644 --- a/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala @@ -86,7 +86,7 @@ class VertexRDD[@specialized VD: ClassManifest]( * Caching a VertexRDD causes the index and values to be cached separately. */ override def persist(newLevel: StorageLevel): VertexRDD[VD] = { - partitionsRDD.persist(newLevel) + super.persist(newLevel) this } From 2f2524fd11876a19e315f994020877564a8a0df8 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Tue, 31 Dec 2013 21:37:51 -0800 Subject: [PATCH 342/531] Addressing issue in compute where compute is invoked instead of iterator on the parent RDD. --- graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala | 8 ++++---- .../src/main/scala/org/apache/spark/graph/VertexRDD.scala | 4 ++-- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala b/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala index b1640bf9ceb02..9aa76c93945f9 100644 --- a/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala @@ -22,9 +22,8 @@ class EdgeRDD[@specialized ED: ClassManifest]( override val partitioner = partitionsRDD.partitioner.orElse(Some(Partitioner.defaultPartitioner(partitionsRDD))) - override def compute(split: Partition, context: TaskContext): Iterator[Edge[ED]] = { - val edgePartition = partitionsRDD.compute(split, context).next()._2 - edgePartition.iterator + override def compute(part: Partition, context: TaskContext): Iterator[Edge[ED]] = { + firstParent[(Pid, EdgePartition[ED])].iterator(part, context).next._2.iterator } override def collect(): Array[Edge[ED]] = this.map(_.copy()).collect() @@ -33,7 +32,7 @@ class EdgeRDD[@specialized ED: ClassManifest]( * Caching a VertexRDD causes the index and values to be cached separately. */ override def persist(newLevel: StorageLevel): EdgeRDD[ED] = { - super.persist(newLevel) + partitionsRDD.persist(newLevel) this } @@ -45,6 +44,7 @@ class EdgeRDD[@specialized ED: ClassManifest]( def mapEdgePartitions[ED2: ClassManifest](f: EdgePartition[ED] => EdgePartition[ED2]) : EdgeRDD[ED2] = { +// iter => iter.map { case (pid, ep) => (pid, f(ep)) } new EdgeRDD[ED2](partitionsRDD.mapPartitions({ iter => val (pid, ep) = iter.next() Iterator(Tuple2(pid, f(ep))) diff --git a/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala b/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala index 671cf496f8a80..c274e342c7566 100644 --- a/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala @@ -86,7 +86,7 @@ class VertexRDD[@specialized VD: ClassManifest]( * Caching a VertexRDD causes the index and values to be cached separately. */ override def persist(newLevel: StorageLevel): VertexRDD[VD] = { - super.persist(newLevel) + partitionsRDD.persist(newLevel) this } @@ -105,7 +105,7 @@ class VertexRDD[@specialized VD: ClassManifest]( * Provide the `RDD[(Vid, VD)]` equivalent output. */ override def compute(part: Partition, context: TaskContext): Iterator[(Vid, VD)] = { - partitionsRDD.compute(part, context).next().iterator + firstParent[VertexPartition[VD]].iterator(part, context).next.iterator } /** From cfab8f2062fad9ea400716afce28d200dd714c2b Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 3 Jan 2014 17:55:58 -0700 Subject: [PATCH 343/531] Revert "Create VertexPlacement on demand in VTableReplicated" This reverts commit 32f957f3317bd62768b415da0c0cd9114f59782c. Conflicts: graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala graph/src/main/scala/org/apache/spark/graph/impl/ReplicatedVertexView.scala --- .../apache/spark/graph/impl/GraphImpl.scala | 55 +++++++++++----- .../graph/impl/ReplicatedVertexView.scala | 65 ++----------------- .../spark/graph/impl/RoutingTable.scala | 64 ++++++++++++++++++ 3 files changed, 109 insertions(+), 75 deletions(-) create mode 100644 graph/src/main/scala/org/apache/spark/graph/impl/RoutingTable.scala diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index e7c4b5db82603..16d73820f07df 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -19,19 +19,28 @@ import org.apache.spark.util.ClosureCleaner * edge-partitioned. `vertices` contains vertex attributes, which are vertex-partitioned. `edges` * contains edge attributes, which are edge-partitioned. For operations on vertex neighborhoods, * vertex attributes are replicated to the edge partitions where they appear as sources or - * destinations. `replicatedVertexView` stores a view of the replicated vertex attributes, which are - * co-partitioned with the relevant edges. + * destinations. `routingTable` stores the routing information for shipping vertex attributes to + * edge partitions. `replicatedVertexView` stores a view of the replicated vertex attributes created + * using the routing table. */ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( @transient val vertices: VertexRDD[VD], @transient val edges: EdgeRDD[ED], + @transient val routingTable: RoutingTable, @transient val replicatedVertexView: ReplicatedVertexView[VD]) extends Graph[VD, ED] { + def this( + vertices: VertexRDD[VD], + edges: EdgeRDD[ED], + routingTable: RoutingTable) = { + this(vertices, edges, routingTable, new ReplicatedVertexView(vertices, edges, routingTable)) + } + def this( vertices: VertexRDD[VD], edges: EdgeRDD[ED]) = { - this(vertices, edges, new ReplicatedVertexView(vertices, edges)) + this(vertices, edges, new RoutingTable(edges, vertices)) } /** Return a RDD that brings edges together with their source and destination vertices. */ @@ -76,8 +85,16 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( } 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[Vid]]]): 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 @@ -85,6 +102,9 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( 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) @@ -121,13 +141,16 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( 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] = - new GraphImpl(vertices, edges.mapEdgePartitions(_.reverse), replicatedVertexView) + new GraphImpl(vertices, edges.mapEdgePartitions(_.reverse), routingTable, replicatedVertexView) override def mapVertices[VD2: ClassManifest](f: (Vid, VD) => VD2): Graph[VD2, ED] = { if (classManifest[VD] equals classManifest[VD2]) { @@ -135,16 +158,17 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( val newVerts = vertices.mapVertexPartitions(_.map(f)) val changedVerts = vertices.asInstanceOf[VertexRDD[VD2]].diff(newVerts) val newReplicatedVertexView = new ReplicatedVertexView[VD2]( - changedVerts, edges, Some(replicatedVertexView.asInstanceOf[ReplicatedVertexView[VD2]])) - new GraphImpl(newVerts, edges, newReplicatedVertexView) + changedVerts, edges, routingTable, + Some(replicatedVertexView.asInstanceOf[ReplicatedVertexView[VD2]])) + new GraphImpl(newVerts, edges, routingTable, newReplicatedVertexView) } else { // The map does not preserve type, so we must re-replicate all vertices - new GraphImpl(vertices.mapVertexPartitions(_.map(f)), edges) + new GraphImpl(vertices.mapVertexPartitions(_.map(f)), edges, routingTable) } } override def mapEdges[ED2: ClassManifest](f: Edge[ED] => ED2): Graph[VD, ED2] = - new GraphImpl(vertices, edges.mapEdgePartitions(_.map(f)), replicatedVertexView) + new GraphImpl(vertices, edges.mapEdgePartitions(_.map(f)), routingTable, replicatedVertexView) override def mapTriplets[ED2: ClassManifest](f: EdgeTriplet[VD, ED] => ED2): Graph[VD, ED2] = { // Use an explicit manifest in PrimitiveKeyOpenHashMap init so we don't pull in the implicit @@ -162,7 +186,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( } Iterator((pid, newEdgePartition)) } - new GraphImpl(vertices, new EdgeRDD(newEdgePartitions), replicatedVertexView) + new GraphImpl(vertices, new EdgeRDD(newEdgePartitions), routingTable, replicatedVertexView) } override def subgraph( @@ -185,7 +209,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( // Reuse the previous ReplicatedVertexView unmodified. The replicated vertices that have been // removed will be ignored, since we only refer to replicated vertices when they are adjacent to // an edge. - new GraphImpl(newVerts, newEdges, replicatedVertexView) + new GraphImpl(newVerts, newEdges, new RoutingTable(newEdges, newVerts), replicatedVertexView) } // end of subgraph override def mask[VD2: ClassManifest, ED2: ClassManifest] ( @@ -195,13 +219,13 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( // Reuse the previous ReplicatedVertexView unmodified. The replicated vertices that have been // removed will be ignored, since we only refer to replicated vertices when they are adjacent to // an edge. - new GraphImpl(newVerts, newEdges, replicatedVertexView) + new GraphImpl(newVerts, newEdges, routingTable, replicatedVertexView) } override def groupEdges(merge: (ED, ED) => ED): Graph[VD, ED] = { ClosureCleaner.clean(merge) val newEdges = edges.mapEdgePartitions(_.groupEdges(merge)) - new GraphImpl(vertices, newEdges, replicatedVertexView) + new GraphImpl(vertices, newEdges, routingTable, replicatedVertexView) } ////////////////////////////////////////////////////////////////////////////////////////////////// @@ -281,12 +305,13 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( val newVerts = vertices.leftJoin(updates)(updateF) val changedVerts = vertices.asInstanceOf[VertexRDD[VD2]].diff(newVerts) val newReplicatedVertexView = new ReplicatedVertexView[VD2]( - changedVerts, edges, Some(replicatedVertexView.asInstanceOf[ReplicatedVertexView[VD2]])) - new GraphImpl(newVerts, edges, newReplicatedVertexView) + changedVerts, edges, routingTable, + Some(replicatedVertexView.asInstanceOf[ReplicatedVertexView[VD2]])) + 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) - new GraphImpl(newVerts, edges) + new GraphImpl(newVerts, edges, routingTable) } } diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/ReplicatedVertexView.scala b/graph/src/main/scala/org/apache/spark/graph/impl/ReplicatedVertexView.scala index bef99810bd089..175586b87eb16 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/ReplicatedVertexView.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/ReplicatedVertexView.scala @@ -1,13 +1,11 @@ package org.apache.spark.graph.impl -import org.apache.spark.Partitioner -import scala.collection.mutable - import org.apache.spark.SparkContext._ -import org.apache.spark.graph._ import org.apache.spark.rdd.RDD import org.apache.spark.util.collection.{PrimitiveVector, OpenHashSet} +import org.apache.spark.graph._ + /** * A view of the vertices after they are shipped to the join sites specified in * `vertexPlacement`. The resulting view is co-partitioned with `edges`. If `prevViewOpt` is @@ -22,6 +20,7 @@ private[impl] class ReplicatedVertexView[VD: ClassManifest]( updatedVerts: VertexRDD[VD], edges: EdgeRDD[_], + routingTable: RoutingTable, prevViewOpt: Option[ReplicatedVertexView[VD]] = None) { /** @@ -50,9 +49,6 @@ class ReplicatedVertexView[VD: ClassManifest]( private lazy val dstAttrOnly: RDD[(Pid, VertexPartition[VD])] = create(false, true) private lazy val noAttrs: RDD[(Pid, VertexPartition[VD])] = create(false, false) - private val routingTables: mutable.Map[(Boolean, Boolean), RDD[Array[Array[Vid]]]] = - new mutable.HashMap[(Boolean, Boolean), RDD[Array[Array[Vid]]]] - def get(includeSrc: Boolean, includeDst: Boolean): RDD[(Pid, VertexPartition[VD])] = { (includeSrc, includeDst) match { case (true, true) => bothAttrs @@ -70,7 +66,7 @@ class ReplicatedVertexView[VD: ClassManifest]( // includeDst. These flags govern attribute shipping, but the activeness of a vertex must be // shipped to all edges mentioning that vertex, regardless of whether the vertex attribute is // also shipped there. - val shippedActives = getRoutingTable(true, true) + val shippedActives = routingTable.get(true, true) .zipPartitions(actives.partitionsRDD)(ReplicatedVertexView.buildActiveBuffer(_, _)) .partitionBy(edges.partitioner.get) // Update the view with shippedActives, setting activeness flags in the resulting @@ -88,7 +84,7 @@ class ReplicatedVertexView[VD: ClassManifest]( // Ship vertex attributes to edge partitions according to vertexPlacement val verts = updatedVerts.partitionsRDD - val shippedVerts = getRoutingTable(includeSrc, includeDst) + val shippedVerts = routingTable.get(includeSrc, includeDst) .zipPartitions(verts)(ReplicatedVertexView.buildBuffer(_, _)(vdManifest)) .partitionBy(edges.partitioner.get) // TODO: Consider using a specialized shuffler. @@ -126,19 +122,6 @@ class ReplicatedVertexView[VD: ClassManifest]( }.cache().setName("ReplicatedVertexView %s %s".format(includeSrc, includeDst)) } } - - /** - * Returns an RDD with the locations of edge-partition join sites for each vertex attribute in - * `vertices`; that is, the routing information for shipping vertex attributes to edge - * partitions. The routing information is stored as a compressed bitmap for each vertex partition. - */ - private def getRoutingTable( - includeSrc: Boolean, includeDst: Boolean): RDD[Array[Array[Vid]]] = { - routingTables.getOrElseUpdate( - (includeSrc, includeDst), - ReplicatedVertexView.createRoutingTable( - edges, updatedVerts.partitioner.get, includeSrc, includeDst)) - } } object ReplicatedVertexView { @@ -188,44 +171,6 @@ object ReplicatedVertexView { (pid, actives.trim().array) } } - - private def createRoutingTable( - edges: EdgeRDD[_], - vertexPartitioner: Partitioner, - includeSrc: Boolean, - includeDst: Boolean): RDD[Array[Array[Vid]]] = { - // Determine which vertices each edge partition needs by creating a mapping from vid to pid. - val vid2pid: RDD[(Vid, Pid)] = edges.partitionsRDD.mapPartitions { iter => - val (pid: Pid, edgePartition: EdgePartition[_]) = iter.next() - val numEdges = edgePartition.size - val vSet = new VertexSet - if (includeSrc) { // Add src vertices to the set. - var i = 0 - while (i < numEdges) { - vSet.add(edgePartition.srcIds(i)) - i += 1 - } - } - if (includeDst) { // Add dst vertices to the set. - var i = 0 - while (i < numEdges) { - vSet.add(edgePartition.dstIds(i)) - i += 1 - } - } - vSet.iterator.map { vid => (vid, pid) } - } - - val numPartitions = vertexPartitioner.numPartitions - vid2pid.partitionBy(vertexPartitioner).mapPartitions { iter => - val pid2vid = Array.fill(numPartitions)(new PrimitiveVector[Vid]) - for ((vid, pid) <- iter) { - pid2vid(pid) += vid - } - - Iterator(pid2vid.map(_.trim().array)) - }.cache().setName("VertexPlacement %s %s".format(includeSrc, includeDst)) - } } class VertexAttributeBlock[VD: ClassManifest](val vids: Array[Vid], val attrs: Array[VD]) { diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/RoutingTable.scala b/graph/src/main/scala/org/apache/spark/graph/impl/RoutingTable.scala new file mode 100644 index 0000000000000..b6cd048b33bb1 --- /dev/null +++ b/graph/src/main/scala/org/apache/spark/graph/impl/RoutingTable.scala @@ -0,0 +1,64 @@ +package org.apache.spark.graph.impl + +import org.apache.spark.SparkContext._ +import org.apache.spark.graph._ +import org.apache.spark.rdd.RDD +import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.collection.PrimitiveVector + +/** + * Stores the locations of edge-partition join sites for each vertex attribute; that is, the routing + * information for shipping vertex attributes to edge partitions. This is always cached because it + * may be used multiple times in ReplicatedVertexView -- once to ship the vertex attributes and + * (possibly) once to ship the active-set information. + */ +class RoutingTable(edges: EdgeRDD[_], vertices: VertexRDD[_]) { + + val bothAttrs: RDD[Array[Array[Vid]]] = createPid2Vid(true, true) + val srcAttrOnly: RDD[Array[Array[Vid]]] = createPid2Vid(true, false) + val dstAttrOnly: RDD[Array[Array[Vid]]] = createPid2Vid(false, true) + val noAttrs: RDD[Array[Array[Vid]]] = createPid2Vid(false, false) + + def get(includeSrcAttr: Boolean, includeDstAttr: Boolean): RDD[Array[Array[Vid]]] = + (includeSrcAttr, includeDstAttr) match { + case (true, true) => bothAttrs + case (true, false) => srcAttrOnly + case (false, true) => dstAttrOnly + case (false, false) => noAttrs + } + + private def createPid2Vid( + includeSrcAttr: Boolean, includeDstAttr: Boolean): RDD[Array[Array[Vid]]] = { + // Determine which vertices each edge partition needs by creating a mapping from vid to pid. + val vid2pid: RDD[(Vid, Pid)] = edges.partitionsRDD.mapPartitions { iter => + val (pid: Pid, edgePartition: EdgePartition[_]) = iter.next() + val numEdges = edgePartition.size + val vSet = new VertexSet + if (includeSrcAttr) { // Add src vertices to the set. + var i = 0 + while (i < numEdges) { + vSet.add(edgePartition.srcIds(i)) + i += 1 + } + } + if (includeDstAttr) { // Add dst vertices to the set. + var i = 0 + while (i < numEdges) { + vSet.add(edgePartition.dstIds(i)) + i += 1 + } + } + vSet.iterator.map { vid => (vid, pid) } + } + + val numPartitions = vertices.partitions.size + vid2pid.partitionBy(vertices.partitioner.get).mapPartitions { iter => + val pid2vid = Array.fill(numPartitions)(new PrimitiveVector[Vid]) + for ((vid, pid) <- iter) { + pid2vid(pid) += vid + } + + Iterator(pid2vid.map(_.trim().array)) + }.cache().setName("RoutingTable %s %s".format(includeSrcAttr, includeDstAttr)) + } +} From 0b3efbcf6273cbc11c597c368c2c123403575dae Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Wed, 1 Jan 2014 17:27:59 -0800 Subject: [PATCH 344/531] Adding partition level mapEdges and mapTriplets. This is necessary to support computation with random number generation. --- .../org/apache/spark/graph/EdgeRDD.scala | 16 ++-- .../scala/org/apache/spark/graph/Graph.scala | 76 ++++++++++++++++--- .../spark/graph/impl/EdgePartition.scala | 19 +++++ .../apache/spark/graph/impl/GraphImpl.scala | 49 +++++++----- 4 files changed, 125 insertions(+), 35 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala b/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala index 9aa76c93945f9..3dda5c7c604e5 100644 --- a/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala @@ -42,32 +42,32 @@ class EdgeRDD[@specialized ED: ClassManifest]( /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */ override def cache(): EdgeRDD[ED] = persist() - def mapEdgePartitions[ED2: ClassManifest](f: EdgePartition[ED] => EdgePartition[ED2]) + def mapEdgePartitions[ED2: ClassManifest](f: (Pid, EdgePartition[ED]) => EdgePartition[ED2]) : EdgeRDD[ED2] = { // iter => iter.map { case (pid, ep) => (pid, f(ep)) } new EdgeRDD[ED2](partitionsRDD.mapPartitions({ iter => val (pid, ep) = iter.next() - Iterator(Tuple2(pid, f(ep))) + Iterator(Tuple2(pid, f(pid, ep))) }, preservesPartitioning = true)) } def zipEdgePartitions[T: ClassManifest, U: ClassManifest] (other: RDD[T]) - (f: (EdgePartition[ED], Iterator[T]) => Iterator[U]): RDD[U] = { + (f: (Pid, EdgePartition[ED], Iterator[T]) => Iterator[U]): RDD[U] = { partitionsRDD.zipPartitions(other, preservesPartitioning = true) { (ePartIter, otherIter) => - val (_, edgePartition) = ePartIter.next() - f(edgePartition, otherIter) + val (pid, edgePartition) = ePartIter.next() + f(pid, edgePartition, otherIter) } } def zipEdgePartitions[ED2: ClassManifest, ED3: ClassManifest] (other: EdgeRDD[ED2]) - (f: (EdgePartition[ED], EdgePartition[ED2]) => EdgePartition[ED3]): EdgeRDD[ED3] = { + (f: (Pid, EdgePartition[ED], EdgePartition[ED2]) => EdgePartition[ED3]): EdgeRDD[ED3] = { new EdgeRDD[ED3](partitionsRDD.zipPartitions(other.partitionsRDD, preservesPartitioning = true) { (thisIter, otherIter) => val (pid, thisEPart) = thisIter.next() val (_, otherEPart) = otherIter.next() - Iterator(Tuple2(pid, f(thisEPart, otherEPart))) + Iterator(Tuple2(pid, f(pid, thisEPart, otherEPart))) }) } @@ -76,7 +76,7 @@ class EdgeRDD[@specialized ED: ClassManifest]( (f: (Vid, Vid, ED, ED2) => ED3): EdgeRDD[ED3] = { val ed2Manifest = classManifest[ED2] val ed3Manifest = classManifest[ED3] - zipEdgePartitions(other) { (thisEPart, otherEPart) => + zipEdgePartitions(other) { (pid, thisEPart, otherEPart) => thisEPart.innerJoin(otherEPart)(f)(ed2Manifest, ed3Manifest) } } diff --git a/graph/src/main/scala/org/apache/spark/graph/Graph.scala b/graph/src/main/scala/org/apache/spark/graph/Graph.scala index e8fa8e611c9cb..b725b2a15584b 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Graph.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Graph.scala @@ -72,9 +72,15 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { */ val triplets: RDD[EdgeTriplet[VD, ED]] + /** + * Cache the vertices and edges associated with this graph. + * + * @param newLevel the level at which to cache the graph. - - def persist(newLevel: StorageLevel): Graph[VD, ED] + * @return A reference to this graph for convenience. + * + */ + def persist(newLevel: StorageLevel = StorageLevel.MEMORY_ONLY): Graph[VD, ED] /** * Return a graph that is cached when first created. This is used to @@ -120,7 +126,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { def mapVertices[VD2: ClassManifest](map: (Vid, VD) => VD2): Graph[VD2, ED] /** - * Construct a new graph where each the value of each edge is + * Construct a new graph where the value of each edge is * transformed by the map operation. This function is not passed * the vertex value for the vertices adjacent to the edge. If * vertex values are desired use the mapTriplets function. @@ -137,18 +143,44 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * attributes. * */ - def mapEdges[ED2: ClassManifest](map: Edge[ED] => ED2): Graph[VD, ED2] + def mapEdges[ED2: ClassManifest](map: Edge[ED] => ED2): Graph[VD, ED2] = { + mapEdges((pid, iter) => iter.map(map)) + } + + /** + * Construct a new graph transforming the value of each edge using + * the user defined iterator transform. The iterator transform is + * given an iterator over edge triplets within a logical partition + * and should yield a new iterator over the new values of each edge + * in the order in which they are provided to the iterator transform + * If adjacent vertex values are not required, consider using the + * mapEdges function instead. + * + * @note This that this does not change the structure of the + * graph or modify the values of this graph. As a consequence + * the underlying index structures can be reused. + * + * @param map the function which takes a partition id and an iterator + * over all the edges in the partition and must return an iterator over + * the new values for each edge in the order of the input iterator. + * + * @tparam ED2 the new edge data type + * + */ + def mapEdges[ED2: ClassManifest]( + map: (Pid, Iterator[Edge[ED]]) => Iterator[ED2]): + Graph[VD, ED2] /** - * Construct a new graph where each the value of each edge is + * Construct a new graph where the value of each edge is * transformed by the map operation. This function passes vertex * values for the adjacent vertices to the map function. If * adjacent vertex values are not required, consider using the * mapEdges function instead. * - * @note This graph is not changed and that the new graph has the - * same structure. As a consequence the underlying index structures - * can be reused. + * @note This that this does not change the structure of the + * graph or modify the values of this graph. As a consequence + * the underlying index structures can be reused. * * @param map the function from an edge object to a new edge value. * @@ -163,7 +195,33 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * }}} * */ - def mapTriplets[ED2: ClassManifest](map: EdgeTriplet[VD, ED] => ED2): Graph[VD, ED2] + def mapTriplets[ED2: ClassManifest](map: EdgeTriplet[VD, ED] => ED2): Graph[VD, ED2] = { + mapTriplets((pid, iter) => iter.map(map)) + } + + /** + * Construct a new graph transforming the value of each edge using + * the user defined iterator transform. The iterator transform is + * given an iterator over edge triplets within a logical partition + * and should yield a new iterator over the new values of each edge + * in the order in which they are provided to the iterator transform + * If adjacent vertex values are not required, consider using the + * mapEdges function instead. + * + * @note This that this does not change the structure of the + * graph or modify the values of this graph. As a consequence + * the underlying index structures can be reused. + * + * @param map the function which takes a partition id and an iterator + * over all the edges in the partition and must return an iterator over + * the new values for each edge in the order of the input iterator. + * + * @tparam ED2 the new edge data type + * + */ + def mapTriplets[ED2: ClassManifest]( + map: (Pid, Iterator[EdgeTriplet[VD, ED]]) => Iterator[ED2]): + Graph[VD, ED2] /** * Construct a new graph with all the edges reversed. If this graph diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala b/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala index e97522feaeaf2..bd03f14903638 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala @@ -56,6 +56,25 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) new EdgePartition(srcIds, dstIds, newData, index) } + /** + * Construct a new edge partition by using the edge attributes + * contained in the iterator. + * + * @note The input iterator should return edge attributes in the + * order of the edges returned by `EdgePartition.iterator` and + * should return attributes equal to the number of edges. + * + * @param f a function from an edge to a new attribute + * @tparam ED2 the type of the new attribute + * @return a new edge partition with the result of the function `f` + * applied to each edge + */ + def map[ED2: ClassManifest](iter: Iterator[ED2]): EdgePartition[ED2] = { + val newData = iter.toArray + assert(newData.size == data.size) + new EdgePartition(srcIds, dstIds, newData, index) + } + /** * Apply the function f to all edges in this partition. * diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index 16d73820f07df..79c11c780a69a 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -47,8 +47,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( @transient override val triplets: RDD[EdgeTriplet[VD, ED]] = { val vdManifest = classManifest[VD] val edManifest = classManifest[ED] - - edges.zipEdgePartitions(replicatedVertexView.get(true, true)) { (ePart, vPartIter) => + edges.zipEdgePartitions(replicatedVertexView.get(true, true)) { (pid, ePart, vPartIter) => val (_, vPart) = vPartIter.next() new EdgeTripletIterator(vPart.index, vPart.values, ePart)(vdManifest, edManifest) } @@ -149,8 +148,10 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( println(visited) } // end of printLineage - override def reverse: Graph[VD, ED] = - new GraphImpl(vertices, edges.mapEdgePartitions(_.reverse), routingTable, replicatedVertexView) + override def reverse: Graph[VD, ED] = { + val newETable = edges.mapEdgePartitions((pid, part) => part.reverse) + new GraphImpl(vertices, newETable, routingTable, replicatedVertexView) + } override def mapVertices[VD2: ClassManifest](f: (Vid, VD) => VD2): Graph[VD2, ED] = { if (classManifest[VD] equals classManifest[VD2]) { @@ -167,25 +168,36 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( } } - override def mapEdges[ED2: ClassManifest](f: Edge[ED] => ED2): Graph[VD, ED2] = - new GraphImpl(vertices, edges.mapEdgePartitions(_.map(f)), routingTable, replicatedVertexView) + override def mapEdges[ED2: ClassManifest]( + f: (Pid, Iterator[Edge[ED]]) => Iterator[ED2]): Graph[VD, ED2] = { + val newETable = edges.mapEdgePartitions((pid, part) => part.map(f(pid, part.iterator))) + new GraphImpl(vertices, newETable , routingTable, replicatedVertexView) + } - override def mapTriplets[ED2: ClassManifest](f: EdgeTriplet[VD, ED] => ED2): Graph[VD, ED2] = { + override def mapTriplets[ED2: ClassManifest]( + f: (Pid, Iterator[EdgeTriplet[VD, ED]]) => Iterator[ED2]): Graph[VD, ED2] = { // Use an explicit manifest in PrimitiveKeyOpenHashMap init so we don't pull in the implicit // manifest from GraphImpl (which would require serializing GraphImpl). val vdManifest = classManifest[VD] val newEdgePartitions = - edges.zipEdgePartitions(replicatedVertexView.get(true, true)) { (edgePartition, vPartIter) => - val (pid, vPart) = vPartIter.next() + edges.zipEdgePartitions(replicatedVertexView.get(true, true)) { + (ePid, edgePartition, vTableReplicatedIter) => + val (vPid, vPart) = vTableReplicatedIter.next() + assert(!vTableReplicatedIter.hasNext) + assert(ePid == vPid) val et = new EdgeTriplet[VD, ED] - val newEdgePartition = edgePartition.map { e => + val inputIterator = edgePartition.iterator.map { e => et.set(e) et.srcAttr = vPart(e.srcId) et.dstAttr = vPart(e.dstId) - f(et) + et } - Iterator((pid, newEdgePartition)) - } + // Apply the user function to the vertex partition + val outputIter = f(ePid, inputIterator) + // Consume the iterator to update the edge attributes + val newEdgePartition = edgePartition.map(outputIter) + Iterator((ePid, newEdgePartition)) + } new GraphImpl(vertices, new EdgeRDD(newEdgePartitions), routingTable, replicatedVertexView) } @@ -224,8 +236,8 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( override def groupEdges(merge: (ED, ED) => ED): Graph[VD, ED] = { ClosureCleaner.clean(merge) - val newEdges = edges.mapEdgePartitions(_.groupEdges(merge)) - new GraphImpl(vertices, newEdges, routingTable, replicatedVertexView) + val newETable = edges.mapEdgePartitions((pid, part) => part.groupEdges(merge)) + new GraphImpl(vertices, newETable, routingTable, replicatedVertexView) } ////////////////////////////////////////////////////////////////////////////////////////////////// @@ -253,9 +265,10 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( val activeDirectionOpt = activeSetOpt.map(_._2) // Map and combine. - val preAgg = edges.zipEdgePartitions(vs) { (edgePartition, vPartIter) => - val (_, vPart) = vPartIter.next() - + val preAgg = edges.zipEdgePartitions(vs) { (ePid, edgePartition, vPartIter) => + val (vPid, vPart) = vPartIter.next() + assert(!vPartIter.hasNext) + assert(ePid == vPid) // Choose scan method val activeFraction = vPart.numActives.getOrElse(0) / edgePartition.indexSize.toFloat val edgeIter = activeDirectionOpt match { From 6592be2594c50ac4a0018b59111ae23de89601b9 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Thu, 2 Jan 2014 00:00:24 -0800 Subject: [PATCH 345/531] slightly more efficient map operation --- .../org/apache/spark/graph/impl/EdgePartition.scala | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala b/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala index bd03f14903638..4fcf08efce382 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala @@ -70,8 +70,13 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) * applied to each edge */ def map[ED2: ClassManifest](iter: Iterator[ED2]): EdgePartition[ED2] = { - val newData = iter.toArray - assert(newData.size == data.size) + val newData = new Array[ED2](data.size) + var i = 0 + while (iter.hasNext) { + newData(i) = iter.next() + i += 1 + } + assert(newData.size == i) new EdgePartition(srcIds, dstIds, newData, index) } From fa8ce3fdd7f91add0e8ead6e48f4d69e67c604b9 Mon Sep 17 00:00:00 2001 From: Adam Novak Date: Mon, 6 Jan 2014 14:45:00 -0800 Subject: [PATCH 346/531] Changing org.apache.spark.util.collection.PrimitiveKeyOpenHashMap to have a real no-argument constructor, instead of a one-argument constructor with a default value. The lack of a real no-argument constructor was causing "sbt/sbt publish-local" to fail thusly: ``` [error] /pod/home/anovak/build/graphx/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala:172: not enough arguments for constructor PrimitiveKeyOpenHashMap: (initialCapacity: Int)(implicit evidence$3: ClassManifest[Int], implicit evidence$4: ClassManifest[Int])org.apache.spark.util.collection.PrimitiveKeyOpenHashMap[Int,Int] [error] private val mapIdToIndex = new PrimitiveKeyOpenHashMap[Int, Int]() [error] ^ [info] No documentation generated with unsucessful compiler run [error] one error found [error] (core/compile:doc) Scaladoc generation failed [error] Total time: 67 s, completed Jan 6, 2014 2:20:51 PM ``` In theory a no-argument constructor ought not to differ from one with a single argument that has a default value, but in practice there seems to be an issue. --- .../spark/util/collection/PrimitiveKeyOpenHashMap.scala | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMap.scala b/core/src/main/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMap.scala index ee1b168028b4a..d6a3cdb405cbc 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMap.scala @@ -35,8 +35,15 @@ class PrimitiveKeyOpenHashMap[@specialized(Long, Int) K: ClassManifest, /** * Allocate an OpenHashMap with a fixed initial capacity */ - def this(initialCapacity: Int = 64) = + def this(initialCapacity: Int) = this(new OpenHashSet[K](initialCapacity), new Array[V](initialCapacity)) + + /** + * Allocate an OpenHashMap with a default initial capacity, providing a true + * no-argument constructor. + */ + def this() = this(64) + /** * Allocate an OpenHashMap with a fixed initial capacity From 78d6b13ac88d4f0e52cf430d0bc3c1eb5369e4dc Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 3 Jan 2014 12:33:23 -0700 Subject: [PATCH 347/531] Fix mis-merge in 44fd30d3fbcf830deecbe8ea3e8ea165e74e6edd --- core/src/main/scala/org/apache/spark/rdd/RDD.scala | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 3f41b66279987..2142ae730e9ff 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -547,6 +547,11 @@ abstract class RDD[T: ClassTag]( * *same number of partitions*, but does *not* require them to have the same number * of elements in each partition. */ + def zipPartitions[B: ClassTag, V: ClassTag] + (rdd2: RDD[B], preservesPartitioning: Boolean) + (f: (Iterator[T], Iterator[B]) => Iterator[V]): RDD[V] = + new ZippedPartitionsRDD2(sc, sc.clean(f), this, rdd2, preservesPartitioning) + def zipPartitions[B: ClassTag, V: ClassTag] (rdd2: RDD[B]) (f: (Iterator[T], Iterator[B]) => Iterator[V]): RDD[V] = From ac536345f86e467ac83cb9c0dccbb34150335e26 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 3 Jan 2014 12:50:56 -0700 Subject: [PATCH 348/531] ClassManifest -> ClassTag --- .../org/apache/spark/graph/Analytics.scala | 2 +- .../org/apache/spark/graph/EdgeRDD.scala | 16 ++++---- .../org/apache/spark/graph/EdgeTriplet.scala | 4 +- .../scala/org/apache/spark/graph/Graph.scala | 28 +++++++------ .../org/apache/spark/graph/GraphLab.scala | 4 +- .../org/apache/spark/graph/GraphLoader.scala | 4 +- .../org/apache/spark/graph/GraphOps.scala | 10 +++-- .../scala/org/apache/spark/graph/Pregel.scala | 4 +- .../org/apache/spark/graph/VertexRDD.scala | 28 +++++++------ .../graph/algorithms/TriangleCount.scala | 4 +- .../spark/graph/impl/EdgePartition.scala | 10 +++-- .../graph/impl/EdgePartitionBuilder.scala | 3 +- .../graph/impl/EdgeTripletIterator.scala | 4 +- .../apache/spark/graph/impl/GraphImpl.scala | 40 ++++++++++--------- .../spark/graph/impl/MessageToPartition.scala | 24 ++++++----- .../graph/impl/ReplicatedVertexView.scala | 10 +++-- .../spark/graph/impl/VertexPartition.scala | 22 +++++----- .../spark/graph/util/GraphGenerators.scala | 20 ++-------- .../spark/graph/impl/EdgePartitionSuite.scala | 3 +- 19 files changed, 129 insertions(+), 111 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala index 2012dadb2f305..14b9be73f1651 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Analytics.scala @@ -335,7 +335,7 @@ object Analytics extends Logging { // /** // * // */ - // def alternatingLeastSquares[VD: ClassManifest, ED: ClassManifest](graph: Graph[VD, Double], + // def alternatingLeastSquares[VD: ClassTag, ED: ClassTag](graph: Graph[VD, Double], // latentK: Int, lambda: Double, numIter: Int) = { // val vertices = graph.vertices.mapPartitions( _.map { // case (vid, _) => (vid, Array.fill(latentK){ scala.util.Random.nextDouble() } ) diff --git a/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala b/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala index 3dda5c7c604e5..1c21967c9cab3 100644 --- a/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala @@ -1,12 +1,14 @@ package org.apache.spark.graph +import scala.reflect.{classTag, ClassTag} + import org.apache.spark.{OneToOneDependency, Partition, Partitioner, TaskContext} import org.apache.spark.graph.impl.EdgePartition import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel -class EdgeRDD[@specialized ED: ClassManifest]( +class EdgeRDD[@specialized ED: ClassTag]( val partitionsRDD: RDD[(Pid, EdgePartition[ED])]) extends RDD[Edge[ED]](partitionsRDD.context, List(new OneToOneDependency(partitionsRDD))) { @@ -42,7 +44,7 @@ class EdgeRDD[@specialized ED: ClassManifest]( /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */ override def cache(): EdgeRDD[ED] = persist() - def mapEdgePartitions[ED2: ClassManifest](f: (Pid, EdgePartition[ED]) => EdgePartition[ED2]) + def mapEdgePartitions[ED2: ClassTag](f: (Pid, EdgePartition[ED]) => EdgePartition[ED2]) : EdgeRDD[ED2] = { // iter => iter.map { case (pid, ep) => (pid, f(ep)) } new EdgeRDD[ED2](partitionsRDD.mapPartitions({ iter => @@ -51,7 +53,7 @@ class EdgeRDD[@specialized ED: ClassManifest]( }, preservesPartitioning = true)) } - def zipEdgePartitions[T: ClassManifest, U: ClassManifest] + def zipEdgePartitions[T: ClassTag, U: ClassTag] (other: RDD[T]) (f: (Pid, EdgePartition[ED], Iterator[T]) => Iterator[U]): RDD[U] = { partitionsRDD.zipPartitions(other, preservesPartitioning = true) { (ePartIter, otherIter) => @@ -60,7 +62,7 @@ class EdgeRDD[@specialized ED: ClassManifest]( } } - def zipEdgePartitions[ED2: ClassManifest, ED3: ClassManifest] + def zipEdgePartitions[ED2: ClassTag, ED3: ClassTag] (other: EdgeRDD[ED2]) (f: (Pid, EdgePartition[ED], EdgePartition[ED2]) => EdgePartition[ED3]): EdgeRDD[ED3] = { new EdgeRDD[ED3](partitionsRDD.zipPartitions(other.partitionsRDD, preservesPartitioning = true) { @@ -71,11 +73,11 @@ class EdgeRDD[@specialized ED: ClassManifest]( }) } - def innerJoin[ED2: ClassManifest, ED3: ClassManifest] + def innerJoin[ED2: ClassTag, ED3: ClassTag] (other: EdgeRDD[ED2]) (f: (Vid, Vid, ED, ED2) => ED3): EdgeRDD[ED3] = { - val ed2Manifest = classManifest[ED2] - val ed3Manifest = classManifest[ED3] + val ed2Manifest = classTag[ED2] + val ed3Manifest = classTag[ED3] zipEdgePartitions(other) { (pid, thisEPart, otherEPart) => thisEPart.innerJoin(otherEPart)(f)(ed2Manifest, ed3Manifest) } diff --git a/graph/src/main/scala/org/apache/spark/graph/EdgeTriplet.scala b/graph/src/main/scala/org/apache/spark/graph/EdgeTriplet.scala index 76768489eed37..5a384a5f84dd9 100644 --- a/graph/src/main/scala/org/apache/spark/graph/EdgeTriplet.scala +++ b/graph/src/main/scala/org/apache/spark/graph/EdgeTriplet.scala @@ -14,8 +14,8 @@ import org.apache.spark.graph.impl.VertexPartition * that is not a trait. */ class EdgeTriplet[VD, ED] extends Edge[ED] { -// class EdgeTriplet[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) VD: ClassManifest, -// @specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED: ClassManifest] extends Edge[ED] { +// class EdgeTriplet[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) VD: ClassTag, +// @specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED: ClassTag] extends Edge[ED] { /** diff --git a/graph/src/main/scala/org/apache/spark/graph/Graph.scala b/graph/src/main/scala/org/apache/spark/graph/Graph.scala index b725b2a15584b..9dd26f7679a2d 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Graph.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Graph.scala @@ -1,5 +1,7 @@ package org.apache.spark.graph +import scala.reflect.ClassTag + import org.apache.spark.graph.impl._ import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel @@ -23,7 +25,7 @@ import org.apache.spark.storage.StorageLevel * @tparam VD the vertex attribute type * @tparam ED the edge attribute type */ -abstract class Graph[VD: ClassManifest, ED: ClassManifest] { +abstract class Graph[VD: ClassTag, ED: ClassTag] { /** * Get the vertices and their data. @@ -123,7 +125,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * }}} * */ - def mapVertices[VD2: ClassManifest](map: (Vid, VD) => VD2): Graph[VD2, ED] + def mapVertices[VD2: ClassTag](map: (Vid, VD) => VD2): Graph[VD2, ED] /** * Construct a new graph where the value of each edge is @@ -143,7 +145,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * attributes. * */ - def mapEdges[ED2: ClassManifest](map: Edge[ED] => ED2): Graph[VD, ED2] = { + def mapEdges[ED2: ClassTag](map: Edge[ED] => ED2): Graph[VD, ED2] = { mapEdges((pid, iter) => iter.map(map)) } @@ -167,7 +169,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * @tparam ED2 the new edge data type * */ - def mapEdges[ED2: ClassManifest]( + def mapEdges[ED2: ClassTag]( map: (Pid, Iterator[Edge[ED]]) => Iterator[ED2]): Graph[VD, ED2] @@ -195,7 +197,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * }}} * */ - def mapTriplets[ED2: ClassManifest](map: EdgeTriplet[VD, ED] => ED2): Graph[VD, ED2] = { + def mapTriplets[ED2: ClassTag](map: EdgeTriplet[VD, ED] => ED2): Graph[VD, ED2] = { mapTriplets((pid, iter) => iter.map(map)) } @@ -219,7 +221,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * @tparam ED2 the new edge data type * */ - def mapTriplets[ED2: ClassManifest]( + def mapTriplets[ED2: ClassTag]( map: (Pid, Iterator[EdgeTriplet[VD, ED]]) => Iterator[ED2]): Graph[VD, ED2] @@ -261,7 +263,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * @return a graph with vertices and edges that exists in both the current graph and other, * with vertex and edge data from the current graph. */ - def mask[VD2: ClassManifest, ED2: ClassManifest](other: Graph[VD2, ED2]): Graph[VD, ED] + def mask[VD2: ClassTag, ED2: ClassTag](other: Graph[VD2, ED2]): Graph[VD, ED] /** * This function merges multiple edges between two vertices into a single Edge. For correct @@ -313,7 +315,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * predicate or implement PageRank. * */ - def mapReduceTriplets[A: ClassManifest]( + def mapReduceTriplets[A: ClassTag]( mapFunc: EdgeTriplet[VD, ED] => Iterator[(Vid, A)], reduceFunc: (A, A) => A, activeSetOpt: Option[(VertexRDD[_], EdgeDirection)] = None) @@ -348,7 +350,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * }}} * */ - def outerJoinVertices[U: ClassManifest, VD2: ClassManifest](table: RDD[(Vid, U)]) + def outerJoinVertices[U: ClassTag, VD2: ClassTag](table: RDD[(Vid, U)]) (mapFunc: (Vid, VD, Option[U]) => VD2) : Graph[VD2, ED] @@ -376,7 +378,7 @@ object Graph { * @return a graph with edge attributes containing either the count of duplicate edges or 1 * (if `uniqueEdges=None`) and vertex attributes containing the total degree of each vertex. */ - def fromEdgeTuples[VD: ClassManifest]( + def fromEdgeTuples[VD: ClassTag]( rawEdges: RDD[(Vid, Vid)], defaultValue: VD, uniqueEdges: Option[PartitionStrategy] = None): Graph[VD, Int] = { @@ -397,7 +399,7 @@ object Graph { * @return a graph with edge attributes described by `edges` and vertices * given by all vertices in `edges` with value `defaultValue` */ - def fromEdges[VD: ClassManifest, ED: ClassManifest]( + def fromEdges[VD: ClassTag, ED: ClassTag]( edges: RDD[Edge[ED]], defaultValue: VD): Graph[VD, ED] = { GraphImpl(edges, defaultValue) @@ -418,7 +420,7 @@ object Graph { * @param partitionStrategy the partition strategy to use when * partitioning the edges. */ - def apply[VD: ClassManifest, ED: ClassManifest]( + def apply[VD: ClassTag, ED: ClassTag]( vertices: RDD[(Vid, VD)], edges: RDD[Edge[ED]], defaultVertexAttr: VD = null.asInstanceOf[VD]): Graph[VD, ED] = { @@ -432,5 +434,5 @@ object Graph { * convenience operations are defined in the GraphOps class which may be shared across multiple * graph implementations. */ - implicit def graphToGraphOps[VD: ClassManifest, ED: ClassManifest](g: Graph[VD, ED]) = g.ops + implicit def graphToGraphOps[VD: ClassTag, ED: ClassTag](g: Graph[VD, ED]) = g.ops } // end of Graph object diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala b/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala index 5d2f0f4bda6f1..c1ce5cd9ccada 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala @@ -1,5 +1,7 @@ package org.apache.spark.graph +import scala.reflect.ClassTag + import org.apache.spark.Logging import scala.collection.JavaConversions._ import org.apache.spark.rdd.RDD @@ -36,7 +38,7 @@ object GraphLab extends Logging { * @tparam A The type accumulated during the gather phase * @return the resulting graph after the algorithm converges */ - def apply[VD: ClassManifest, ED: ClassManifest, A: ClassManifest] + def apply[VD: ClassTag, ED: ClassTag, A: ClassTag] (graph: Graph[VD, ED], numIter: Int, gatherDirection: EdgeDirection = EdgeDirection.In, scatterDirection: EdgeDirection = EdgeDirection.Out) diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala b/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala index a69bfde5322df..7daac4fcc56c3 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala @@ -1,6 +1,8 @@ package org.apache.spark.graph import java.util.{Arrays => JArrays} +import scala.reflect.ClassTag + import org.apache.spark.graph.impl.EdgePartitionBuilder import org.apache.spark.{Logging, SparkContext} import org.apache.spark.graph.impl.{EdgePartition, GraphImpl} @@ -22,7 +24,7 @@ object GraphLoader extends Logging { * the Edge RDD * */ - def textFile[ED: ClassManifest]( + def textFile[ED: ClassTag]( sc: SparkContext, path: String, edgeParser: Array[String] => ED, diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala b/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala index 091c7782757f2..11c6120beb414 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala @@ -1,5 +1,7 @@ package org.apache.spark.graph +import scala.reflect.ClassTag + import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext._ import org.apache.spark.SparkException @@ -15,7 +17,7 @@ import org.apache.spark.SparkException * @tparam ED the edge attribute type * */ -class GraphOps[VD: ClassManifest, ED: ClassManifest](graph: Graph[VD, ED]) { +class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) { /** * Compute the number of edges in the graph. @@ -109,7 +111,7 @@ class GraphOps[VD: ClassManifest, ED: ClassManifest](graph: Graph[VD, ED]) { * @todo Should this return a graph with the new vertex values? * */ - def aggregateNeighbors[A: ClassManifest]( + def aggregateNeighbors[A: ClassTag]( mapFunc: (Vid, EdgeTriplet[VD, ED]) => Option[A], reduceFunc: (A, A) => A, dir: EdgeDirection) @@ -226,7 +228,7 @@ class GraphOps[VD: ClassManifest, ED: ClassManifest](graph: Graph[VD, ED]) { * }}} * */ - def joinVertices[U: ClassManifest](table: RDD[(Vid, U)])(mapFunc: (Vid, VD, U) => VD) + def joinVertices[U: ClassTag](table: RDD[(Vid, U)])(mapFunc: (Vid, VD, U) => VD) : Graph[VD, ED] = { val uf = (id: Vid, data: VD, o: Option[U]) => { o match { @@ -262,7 +264,7 @@ class GraphOps[VD: ClassManifest, ED: ClassManifest](graph: Graph[VD, ED]) { * }}} * */ - def filter[VD2: ClassManifest, ED2: ClassManifest]( + def filter[VD2: ClassTag, ED2: ClassTag]( preprocess: Graph[VD, ED] => Graph[VD2, ED2], epred: (EdgeTriplet[VD2, ED2]) => Boolean = (x: EdgeTriplet[VD2, ED2]) => true, vpred: (Vid, VD2) => Boolean = (v:Vid, d:VD2) => true): Graph[VD, ED] = { diff --git a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala index 285e857b693fc..4664091b5714b 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala @@ -1,5 +1,7 @@ package org.apache.spark.graph +import scala.reflect.ClassTag + /** * This object implements a Pregel-like bulk-synchronous @@ -84,7 +86,7 @@ object Pregel { * @return the resulting graph at the end of the computation * */ - def apply[VD: ClassManifest, ED: ClassManifest, A: ClassManifest] + def apply[VD: ClassTag, ED: ClassTag, A: ClassTag] (graph: Graph[VD, ED], initialMsg: A, maxIterations: Int = Int.MaxValue)( vprog: (Vid, VD, A) => VD, sendMsg: EdgeTriplet[VD, ED] => Iterator[(Vid,A)], diff --git a/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala b/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala index c274e342c7566..8e5e319928274 100644 --- a/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala @@ -17,6 +17,8 @@ package org.apache.spark.graph +import scala.reflect.ClassTag + import org.apache.spark._ import org.apache.spark.SparkContext._ import org.apache.spark.rdd._ @@ -50,7 +52,7 @@ import org.apache.spark.graph.impl.VertexPartition * }}} * */ -class VertexRDD[@specialized VD: ClassManifest]( +class VertexRDD[@specialized VD: ClassTag]( val partitionsRDD: RDD[VertexPartition[VD]]) extends RDD[(Vid, VD)](partitionsRDD.context, List(new OneToOneDependency(partitionsRDD))) { @@ -111,7 +113,7 @@ class VertexRDD[@specialized VD: ClassManifest]( /** * Return a new VertexRDD by applying a function to each VertexPartition of this RDD. */ - def mapVertexPartitions[VD2: ClassManifest](f: VertexPartition[VD] => VertexPartition[VD2]) + def mapVertexPartitions[VD2: ClassTag](f: VertexPartition[VD] => VertexPartition[VD2]) : VertexRDD[VD2] = { val newPartitionsRDD = partitionsRDD.mapPartitions(_.map(f), preservesPartitioning = true) new VertexRDD(newPartitionsRDD) @@ -121,7 +123,7 @@ class VertexRDD[@specialized VD: ClassManifest]( * Return a new VertexRDD by applying a function to corresponding * VertexPartitions of this VertexRDD and another one. */ - def zipVertexPartitions[VD2: ClassManifest, VD3: ClassManifest] + def zipVertexPartitions[VD2: ClassTag, VD3: ClassTag] (other: VertexRDD[VD2]) (f: (VertexPartition[VD], VertexPartition[VD2]) => VertexPartition[VD3]): VertexRDD[VD3] = { val newPartitionsRDD = partitionsRDD.zipPartitions( @@ -160,7 +162,7 @@ class VertexRDD[@specialized VD: ClassManifest]( * each of the entries in the original VertexRDD. The resulting * VertexRDD retains the same index. */ - def mapValues[VD2: ClassManifest](f: VD => VD2): VertexRDD[VD2] = + def mapValues[VD2: ClassTag](f: VD => VD2): VertexRDD[VD2] = this.mapVertexPartitions(_.map((vid, attr) => f(attr))) /** @@ -174,7 +176,7 @@ class VertexRDD[@specialized VD: ClassManifest]( * each of the entries in the original VertexRDD. The resulting * VertexRDD retains the same index. */ - def mapValues[VD2: ClassManifest](f: (Vid, VD) => VD2): VertexRDD[VD2] = + def mapValues[VD2: ClassTag](f: (Vid, VD) => VD2): VertexRDD[VD2] = this.mapVertexPartitions(_.map(f)) /** @@ -205,7 +207,7 @@ class VertexRDD[@specialized VD: ClassManifest]( * other VertexSet. * */ - def leftZipJoin[VD2: ClassManifest, VD3: ClassManifest] + def leftZipJoin[VD2: ClassTag, VD3: ClassTag] (other: VertexRDD[VD2])(f: (Vid, VD, Option[VD2]) => VD3): VertexRDD[VD3] = { this.zipVertexPartitions(other) { (thisPart, otherPart) => thisPart.leftJoin(otherPart)(f) @@ -231,7 +233,7 @@ class VertexRDD[@specialized VD: ClassManifest]( * @return a VertexRDD containing all the vertices in this * VertexRDD with the attribute emitted by f. */ - def leftJoin[VD2: ClassManifest, VD3: ClassManifest] + def leftJoin[VD2: ClassTag, VD3: ClassTag] (other: RDD[(Vid, VD2)]) (f: (Vid, VD, Option[VD2]) => VD3) : VertexRDD[VD3] = @@ -257,7 +259,7 @@ class VertexRDD[@specialized VD: ClassManifest]( * Same effect as leftJoin(other) { (vid, a, bOpt) => bOpt.getOrElse(a) }, but `this` and `other` * must have the same index. */ - def innerZipJoin[U: ClassManifest, VD2: ClassManifest](other: VertexRDD[U]) + def innerZipJoin[U: ClassTag, VD2: ClassTag](other: VertexRDD[U]) (f: (Vid, VD, U) => VD2): VertexRDD[VD2] = { this.zipVertexPartitions(other) { (thisPart, otherPart) => thisPart.innerJoin(otherPart)(f) @@ -268,7 +270,7 @@ class VertexRDD[@specialized VD: ClassManifest]( * Replace vertices with corresponding vertices in `other`, and drop vertices without a * corresponding vertex in `other`. */ - def innerJoin[U: ClassManifest, VD2: ClassManifest](other: RDD[(Vid, U)]) + def innerJoin[U: ClassTag, VD2: ClassTag](other: RDD[(Vid, U)]) (f: (Vid, VD, U) => VD2): VertexRDD[VD2] = { // Test if the other vertex is a VertexRDD to choose the optimal join strategy. // If the other set is a VertexRDD then we use the much more efficient innerZipJoin @@ -291,7 +293,7 @@ class VertexRDD[@specialized VD: ClassManifest]( * Aggregate messages with the same ids using `reduceFunc`, returning a VertexRDD that is * co-indexed with this one. */ - def aggregateUsingIndex[VD2: ClassManifest]( + def aggregateUsingIndex[VD2: ClassTag]( messages: RDD[(Vid, VD2)], reduceFunc: (VD2, VD2) => VD2): VertexRDD[VD2] = { val shuffled = MsgRDDFunctions.partitionForAggregation(messages, this.partitioner.get) @@ -318,7 +320,7 @@ object VertexRDD { * * @param rdd the collection of vertex-attribute pairs */ - def apply[VD: ClassManifest](rdd: RDD[(Vid, VD)]): VertexRDD[VD] = { + def apply[VD: ClassTag](rdd: RDD[(Vid, VD)]): VertexRDD[VD] = { val partitioned: RDD[(Vid, VD)] = rdd.partitioner match { case Some(p) => rdd case None => rdd.partitionBy(new HashPartitioner(rdd.partitions.size)) @@ -338,7 +340,7 @@ object VertexRDD { * @param rdd the collection of vertex-attribute pairs * @param mergeFunc the associative, commutative merge function. */ - def apply[VD: ClassManifest](rdd: RDD[(Vid, VD)], mergeFunc: (VD, VD) => VD): VertexRDD[VD] = + def apply[VD: ClassTag](rdd: RDD[(Vid, VD)], mergeFunc: (VD, VD) => VD): VertexRDD[VD] = { val partitioned: RDD[(Vid, VD)] = rdd.partitioner match { case Some(p) => rdd @@ -350,7 +352,7 @@ object VertexRDD { new VertexRDD(vertexPartitions) } - def apply[VD: ClassManifest](vids: RDD[Vid], rdd: RDD[(Vid, VD)], defaultVal: VD) + def apply[VD: ClassTag](vids: RDD[Vid], rdd: RDD[(Vid, VD)], defaultVal: VD) : VertexRDD[VD] = { VertexRDD(vids.map(vid => (vid, defaultVal))).leftJoin(rdd) { (vid, default, value) => diff --git a/graph/src/main/scala/org/apache/spark/graph/algorithms/TriangleCount.scala b/graph/src/main/scala/org/apache/spark/graph/algorithms/TriangleCount.scala index b1cd3c47d0cf1..a6384320bab6b 100644 --- a/graph/src/main/scala/org/apache/spark/graph/algorithms/TriangleCount.scala +++ b/graph/src/main/scala/org/apache/spark/graph/algorithms/TriangleCount.scala @@ -1,5 +1,7 @@ package org.apache.spark.graph.algorithms +import scala.reflect.ClassTag + import org.apache.spark.graph._ @@ -21,7 +23,7 @@ object TriangleCount { * * @return */ - def run[VD: ClassManifest, ED: ClassManifest](graph: Graph[VD,ED]): Graph[Int, ED] = { + def run[VD: ClassTag, ED: ClassTag](graph: Graph[VD,ED]): Graph[Int, ED] = { // Remove redundant edges val g = graph.groupEdges((a, b) => a).cache diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala b/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala index 4fcf08efce382..7367269f67bfa 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala @@ -1,5 +1,7 @@ package org.apache.spark.graph.impl +import scala.reflect.ClassTag + import org.apache.spark.graph._ import org.apache.spark.util.collection.PrimitiveKeyOpenHashMap @@ -13,7 +15,7 @@ import org.apache.spark.util.collection.PrimitiveKeyOpenHashMap * @param index a clustered index on source vertex id * @tparam ED the edge attribute type. */ -class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED: ClassManifest]( +class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED: ClassTag]( val srcIds: Array[Vid], val dstIds: Array[Vid], val data: Array[ED], @@ -41,7 +43,7 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) * @return a new edge partition with the result of the function `f` * applied to each edge */ - def map[ED2: ClassManifest](f: Edge[ED] => ED2): EdgePartition[ED2] = { + def map[ED2: ClassTag](f: Edge[ED] => ED2): EdgePartition[ED2] = { val newData = new Array[ED2](data.size) val edge = new Edge[ED]() val size = data.size @@ -69,7 +71,7 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) * @return a new edge partition with the result of the function `f` * applied to each edge */ - def map[ED2: ClassManifest](iter: Iterator[ED2]): EdgePartition[ED2] = { + def map[ED2: ClassTag](iter: Iterator[ED2]): EdgePartition[ED2] = { val newData = new Array[ED2](data.size) var i = 0 while (iter.hasNext) { @@ -132,7 +134,7 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) * If there are multiple edges with the same src and dst in `other`, `f` will only be invoked * once. */ - def innerJoin[ED2: ClassManifest, ED3: ClassManifest] + def innerJoin[ED2: ClassTag, ED3: ClassTag] (other: EdgePartition[ED2]) (f: (Vid, Vid, ED, ED2) => ED3): EdgePartition[ED3] = { val builder = new EdgePartitionBuilder[ED3] diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartitionBuilder.scala b/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartitionBuilder.scala index 38762733692cc..ae3f3a6d03145 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartitionBuilder.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartitionBuilder.scala @@ -1,5 +1,6 @@ package org.apache.spark.graph.impl +import scala.reflect.ClassTag import scala.util.Sorting import org.apache.spark.graph._ @@ -7,7 +8,7 @@ import org.apache.spark.util.collection.{PrimitiveKeyOpenHashMap, PrimitiveVecto //private[graph] -class EdgePartitionBuilder[@specialized(Long, Int, Double) ED: ClassManifest](size: Int = 64) { +class EdgePartitionBuilder[@specialized(Long, Int, Double) ED: ClassTag](size: Int = 64) { var edges = new PrimitiveVector[Edge[ED]](size) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/EdgeTripletIterator.scala b/graph/src/main/scala/org/apache/spark/graph/impl/EdgeTripletIterator.scala index c9e1e081534cb..4d5eb240a91b7 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/EdgeTripletIterator.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/EdgeTripletIterator.scala @@ -1,5 +1,7 @@ package org.apache.spark.graph.impl +import scala.reflect.ClassTag + import org.apache.spark.graph._ import org.apache.spark.util.collection.PrimitiveKeyOpenHashMap @@ -10,7 +12,7 @@ import org.apache.spark.util.collection.PrimitiveKeyOpenHashMap * debug / profile. */ private[impl] -class EdgeTripletIterator[VD: ClassManifest, ED: ClassManifest]( +class EdgeTripletIterator[VD: ClassTag, ED: ClassTag]( val vidToIndex: VertexIdToIndexMap, val vertexArray: Array[VD], val edgePartition: EdgePartition[ED]) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index 79c11c780a69a..1dfd9cf316efb 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -1,5 +1,7 @@ package org.apache.spark.graph.impl +import scala.reflect.{classTag, ClassTag} + import org.apache.spark.util.collection.PrimitiveVector import org.apache.spark.{HashPartitioner, Partitioner} import org.apache.spark.SparkContext._ @@ -23,7 +25,7 @@ import org.apache.spark.util.ClosureCleaner * edge partitions. `replicatedVertexView` stores a view of the replicated vertex attributes created * using the routing table. */ -class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( +class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( @transient val vertices: VertexRDD[VD], @transient val edges: EdgeRDD[ED], @transient val routingTable: RoutingTable, @@ -45,8 +47,8 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( /** Return a RDD that brings edges together with their source and destination vertices. */ @transient override val triplets: RDD[EdgeTriplet[VD, ED]] = { - val vdManifest = classManifest[VD] - val edManifest = classManifest[ED] + val vdManifest = classTag[VD] + val edManifest = classTag[ED] edges.zipEdgePartitions(replicatedVertexView.get(true, true)) { (pid, ePart, vPartIter) => val (_, vPart) = vPartIter.next() new EdgeTripletIterator(vPart.index, vPart.values, ePart)(vdManifest, edManifest) @@ -63,7 +65,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( override def partitionBy(partitionStrategy: PartitionStrategy): Graph[VD, ED] = { val numPartitions = edges.partitions.size - val edManifest = classManifest[ED] + val edManifest = classTag[ED] val newEdges = new EdgeRDD(edges.map { e => val part: Pid = partitionStrategy.getPartition(e.srcId, e.dstId, numPartitions) @@ -153,8 +155,8 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( new GraphImpl(vertices, newETable, routingTable, replicatedVertexView) } - override def mapVertices[VD2: ClassManifest](f: (Vid, VD) => VD2): Graph[VD2, ED] = { - if (classManifest[VD] equals classManifest[VD2]) { + override def mapVertices[VD2: ClassTag](f: (Vid, VD) => VD2): Graph[VD2, ED] = { + if (classTag[VD] equals classTag[VD2]) { // The map preserves type, so we can use incremental replication val newVerts = vertices.mapVertexPartitions(_.map(f)) val changedVerts = vertices.asInstanceOf[VertexRDD[VD2]].diff(newVerts) @@ -168,17 +170,17 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( } } - override def mapEdges[ED2: ClassManifest]( + override def mapEdges[ED2: ClassTag]( f: (Pid, Iterator[Edge[ED]]) => Iterator[ED2]): Graph[VD, ED2] = { val newETable = edges.mapEdgePartitions((pid, part) => part.map(f(pid, part.iterator))) new GraphImpl(vertices, newETable , routingTable, replicatedVertexView) } - override def mapTriplets[ED2: ClassManifest]( + override def mapTriplets[ED2: ClassTag]( f: (Pid, Iterator[EdgeTriplet[VD, ED]]) => Iterator[ED2]): Graph[VD, ED2] = { // Use an explicit manifest in PrimitiveKeyOpenHashMap init so we don't pull in the implicit // manifest from GraphImpl (which would require serializing GraphImpl). - val vdManifest = classManifest[VD] + val vdManifest = classTag[VD] val newEdgePartitions = edges.zipEdgePartitions(replicatedVertexView.get(true, true)) { (ePid, edgePartition, vTableReplicatedIter) => @@ -208,7 +210,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( val newVerts = vertices.mapVertexPartitions(_.filter(vpred)) // Filter the edges - val edManifest = classManifest[ED] + val edManifest = classTag[ED] val newEdges = new EdgeRDD[ED](triplets.filter { et => vpred(et.srcId, et.srcAttr) && vpred(et.dstId, et.dstAttr) && epred(et) }.mapPartitionsWithIndex( { (pid, iter) => @@ -224,7 +226,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( new GraphImpl(newVerts, newEdges, new RoutingTable(newEdges, newVerts), replicatedVertexView) } // end of subgraph - override def mask[VD2: ClassManifest, ED2: ClassManifest] ( + override def mask[VD2: ClassTag, ED2: ClassTag] ( other: Graph[VD2, ED2]): Graph[VD, ED] = { val newVerts = vertices.innerJoin(other.vertices) { (vid, v, w) => v } val newEdges = edges.innerJoin(other.edges) { (src, dst, v, w) => v } @@ -244,7 +246,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( // Lower level transformation methods ////////////////////////////////////////////////////////////////////////////////////////////////// - override def mapReduceTriplets[A: ClassManifest]( + override def mapReduceTriplets[A: ClassTag]( mapFunc: EdgeTriplet[VD, ED] => Iterator[(Vid, A)], reduceFunc: (A, A) => A, activeSetOpt: Option[(VertexRDD[_], EdgeDirection)] = None) = { @@ -311,9 +313,9 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( vertices.aggregateUsingIndex(preAgg, reduceFunc) } // end of mapReduceTriplets - override def outerJoinVertices[U: ClassManifest, VD2: ClassManifest] + override def outerJoinVertices[U: ClassTag, VD2: ClassTag] (updates: RDD[(Vid, U)])(updateF: (Vid, VD, Option[U]) => VD2): Graph[VD2, ED] = { - if (classManifest[VD] equals classManifest[VD2]) { + if (classTag[VD] equals classTag[VD2]) { // updateF preserves type, so we can use incremental replication val newVerts = vertices.leftJoin(updates)(updateF) val changedVerts = vertices.asInstanceOf[VertexRDD[VD2]].diff(newVerts) @@ -340,20 +342,20 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( object GraphImpl { - def apply[VD: ClassManifest, ED: ClassManifest]( + def apply[VD: ClassTag, ED: ClassTag]( edges: RDD[Edge[ED]], defaultVertexAttr: VD): GraphImpl[VD, ED] = { fromEdgeRDD(createEdgeRDD(edges), defaultVertexAttr) } - def fromEdgePartitions[VD: ClassManifest, ED: ClassManifest]( + def fromEdgePartitions[VD: ClassTag, ED: ClassTag]( edgePartitions: RDD[(Pid, EdgePartition[ED])], defaultVertexAttr: VD): GraphImpl[VD, ED] = { fromEdgeRDD(new EdgeRDD(edgePartitions), defaultVertexAttr) } - def apply[VD: ClassManifest, ED: ClassManifest]( + def apply[VD: ClassTag, ED: ClassTag]( vertices: RDD[(Vid, VD)], edges: RDD[Edge[ED]], defaultVertexAttr: VD): GraphImpl[VD, ED] = @@ -381,7 +383,7 @@ object GraphImpl { * pair: the key is the partition id, and the value is an EdgePartition object containing all the * edges in a partition. */ - private def createEdgeRDD[ED: ClassManifest]( + private def createEdgeRDD[ED: ClassTag]( edges: RDD[Edge[ED]]): EdgeRDD[ED] = { val edgePartitions = edges.mapPartitionsWithIndex { (pid, iter) => val builder = new EdgePartitionBuilder[ED] @@ -393,7 +395,7 @@ object GraphImpl { new EdgeRDD(edgePartitions) } - private def fromEdgeRDD[VD: ClassManifest, ED: ClassManifest]( + private def fromEdgeRDD[VD: ClassTag, ED: ClassTag]( edges: EdgeRDD[ED], defaultVertexAttr: VD): GraphImpl[VD, ED] = { edges.cache() diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/MessageToPartition.scala b/graph/src/main/scala/org/apache/spark/graph/impl/MessageToPartition.scala index c2e452cc72ab5..66fe796d2eae5 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/MessageToPartition.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/MessageToPartition.scala @@ -1,5 +1,7 @@ package org.apache.spark.graph.impl +import scala.reflect.{classTag, ClassTag} + import org.apache.spark.Partitioner import org.apache.spark.graph.{Pid, Vid} import org.apache.spark.rdd.{ShuffledRDD, RDD} @@ -37,16 +39,16 @@ class MessageToPartition[@specialized(Int, Long, Double, Char, Boolean/*, AnyRef } -class VertexBroadcastMsgRDDFunctions[T: ClassManifest](self: RDD[VertexBroadcastMsg[T]]) { +class VertexBroadcastMsgRDDFunctions[T: ClassTag](self: RDD[VertexBroadcastMsg[T]]) { def partitionBy(partitioner: Partitioner): RDD[VertexBroadcastMsg[T]] = { val rdd = new ShuffledRDD[Pid, (Vid, T), VertexBroadcastMsg[T]](self, partitioner) // Set a custom serializer if the data is of int or double type. - if (classManifest[T] == ClassManifest.Int) { + if (classTag[T] == ClassTag.Int) { rdd.setSerializer(classOf[IntVertexBroadcastMsgSerializer].getName) - } else if (classManifest[T] == ClassManifest.Long) { + } else if (classTag[T] == ClassTag.Long) { rdd.setSerializer(classOf[LongVertexBroadcastMsgSerializer].getName) - } else if (classManifest[T] == ClassManifest.Double) { + } else if (classTag[T] == ClassTag.Double) { rdd.setSerializer(classOf[DoubleVertexBroadcastMsgSerializer].getName) } rdd @@ -54,7 +56,7 @@ class VertexBroadcastMsgRDDFunctions[T: ClassManifest](self: RDD[VertexBroadcast } -class MsgRDDFunctions[T: ClassManifest](self: RDD[MessageToPartition[T]]) { +class MsgRDDFunctions[T: ClassTag](self: RDD[MessageToPartition[T]]) { /** * Return a copy of the RDD partitioned using the specified partitioner. @@ -67,23 +69,23 @@ class MsgRDDFunctions[T: ClassManifest](self: RDD[MessageToPartition[T]]) { object MsgRDDFunctions { - implicit def rdd2PartitionRDDFunctions[T: ClassManifest](rdd: RDD[MessageToPartition[T]]) = { + implicit def rdd2PartitionRDDFunctions[T: ClassTag](rdd: RDD[MessageToPartition[T]]) = { new MsgRDDFunctions(rdd) } - implicit def rdd2vertexMessageRDDFunctions[T: ClassManifest](rdd: RDD[VertexBroadcastMsg[T]]) = { + implicit def rdd2vertexMessageRDDFunctions[T: ClassTag](rdd: RDD[VertexBroadcastMsg[T]]) = { new VertexBroadcastMsgRDDFunctions(rdd) } - def partitionForAggregation[T: ClassManifest](msgs: RDD[(Vid, T)], partitioner: Partitioner) = { + def partitionForAggregation[T: ClassTag](msgs: RDD[(Vid, T)], partitioner: Partitioner) = { val rdd = new ShuffledRDD[Vid, T, (Vid, T)](msgs, partitioner) // Set a custom serializer if the data is of int or double type. - if (classManifest[T] == ClassManifest.Int) { + if (classTag[T] == ClassTag.Int) { rdd.setSerializer(classOf[IntAggMsgSerializer].getName) - } else if (classManifest[T] == ClassManifest.Long) { + } else if (classTag[T] == ClassTag.Long) { rdd.setSerializer(classOf[LongAggMsgSerializer].getName) - } else if (classManifest[T] == ClassManifest.Double) { + } else if (classTag[T] == ClassTag.Double) { rdd.setSerializer(classOf[DoubleAggMsgSerializer].getName) } rdd diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/ReplicatedVertexView.scala b/graph/src/main/scala/org/apache/spark/graph/impl/ReplicatedVertexView.scala index 175586b87eb16..2124144df792e 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/ReplicatedVertexView.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/ReplicatedVertexView.scala @@ -1,5 +1,7 @@ package org.apache.spark.graph.impl +import scala.reflect.{classTag, ClassTag} + import org.apache.spark.SparkContext._ import org.apache.spark.rdd.RDD import org.apache.spark.util.collection.{PrimitiveVector, OpenHashSet} @@ -17,7 +19,7 @@ import org.apache.spark.graph._ * example. */ private[impl] -class ReplicatedVertexView[VD: ClassManifest]( +class ReplicatedVertexView[VD: ClassTag]( updatedVerts: VertexRDD[VD], edges: EdgeRDD[_], routingTable: RoutingTable, @@ -80,7 +82,7 @@ class ReplicatedVertexView[VD: ClassManifest]( private def create(includeSrc: Boolean, includeDst: Boolean) : RDD[(Pid, VertexPartition[VD])] = { - val vdManifest = classManifest[VD] + val vdManifest = classTag[VD] // Ship vertex attributes to edge partitions according to vertexPlacement val verts = updatedVerts.partitionsRDD @@ -125,7 +127,7 @@ class ReplicatedVertexView[VD: ClassManifest]( } object ReplicatedVertexView { - protected def buildBuffer[VD: ClassManifest]( + protected def buildBuffer[VD: ClassTag]( pid2vidIter: Iterator[Array[Array[Vid]]], vertexPartIter: Iterator[VertexPartition[VD]]) = { val pid2vid: Array[Array[Vid]] = pid2vidIter.next() @@ -173,6 +175,6 @@ object ReplicatedVertexView { } } -class VertexAttributeBlock[VD: ClassManifest](val vids: Array[Vid], val attrs: Array[VD]) { +class VertexAttributeBlock[VD: ClassTag](val vids: Array[Vid], val attrs: Array[VD]) { def iterator: Iterator[(Vid, VD)] = (0 until vids.size).iterator.map { i => (vids(i), attrs(i)) } } diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala index 9b2d66999cc5a..7048a40f42364 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala @@ -1,5 +1,7 @@ package org.apache.spark.graph.impl +import scala.reflect.ClassTag + import org.apache.spark.util.collection.{BitSet, PrimitiveKeyOpenHashMap} import org.apache.spark.Logging @@ -8,7 +10,7 @@ import org.apache.spark.graph._ private[graph] object VertexPartition { - def apply[VD: ClassManifest](iter: Iterator[(Vid, VD)]): VertexPartition[VD] = { + def apply[VD: ClassTag](iter: Iterator[(Vid, VD)]): VertexPartition[VD] = { val map = new PrimitiveKeyOpenHashMap[Vid, VD] iter.foreach { case (k, v) => map(k) = v @@ -16,7 +18,7 @@ private[graph] object VertexPartition { new VertexPartition(map.keySet, map._values, map.keySet.getBitSet) } - def apply[VD: ClassManifest](iter: Iterator[(Vid, VD)], mergeFunc: (VD, VD) => VD) + def apply[VD: ClassTag](iter: Iterator[(Vid, VD)], mergeFunc: (VD, VD) => VD) : VertexPartition[VD] = { val map = new PrimitiveKeyOpenHashMap[Vid, VD] @@ -29,7 +31,7 @@ private[graph] object VertexPartition { private[graph] -class VertexPartition[@specialized(Long, Int, Double) VD: ClassManifest]( +class VertexPartition[@specialized(Long, Int, Double) VD: ClassTag]( val index: VertexIdToIndexMap, val values: Array[VD], val mask: BitSet, @@ -70,7 +72,7 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassManifest]( * each of the entries in the original VertexRDD. The resulting * VertexPartition retains the same index. */ - def map[VD2: ClassManifest](f: (Vid, VD) => VD2): VertexPartition[VD2] = { + def map[VD2: ClassTag](f: (Vid, VD) => VD2): VertexPartition[VD2] = { // Construct a view of the map transformation val newValues = new Array[VD2](capacity) var i = mask.nextSetBit(0) @@ -126,7 +128,7 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassManifest]( } /** Left outer join another VertexPartition. */ - def leftJoin[VD2: ClassManifest, VD3: ClassManifest] + def leftJoin[VD2: ClassTag, VD3: ClassTag] (other: VertexPartition[VD2]) (f: (Vid, VD, Option[VD2]) => VD3): VertexPartition[VD3] = { if (index != other.index) { @@ -146,14 +148,14 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassManifest]( } /** Left outer join another iterator of messages. */ - def leftJoin[VD2: ClassManifest, VD3: ClassManifest] + def leftJoin[VD2: ClassTag, VD3: ClassTag] (other: Iterator[(Vid, VD2)]) (f: (Vid, VD, Option[VD2]) => VD3): VertexPartition[VD3] = { leftJoin(createUsingIndex(other))(f) } /** Inner join another VertexPartition. */ - def innerJoin[U: ClassManifest, VD2: ClassManifest](other: VertexPartition[U]) + def innerJoin[U: ClassTag, VD2: ClassTag](other: VertexPartition[U]) (f: (Vid, VD, U) => VD2): VertexPartition[VD2] = { if (index != other.index) { logWarning("Joining two VertexPartitions with different indexes is slow.") @@ -173,7 +175,7 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassManifest]( /** * Inner join an iterator of messages. */ - def innerJoin[U: ClassManifest, VD2: ClassManifest] + def innerJoin[U: ClassTag, VD2: ClassTag] (iter: Iterator[Product2[Vid, U]]) (f: (Vid, VD, U) => VD2): VertexPartition[VD2] = { innerJoin(createUsingIndex(iter))(f) @@ -182,7 +184,7 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassManifest]( /** * Similar effect as aggregateUsingIndex((a, b) => a) */ - def createUsingIndex[VD2: ClassManifest](iter: Iterator[Product2[Vid, VD2]]) + def createUsingIndex[VD2: ClassTag](iter: Iterator[Product2[Vid, VD2]]) : VertexPartition[VD2] = { val newMask = new BitSet(capacity) val newValues = new Array[VD2](capacity) @@ -214,7 +216,7 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassManifest]( new VertexPartition(index, newValues, newMask) } - def aggregateUsingIndex[VD2: ClassManifest]( + def aggregateUsingIndex[VD2: ClassTag]( iter: Iterator[Product2[Vid, VD2]], reduceFunc: (VD2, VD2) => VD2): VertexPartition[VD2] = { val newMask = new BitSet(capacity) diff --git a/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala b/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala index a1e285816b9b7..d61f358bb08bd 100644 --- a/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala +++ b/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala @@ -1,10 +1,9 @@ package org.apache.spark.graph.util -import util._ -import math._ import scala.annotation.tailrec -//import scala.collection.mutable - +import scala.math._ +import scala.reflect.ClassTag +import scala.util._ import org.apache.spark._ import org.apache.spark.serializer._ @@ -155,7 +154,7 @@ object GraphGenerators { } - def outDegreeFromEdges[ED: ClassManifest](edges: RDD[Edge[ED]]): Graph[Int, ED] = { + def outDegreeFromEdges[ED: ClassTag](edges: RDD[Edge[ED]]): Graph[Int, ED] = { val vertices = edges.flatMap { edge => List((edge.srcId, 1)) } .reduceByKey(_ + _) @@ -281,14 +280,3 @@ object GraphGenerators { } // end of Graph Generators - - - - - - - - - - - diff --git a/graph/src/test/scala/org/apache/spark/graph/impl/EdgePartitionSuite.scala b/graph/src/test/scala/org/apache/spark/graph/impl/EdgePartitionSuite.scala index a52a5653e2cf6..f951fd7a82741 100644 --- a/graph/src/test/scala/org/apache/spark/graph/impl/EdgePartitionSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/impl/EdgePartitionSuite.scala @@ -1,5 +1,6 @@ package org.apache.spark.graph.impl +import scala.reflect.ClassTag import scala.util.Random import org.scalatest.FunSuite @@ -59,7 +60,7 @@ class EdgePartitionSuite extends FunSuite { } test("innerJoin") { - def makeEdgePartition[A: ClassManifest](xs: Iterable[(Int, Int, A)]): EdgePartition[A] = { + def makeEdgePartition[A: ClassTag](xs: Iterable[(Int, Int, A)]): EdgePartition[A] = { val builder = new EdgePartitionBuilder[A] for ((src, dst, attr) <- xs) { builder.add(src: Vid, dst: Vid, attr) } builder.toEdgePartition From 0ad75cdfb0093a0b525c598c5af4b9745581a6d7 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 3 Jan 2014 12:55:05 -0700 Subject: [PATCH 349/531] Manifest -> Tag in variable names --- .../scala/org/apache/spark/graph/EdgeRDD.scala | 6 +++--- .../org/apache/spark/graph/impl/GraphImpl.scala | 16 ++++++++-------- .../spark/graph/impl/ReplicatedVertexView.scala | 8 ++++---- 3 files changed, 15 insertions(+), 15 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala b/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala index 1c21967c9cab3..6f1d790325ce1 100644 --- a/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala @@ -76,10 +76,10 @@ class EdgeRDD[@specialized ED: ClassTag]( def innerJoin[ED2: ClassTag, ED3: ClassTag] (other: EdgeRDD[ED2]) (f: (Vid, Vid, ED, ED2) => ED3): EdgeRDD[ED3] = { - val ed2Manifest = classTag[ED2] - val ed3Manifest = classTag[ED3] + val ed2Tag = classTag[ED2] + val ed3Tag = classTag[ED3] zipEdgePartitions(other) { (pid, thisEPart, otherEPart) => - thisEPart.innerJoin(otherEPart)(f)(ed2Manifest, ed3Manifest) + thisEPart.innerJoin(otherEPart)(f)(ed2Tag, ed3Tag) } } diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index 1dfd9cf316efb..826c1074a85f6 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -47,11 +47,11 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( /** Return a RDD that brings edges together with their source and destination vertices. */ @transient override val triplets: RDD[EdgeTriplet[VD, ED]] = { - val vdManifest = classTag[VD] - val edManifest = classTag[ED] + val vdTag = classTag[VD] + val edTag = classTag[ED] edges.zipEdgePartitions(replicatedVertexView.get(true, true)) { (pid, ePart, vPartIter) => val (_, vPart) = vPartIter.next() - new EdgeTripletIterator(vPart.index, vPart.values, ePart)(vdManifest, edManifest) + new EdgeTripletIterator(vPart.index, vPart.values, ePart)(vdTag, edTag) } } @@ -65,7 +65,7 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( override def partitionBy(partitionStrategy: PartitionStrategy): Graph[VD, ED] = { val numPartitions = edges.partitions.size - val edManifest = classTag[ED] + val edTag = classTag[ED] val newEdges = new EdgeRDD(edges.map { e => val part: Pid = partitionStrategy.getPartition(e.srcId, e.dstId, numPartitions) @@ -74,7 +74,7 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( } .partitionBy(new HashPartitioner(numPartitions)) .mapPartitionsWithIndex( { (pid, iter) => - val builder = new EdgePartitionBuilder[ED]()(edManifest) + val builder = new EdgePartitionBuilder[ED]()(edTag) iter.foreach { message => val data = message.data builder.add(data._1, data._2, data._3) @@ -180,7 +180,7 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( f: (Pid, Iterator[EdgeTriplet[VD, ED]]) => Iterator[ED2]): Graph[VD, ED2] = { // Use an explicit manifest in PrimitiveKeyOpenHashMap init so we don't pull in the implicit // manifest from GraphImpl (which would require serializing GraphImpl). - val vdManifest = classTag[VD] + val vdTag = classTag[VD] val newEdgePartitions = edges.zipEdgePartitions(replicatedVertexView.get(true, true)) { (ePid, edgePartition, vTableReplicatedIter) => @@ -210,11 +210,11 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( val newVerts = vertices.mapVertexPartitions(_.filter(vpred)) // Filter the edges - val edManifest = classTag[ED] + val edTag = classTag[ED] val newEdges = new EdgeRDD[ED](triplets.filter { et => vpred(et.srcId, et.srcAttr) && vpred(et.dstId, et.dstAttr) && epred(et) }.mapPartitionsWithIndex( { (pid, iter) => - val builder = new EdgePartitionBuilder[ED]()(edManifest) + val builder = new EdgePartitionBuilder[ED]()(edTag) iter.foreach { et => builder.add(et.srcId, et.dstId, et.attr) } val edgePartition = builder.toEdgePartition Iterator((pid, edgePartition)) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/ReplicatedVertexView.scala b/graph/src/main/scala/org/apache/spark/graph/impl/ReplicatedVertexView.scala index 2124144df792e..033971c1af3c3 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/ReplicatedVertexView.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/ReplicatedVertexView.scala @@ -82,12 +82,12 @@ class ReplicatedVertexView[VD: ClassTag]( private def create(includeSrc: Boolean, includeDst: Boolean) : RDD[(Pid, VertexPartition[VD])] = { - val vdManifest = classTag[VD] + val vdTag = classTag[VD] // Ship vertex attributes to edge partitions according to vertexPlacement val verts = updatedVerts.partitionsRDD val shippedVerts = routingTable.get(includeSrc, includeDst) - .zipPartitions(verts)(ReplicatedVertexView.buildBuffer(_, _)(vdManifest)) + .zipPartitions(verts)(ReplicatedVertexView.buildBuffer(_, _)(vdTag)) .partitionBy(edges.partitioner.get) // TODO: Consider using a specialized shuffler. @@ -109,7 +109,7 @@ class ReplicatedVertexView[VD: ClassTag]( val (pid, vidToIndex) = mapIter.next() assert(!mapIter.hasNext) // Populate the vertex array using the vidToIndex map - val vertexArray = vdManifest.newArray(vidToIndex.capacity) + val vertexArray = vdTag.newArray(vidToIndex.capacity) for ((_, block) <- shippedVertsIter) { for (i <- 0 until block.vids.size) { val vid = block.vids(i) @@ -119,7 +119,7 @@ class ReplicatedVertexView[VD: ClassTag]( } } val newVPart = new VertexPartition( - vidToIndex, vertexArray, vidToIndex.getBitSet)(vdManifest) + vidToIndex, vertexArray, vidToIndex.getBitSet)(vdTag) Iterator((pid, newVPart)) }.cache().setName("ReplicatedVertexView %s %s".format(includeSrc, includeDst)) } From ab861d8450140cdb0a3d9f9b830ec076d8af746d Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 3 Jan 2014 13:25:03 -0700 Subject: [PATCH 350/531] Take SparkConf in constructor of Serializer subclasses --- .../apache/spark/graph/impl/Serializers.scala | 15 +++++----- .../apache/spark/graph/SerializerSuite.scala | 30 +++++++++++-------- 2 files changed, 26 insertions(+), 19 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala b/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala index e4fa4a4421786..dcf619fa85e8a 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala @@ -3,10 +3,11 @@ package org.apache.spark.graph.impl import java.io.{EOFException, InputStream, OutputStream} import java.nio.ByteBuffer +import org.apache.spark.SparkConf import org.apache.spark.graph._ import org.apache.spark.serializer._ -class VidMsgSerializer extends Serializer { +class VidMsgSerializer(conf: SparkConf) extends Serializer { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { @@ -26,7 +27,7 @@ class VidMsgSerializer extends Serializer { } /** A special shuffle serializer for VertexBroadcastMessage[Int]. */ -class IntVertexBroadcastMsgSerializer extends Serializer { +class IntVertexBroadcastMsgSerializer(conf: SparkConf) extends Serializer { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { @@ -49,7 +50,7 @@ class IntVertexBroadcastMsgSerializer extends Serializer { } /** A special shuffle serializer for VertexBroadcastMessage[Long]. */ -class LongVertexBroadcastMsgSerializer extends Serializer { +class LongVertexBroadcastMsgSerializer(conf: SparkConf) extends Serializer { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { @@ -72,7 +73,7 @@ class LongVertexBroadcastMsgSerializer extends Serializer { } /** A special shuffle serializer for VertexBroadcastMessage[Double]. */ -class DoubleVertexBroadcastMsgSerializer extends Serializer { +class DoubleVertexBroadcastMsgSerializer(conf: SparkConf) extends Serializer { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { @@ -95,7 +96,7 @@ class DoubleVertexBroadcastMsgSerializer extends Serializer { } /** A special shuffle serializer for AggregationMessage[Int]. */ -class IntAggMsgSerializer extends Serializer { +class IntAggMsgSerializer(conf: SparkConf) extends Serializer { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { @@ -118,7 +119,7 @@ class IntAggMsgSerializer extends Serializer { } /** A special shuffle serializer for AggregationMessage[Long]. */ -class LongAggMsgSerializer extends Serializer { +class LongAggMsgSerializer(conf: SparkConf) extends Serializer { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { @@ -141,7 +142,7 @@ class LongAggMsgSerializer extends Serializer { } /** A special shuffle serializer for AggregationMessage[Double]. */ -class DoubleAggMsgSerializer extends Serializer { +class DoubleAggMsgSerializer(conf: SparkConf) extends Serializer { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { diff --git a/graph/src/test/scala/org/apache/spark/graph/SerializerSuite.scala b/graph/src/test/scala/org/apache/spark/graph/SerializerSuite.scala index 80075f3437fb8..4014cbe440d8d 100644 --- a/graph/src/test/scala/org/apache/spark/graph/SerializerSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/SerializerSuite.scala @@ -15,14 +15,15 @@ import org.apache.spark.serializer.SerializationStream class SerializerSuite extends FunSuite with LocalSparkContext { test("IntVertexBroadcastMsgSerializer") { + val conf = new SparkConf(false) val outMsg = new VertexBroadcastMsg[Int](3, 4, 5) val bout = new ByteArrayOutputStream - val outStrm = new IntVertexBroadcastMsgSerializer().newInstance().serializeStream(bout) + val outStrm = new IntVertexBroadcastMsgSerializer(conf).newInstance().serializeStream(bout) outStrm.writeObject(outMsg) outStrm.writeObject(outMsg) bout.flush() val bin = new ByteArrayInputStream(bout.toByteArray) - val inStrm = new IntVertexBroadcastMsgSerializer().newInstance().deserializeStream(bin) + val inStrm = new IntVertexBroadcastMsgSerializer(conf).newInstance().deserializeStream(bin) val inMsg1: VertexBroadcastMsg[Int] = inStrm.readObject() val inMsg2: VertexBroadcastMsg[Int] = inStrm.readObject() assert(outMsg.vid === inMsg1.vid) @@ -36,14 +37,15 @@ class SerializerSuite extends FunSuite with LocalSparkContext { } test("LongVertexBroadcastMsgSerializer") { + val conf = new SparkConf(false) val outMsg = new VertexBroadcastMsg[Long](3, 4, 5) val bout = new ByteArrayOutputStream - val outStrm = new LongVertexBroadcastMsgSerializer().newInstance().serializeStream(bout) + val outStrm = new LongVertexBroadcastMsgSerializer(conf).newInstance().serializeStream(bout) outStrm.writeObject(outMsg) outStrm.writeObject(outMsg) bout.flush() val bin = new ByteArrayInputStream(bout.toByteArray) - val inStrm = new LongVertexBroadcastMsgSerializer().newInstance().deserializeStream(bin) + val inStrm = new LongVertexBroadcastMsgSerializer(conf).newInstance().deserializeStream(bin) val inMsg1: VertexBroadcastMsg[Long] = inStrm.readObject() val inMsg2: VertexBroadcastMsg[Long] = inStrm.readObject() assert(outMsg.vid === inMsg1.vid) @@ -57,14 +59,15 @@ class SerializerSuite extends FunSuite with LocalSparkContext { } test("DoubleVertexBroadcastMsgSerializer") { + val conf = new SparkConf(false) val outMsg = new VertexBroadcastMsg[Double](3, 4, 5.0) val bout = new ByteArrayOutputStream - val outStrm = new DoubleVertexBroadcastMsgSerializer().newInstance().serializeStream(bout) + val outStrm = new DoubleVertexBroadcastMsgSerializer(conf).newInstance().serializeStream(bout) outStrm.writeObject(outMsg) outStrm.writeObject(outMsg) bout.flush() val bin = new ByteArrayInputStream(bout.toByteArray) - val inStrm = new DoubleVertexBroadcastMsgSerializer().newInstance().deserializeStream(bin) + val inStrm = new DoubleVertexBroadcastMsgSerializer(conf).newInstance().deserializeStream(bin) val inMsg1: VertexBroadcastMsg[Double] = inStrm.readObject() val inMsg2: VertexBroadcastMsg[Double] = inStrm.readObject() assert(outMsg.vid === inMsg1.vid) @@ -78,14 +81,15 @@ class SerializerSuite extends FunSuite with LocalSparkContext { } test("IntAggMsgSerializer") { + val conf = new SparkConf(false) val outMsg = (4: Vid, 5) val bout = new ByteArrayOutputStream - val outStrm = new IntAggMsgSerializer().newInstance().serializeStream(bout) + val outStrm = new IntAggMsgSerializer(conf).newInstance().serializeStream(bout) outStrm.writeObject(outMsg) outStrm.writeObject(outMsg) bout.flush() val bin = new ByteArrayInputStream(bout.toByteArray) - val inStrm = new IntAggMsgSerializer().newInstance().deserializeStream(bin) + val inStrm = new IntAggMsgSerializer(conf).newInstance().deserializeStream(bin) val inMsg1: (Vid, Int) = inStrm.readObject() val inMsg2: (Vid, Int) = inStrm.readObject() assert(outMsg === inMsg1) @@ -97,14 +101,15 @@ class SerializerSuite extends FunSuite with LocalSparkContext { } test("LongAggMsgSerializer") { + val conf = new SparkConf(false) val outMsg = (4: Vid, 1L << 32) val bout = new ByteArrayOutputStream - val outStrm = new LongAggMsgSerializer().newInstance().serializeStream(bout) + val outStrm = new LongAggMsgSerializer(conf).newInstance().serializeStream(bout) outStrm.writeObject(outMsg) outStrm.writeObject(outMsg) bout.flush() val bin = new ByteArrayInputStream(bout.toByteArray) - val inStrm = new LongAggMsgSerializer().newInstance().deserializeStream(bin) + val inStrm = new LongAggMsgSerializer(conf).newInstance().deserializeStream(bin) val inMsg1: (Vid, Long) = inStrm.readObject() val inMsg2: (Vid, Long) = inStrm.readObject() assert(outMsg === inMsg1) @@ -116,14 +121,15 @@ class SerializerSuite extends FunSuite with LocalSparkContext { } test("DoubleAggMsgSerializer") { + val conf = new SparkConf(false) val outMsg = (4: Vid, 5.0) val bout = new ByteArrayOutputStream - val outStrm = new DoubleAggMsgSerializer().newInstance().serializeStream(bout) + val outStrm = new DoubleAggMsgSerializer(conf).newInstance().serializeStream(bout) outStrm.writeObject(outMsg) outStrm.writeObject(outMsg) bout.flush() val bin = new ByteArrayInputStream(bout.toByteArray) - val inStrm = new DoubleAggMsgSerializer().newInstance().deserializeStream(bin) + val inStrm = new DoubleAggMsgSerializer(conf).newInstance().deserializeStream(bin) val inMsg1: (Vid, Double) = inStrm.readObject() val inMsg2: (Vid, Double) = inStrm.readObject() assert(outMsg === inMsg1) From 74fdfac11266652ca87e05ae9b6510b75318728d Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Wed, 8 Jan 2014 11:45:31 -0800 Subject: [PATCH 351/531] Fix AbstractMethodError by inlining zip{Edge,Vertex}Partitions The zip{Edge,Vertex}Partitions methods created doubly-nested closures and passed them to zipPartitions. For some reason this caused an AbstractMethodError when zipPartitions tried to invoke the closure. This commit works around the problem by inlining these methods wherever they are called, eliminating the doubly-nested closure. --- .../org/apache/spark/graph/EdgeRDD.scala | 29 +++---------- .../org/apache/spark/graph/VertexRDD.scala | 43 +++++++++---------- .../apache/spark/graph/impl/GraphImpl.scala | 12 ++++-- 3 files changed, 35 insertions(+), 49 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala b/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala index 6f1d790325ce1..230202d6b0a6f 100644 --- a/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala @@ -53,34 +53,17 @@ class EdgeRDD[@specialized ED: ClassTag]( }, preservesPartitioning = true)) } - def zipEdgePartitions[T: ClassTag, U: ClassTag] - (other: RDD[T]) - (f: (Pid, EdgePartition[ED], Iterator[T]) => Iterator[U]): RDD[U] = { - partitionsRDD.zipPartitions(other, preservesPartitioning = true) { (ePartIter, otherIter) => - val (pid, edgePartition) = ePartIter.next() - f(pid, edgePartition, otherIter) - } - } - - def zipEdgePartitions[ED2: ClassTag, ED3: ClassTag] - (other: EdgeRDD[ED2]) - (f: (Pid, EdgePartition[ED], EdgePartition[ED2]) => EdgePartition[ED3]): EdgeRDD[ED3] = { - new EdgeRDD[ED3](partitionsRDD.zipPartitions(other.partitionsRDD, preservesPartitioning = true) { - (thisIter, otherIter) => - val (pid, thisEPart) = thisIter.next() - val (_, otherEPart) = otherIter.next() - Iterator(Tuple2(pid, f(pid, thisEPart, otherEPart))) - }) - } - def innerJoin[ED2: ClassTag, ED3: ClassTag] (other: EdgeRDD[ED2]) (f: (Vid, Vid, ED, ED2) => ED3): EdgeRDD[ED3] = { val ed2Tag = classTag[ED2] val ed3Tag = classTag[ED3] - zipEdgePartitions(other) { (pid, thisEPart, otherEPart) => - thisEPart.innerJoin(otherEPart)(f)(ed2Tag, ed3Tag) - } + new EdgeRDD[ED3](partitionsRDD.zipPartitions(other.partitionsRDD, true) { + (thisIter, otherIter) => + val (pid, thisEPart) = thisIter.next() + val (_, otherEPart) = otherIter.next() + Iterator(Tuple2(pid, thisEPart.innerJoin(otherEPart)(f)(ed2Tag, ed3Tag))) + }) } def collectVids(): RDD[Vid] = { diff --git a/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala b/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala index 8e5e319928274..c5fb4aeca73a0 100644 --- a/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala @@ -119,22 +119,6 @@ class VertexRDD[@specialized VD: ClassTag]( new VertexRDD(newPartitionsRDD) } - /** - * Return a new VertexRDD by applying a function to corresponding - * VertexPartitions of this VertexRDD and another one. - */ - def zipVertexPartitions[VD2: ClassTag, VD3: ClassTag] - (other: VertexRDD[VD2]) - (f: (VertexPartition[VD], VertexPartition[VD2]) => VertexPartition[VD3]): VertexRDD[VD3] = { - val newPartitionsRDD = partitionsRDD.zipPartitions( - other.partitionsRDD, preservesPartitioning = true - ) { (thisIter, otherIter) => - val thisPart = thisIter.next() - val otherPart = otherIter.next() - Iterator(f(thisPart, otherPart)) - } - new VertexRDD(newPartitionsRDD) - } /** * Restrict the vertex set to the set of vertices satisfying the @@ -184,9 +168,14 @@ class VertexRDD[@specialized VD: ClassTag]( * the values from `other`. */ def diff(other: VertexRDD[VD]): VertexRDD[VD] = { - this.zipVertexPartitions(other) { (thisPart, otherPart) => - thisPart.diff(otherPart) + val newPartitionsRDD = partitionsRDD.zipPartitions( + other.partitionsRDD, preservesPartitioning = true + ) { (thisIter, otherIter) => + val thisPart = thisIter.next() + val otherPart = otherIter.next() + Iterator(thisPart.diff(otherPart)) } + new VertexRDD(newPartitionsRDD) } /** @@ -209,9 +198,14 @@ class VertexRDD[@specialized VD: ClassTag]( */ def leftZipJoin[VD2: ClassTag, VD3: ClassTag] (other: VertexRDD[VD2])(f: (Vid, VD, Option[VD2]) => VD3): VertexRDD[VD3] = { - this.zipVertexPartitions(other) { (thisPart, otherPart) => - thisPart.leftJoin(otherPart)(f) + val newPartitionsRDD = partitionsRDD.zipPartitions( + other.partitionsRDD, preservesPartitioning = true + ) { (thisIter, otherIter) => + val thisPart = thisIter.next() + val otherPart = otherIter.next() + Iterator(thisPart.leftJoin(otherPart)(f)) } + new VertexRDD(newPartitionsRDD) } /** @@ -261,9 +255,14 @@ class VertexRDD[@specialized VD: ClassTag]( */ def innerZipJoin[U: ClassTag, VD2: ClassTag](other: VertexRDD[U]) (f: (Vid, VD, U) => VD2): VertexRDD[VD2] = { - this.zipVertexPartitions(other) { (thisPart, otherPart) => - thisPart.innerJoin(otherPart)(f) + val newPartitionsRDD = partitionsRDD.zipPartitions( + other.partitionsRDD, preservesPartitioning = true + ) { (thisIter, otherIter) => + val thisPart = thisIter.next() + val otherPart = otherIter.next() + Iterator(thisPart.innerJoin(otherPart)(f)) } + new VertexRDD(newPartitionsRDD) } /** diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index 826c1074a85f6..4d35755e7e5cc 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -49,7 +49,9 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( @transient override val triplets: RDD[EdgeTriplet[VD, ED]] = { val vdTag = classTag[VD] val edTag = classTag[ED] - edges.zipEdgePartitions(replicatedVertexView.get(true, true)) { (pid, ePart, vPartIter) => + edges.partitionsRDD.zipPartitions( + replicatedVertexView.get(true, true), true) { (ePartIter, vPartIter) => + val (pid, ePart) = ePartIter.next() val (_, vPart) = vPartIter.next() new EdgeTripletIterator(vPart.index, vPart.values, ePart)(vdTag, edTag) } @@ -182,8 +184,9 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( // manifest from GraphImpl (which would require serializing GraphImpl). val vdTag = classTag[VD] val newEdgePartitions = - edges.zipEdgePartitions(replicatedVertexView.get(true, true)) { - (ePid, edgePartition, vTableReplicatedIter) => + edges.partitionsRDD.zipPartitions(replicatedVertexView.get(true, true), true) { + (ePartIter, vTableReplicatedIter) => + val (ePid, edgePartition) = ePartIter.next() val (vPid, vPart) = vTableReplicatedIter.next() assert(!vTableReplicatedIter.hasNext) assert(ePid == vPid) @@ -267,7 +270,8 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( val activeDirectionOpt = activeSetOpt.map(_._2) // Map and combine. - val preAgg = edges.zipEdgePartitions(vs) { (ePid, edgePartition, vPartIter) => + val preAgg = edges.partitionsRDD.zipPartitions(vs, true) { (ePartIter, vPartIter) => + val (ePid, edgePartition) = ePartIter.next() val (vPid, vPart) = vPartIter.next() assert(!vPartIter.hasNext) assert(ePid == vPid) From 22374559a23adbcb5c286e0aadc7cd40c228726f Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Wed, 8 Jan 2014 22:48:54 -0800 Subject: [PATCH 352/531] Remove GraphX README --- README.md | 184 ++++++++++++++++-------------------------------------- 1 file changed, 53 insertions(+), 131 deletions(-) diff --git a/README.md b/README.md index 5b06d82225229..c840a68f76b17 100644 --- a/README.md +++ b/README.md @@ -1,143 +1,57 @@ -# GraphX: Unifying Graphs and Tables +# Apache Spark - -GraphX extends the distributed fault-tolerant collections API and -interactive console of [Spark](http://spark.incubator.apache.org) 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. - - -## Motivation - -From social networks and targeted advertising to protein modeling and -astrophysics, big graphs capture the structure in data and are central -to the recent advances in machine learning and data mining. Directly -applying existing *data-parallel* tools (e.g., -[Hadoop](http://hadoop.apache.org) and -[Spark](http://spark.incubator.apache.org)) to graph computation tasks -can be cumbersome and inefficient. The need for intuitive, scalable -tools for graph computation has lead to the development of new -*graph-parallel* systems (e.g., -[Pregel](http://http://giraph.apache.org) and -[GraphLab](http://graphlab.org)) which are designed to efficiently -execute graph algorithms. Unfortunately, these systems do not address -the challenges of graph construction and transformation and provide -limited fault-tolerance and support for interactive analysis. - -

- -

- - - -## Solution - -The GraphX project combines the advantages of both data-parallel and -graph-parallel systems by efficiently expressing graph computation -within the [Spark](http://spark.incubator.apache.org) framework. We -leverage new ideas in distributed graph representation to efficiently -distribute graphs as tabular data-structures. Similarly, we leverage -advances in data-flow systems to exploit in-memory computation and -fault-tolerance. We provide powerful new operations to simplify graph -construction and transformation. Using these primitives we implement -the PowerGraph and Pregel abstractions in less than 20 lines of code. -Finally, by exploiting the Scala foundation of Spark, we enable users -to interactively load, transform, and compute on massive graphs. - -

- -

- -## Examples - -Suppose I want to build a graph from some text files, restrict the graph -to important relationships and users, run page-rank on the sub-graph, and -then finally return attributes associated with the top users. I can do -all of this in just a few lines with GraphX: - -```scala -// 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) ) - -// Parse the edge data which is already in userId -> userId format -val followerGraph = Graph.textFile(sc, "hdfs://followers.tsv") - -// Attach the user attributes -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) - -// Compute the PageRank -val pagerankGraph = Analytics.pagerank(subgraph) - -// 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) - } - -println(userInfoWithPageRank.top(5)) - -``` +Lightning-Fast Cluster Computing - ## Online Documentation You can find the latest Spark documentation, including a programming -guide, on the project webpage at -. This README -file only contains basic setup instructions. +guide, on the project webpage at . +This README file only contains basic setup instructions. ## Building -Spark requires Scala 2.9.3 (Scala 2.10 is not yet supported). The -project is built using Simple Build Tool (SBT), which is packaged with -it. To build Spark and its example programs, run: +Spark requires Scala 2.10. The project is built using Simple Build Tool (SBT), +which can be obtained [here](http://www.scala-sbt.org). If SBT is installed we +will use the system version of sbt otherwise we will attempt to download it +automatically. To build Spark and its example programs, run: - sbt/sbt assembly + ./sbt/sbt assembly -Once you've built Spark, the easiest way to start using it is the -shell: +Once you've built Spark, the easiest way to start using it is the shell: - ./spark-shell + ./bin/spark-shell -Or, for the Python API, the Python shell (`./pyspark`). +Or, for the Python API, the Python shell (`./bin/pyspark`). -Spark also comes with several sample programs in the `examples` -directory. To run one of them, use `./run-example -`. For example: +Spark also comes with several sample programs in the `examples` directory. +To run one of them, use `./bin/run-example `. For example: - ./run-example org.apache.spark.examples.SparkLR local[2] + ./bin/run-example org.apache.spark.examples.SparkLR local[2] will run the Logistic Regression example locally on 2 CPUs. Each of the example programs prints usage help if no params are given. -All of the Spark samples take a `` parameter that is the -cluster URL to connect to. This can be a mesos:// or spark:// URL, or -"local" to run locally with one thread, or "local[N]" to run locally -with N threads. +All of the Spark samples take a `` parameter that is the cluster URL +to connect to. This can be a mesos:// or spark:// URL, or "local" to run +locally with one thread, or "local[N]" to run locally with N threads. + +## Running tests +Testing first requires [Building](#building) Spark. Once Spark is built, tests +can be run using: +`./sbt/sbt test` + ## A Note About Hadoop Versions -Spark uses the Hadoop core library to talk to HDFS and other -Hadoop-supported storage systems. Because the protocols have changed -in different versions of Hadoop, you must build Spark against the same -version that your cluster runs. You can change the version by setting -the `SPARK_HADOOP_VERSION` environment when building Spark. +Spark uses the Hadoop core library to talk to HDFS and other Hadoop-supported +storage systems. Because the protocols have changed in different versions of +Hadoop, you must build Spark against the same version that your cluster runs. +You can change the version by setting the `SPARK_HADOOP_VERSION` environment +when building Spark. For Apache Hadoop versions 1.x, Cloudera CDH MRv1, and other Hadoop versions without YARN, use: @@ -148,7 +62,7 @@ versions without YARN, use: # Cloudera CDH 4.2.0 with MapReduce v1 $ SPARK_HADOOP_VERSION=2.0.0-mr1-cdh4.2.0 sbt/sbt assembly -For Apache Hadoop 2.x, 0.23.x, Cloudera CDH MRv2, and other Hadoop versions +For Apache Hadoop 2.2.X, 2.1.X, 2.0.X, 0.23.x, Cloudera CDH MRv2, and other Hadoop versions with YARN, also set `SPARK_YARN=true`: # Apache Hadoop 2.0.5-alpha @@ -157,8 +71,8 @@ with YARN, also set `SPARK_YARN=true`: # Cloudera CDH 4.2.0 with MapReduce v2 $ SPARK_HADOOP_VERSION=2.0.0-cdh4.2.0 SPARK_YARN=true sbt/sbt assembly -For convenience, these variables may also be set through the -`conf/spark-env.sh` file described below. + # Apache Hadoop 2.2.X and newer + $ SPARK_HADOOP_VERSION=2.2.0 SPARK_YARN=true sbt/sbt assembly When developing a Spark application, specify the Hadoop version by adding the "hadoop-client" artifact to your project's dependencies. For example, if you're @@ -167,8 +81,7 @@ using Hadoop 1.2.1 and build your application using SBT, add this entry to "org.apache.hadoop" % "hadoop-client" % "1.2.1" -If your project is built with Maven, add this to your POM file's -`` section: +If your project is built with Maven, add this to your POM file's `` section: org.apache.hadoop @@ -179,19 +92,28 @@ If your project is built with Maven, add this to your POM file's ## Configuration -Please refer to the [Configuration -guide](http://spark.incubator.apache.org/docs/latest/configuration.html) +Please refer to the [Configuration guide](http://spark.incubator.apache.org/docs/latest/configuration.html) in the online documentation for an overview on how to configure Spark. -## Contributing to GraphX +## Apache Incubator Notice + +Apache Spark is an effort undergoing incubation at The Apache Software +Foundation (ASF), sponsored by the Apache Incubator. Incubation is required of +all newly accepted projects until a further review indicates that the +infrastructure, communications, and decision making process have stabilized in +a manner consistent with other successful ASF projects. While incubation status +is not necessarily a reflection of the completeness or stability of the code, +it does indicate that the project has yet to be fully endorsed by the ASF. + + +## Contributing to Spark -Contributions via GitHub pull requests are gladly accepted from their -original author. Along with any pull requests, please state that the -contribution is your original work and that you license the work to -the project under the project's open source license. Whether or not -you state this explicitly, by submitting any copyrighted material via -pull request, email, or other means you agree to license the material -under the project's open source license and warrant that you have the -legal authority to do so. +Contributions via GitHub pull requests are gladly accepted from their original +author. Along with any pull requests, please state that the contribution is +your original work and that you license the work to the project under the +project's open source license. Whether or not you state this explicitly, by +submitting any copyrighted material via pull request, email, or other means +you agree to license the material under the project's open source license and +warrant that you have the legal authority to do so. From 7309a29c755ae833f35523546e851be4c2f328be Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Thu, 9 Jan 2014 00:13:23 -0800 Subject: [PATCH 353/531] Removed Kryo dependency and graphx-shell --- .../apache/spark/util/collection/BitSet.scala | 2 +- .../spark/util/collection/OpenHashSet.scala | 2 +- .../scala/org/apache/spark/graph/Edge.scala | 2 +- .../spark/graph/impl/EdgePartition.scala | 2 +- .../spark/graph/impl/MessageToPartition.scala | 4 +- .../graph/impl/ReplicatedVertexView.scala | 3 +- graphx-shell | 124 ------------------ 7 files changed, 8 insertions(+), 131 deletions(-) delete mode 100755 graphx-shell diff --git a/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala b/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala index 0e12779152bda..f6e03bc0e190d 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala @@ -22,7 +22,7 @@ package org.apache.spark.util.collection * A simple, fixed-size bit set implementation. This implementation is fast because it avoids * safety/bound checking. */ -class BitSet(numBits: Int) { +class BitSet(numBits: Int) extends Serializable { private val words = new Array[Long](bit2words(numBits)) private val numWords = words.length diff --git a/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala b/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala index c908512b0f6db..895ccb9be0ca9 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala @@ -279,7 +279,7 @@ object OpenHashSet { * A set of specialized hash function implementation to avoid boxing hash code computation * in the specialized implementation of OpenHashSet. */ - sealed class Hasher[@specialized(Long, Int) T] { + sealed class Hasher[@specialized(Long, Int) T] extends Serializable { def hash(o: T): Int = o.hashCode() } diff --git a/graph/src/main/scala/org/apache/spark/graph/Edge.scala b/graph/src/main/scala/org/apache/spark/graph/Edge.scala index 7e8ae7c790b84..5ac77839eb866 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Edge.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Edge.scala @@ -19,7 +19,7 @@ case class Edge[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED] /** * The attribute associated with the edge. */ - var attr: ED = nullValue[ED]) { + var attr: ED = nullValue[ED]) extends Serializable { /** * Given one vertex in the edge return the other vertex. diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala b/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala index 7367269f67bfa..7ae4d7df43a2f 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala @@ -19,7 +19,7 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) val srcIds: Array[Vid], val dstIds: Array[Vid], val data: Array[ED], - val index: PrimitiveKeyOpenHashMap[Vid, Int]) { + val index: PrimitiveKeyOpenHashMap[Vid, Int]) extends Serializable { /** * Reverse all the edges in this partition. diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/MessageToPartition.scala b/graph/src/main/scala/org/apache/spark/graph/impl/MessageToPartition.scala index 66fe796d2eae5..bf033945dee24 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/MessageToPartition.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/MessageToPartition.scala @@ -11,7 +11,7 @@ class VertexBroadcastMsg[@specialized(Int, Long, Double, Boolean) T]( @transient var partition: Pid, var vid: Vid, var data: T) - extends Product2[Pid, (Vid, T)] { + extends Product2[Pid, (Vid, T)] with Serializable { override def _1 = partition @@ -29,7 +29,7 @@ class VertexBroadcastMsg[@specialized(Int, Long, Double, Boolean) T]( class MessageToPartition[@specialized(Int, Long, Double, Char, Boolean/*, AnyRef*/) T]( @transient var partition: Pid, var data: T) - extends Product2[Pid, T] { + extends Product2[Pid, T] with Serializable { override def _1 = partition diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/ReplicatedVertexView.scala b/graph/src/main/scala/org/apache/spark/graph/impl/ReplicatedVertexView.scala index 033971c1af3c3..970acfed27425 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/ReplicatedVertexView.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/ReplicatedVertexView.scala @@ -175,6 +175,7 @@ object ReplicatedVertexView { } } -class VertexAttributeBlock[VD: ClassTag](val vids: Array[Vid], val attrs: Array[VD]) { +class VertexAttributeBlock[VD: ClassTag](val vids: Array[Vid], val attrs: Array[VD]) + extends Serializable { def iterator: Iterator[(Vid, VD)] = (0 until vids.size).iterator.map { i => (vids(i), attrs(i)) } } diff --git a/graphx-shell b/graphx-shell deleted file mode 100755 index 4dd6c68ace888..0000000000000 --- a/graphx-shell +++ /dev/null @@ -1,124 +0,0 @@ -#!/usr/bin/env bash - -# -# 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. -# - -# -# Shell script for starting the Spark Shell REPL -# Note that it will set MASTER to spark://${SPARK_MASTER_IP}:${SPARK_MASTER_PORT} -# if those two env vars are set in spark-env.sh but MASTER is not. -# Options: -# -c Set the number of cores for REPL to use -# - -# Enter posix mode for bash -set -o posix - - -# Update the the banner logo -export SPARK_BANNER_TEXT="Welcome to - ______ __ _ __ - / ____/________ _____ / /_ | |/ / - / / __/ ___/ __ \`/ __ \/ __ \| / - / /_/ / / / /_/ / /_/ / / / / | - \____/_/ \__,_/ .___/_/ /_/_/|_| - /_/ Alpha Release - -Powered by: - ____ __ - / __/__ ___ _____/ /__ - _\ \/ _ \/ _ \`/ __/ '_/ - /___/ .__/\_,_/_/ /_/\_\ - /_/ version 0.9.0 - -Example: - - scala> val graph = GraphLoader.textFile(sc, \"hdfs://links\") - scala> graph.numVertices - scala> graph.numEdges - scala> val pageRankGraph = Analytics.pagerank(graph, 10) // 10 iterations - scala> val maxPr = pageRankGraph.vertices.map{ case (vid, pr) => pr }.max - scala> println(maxPr) - -" - -export SPARK_SHELL_INIT_BLOCK="import org.apache.spark.graph._;" - -# Set the serializer to use Kryo for graphx objects -SPARK_JAVA_OPTS+=" -Dspark.serializer=org.apache.spark.serializer.KryoSerializer " -SPARK_JAVA_OPTS+="-Dspark.kryo.registrator=org.apache.spark.graph.GraphKryoRegistrator " -SPARK_JAVA_OPTS+="-Dspark.kryoserializer.buffer.mb=10 " - - - -FWDIR="`dirname $0`" - -for o in "$@"; do - if [ "$1" = "-c" -o "$1" = "--cores" ]; then - shift - if [ -n "$1" ]; then - OPTIONS="-Dspark.cores.max=$1" - shift - fi - fi -done - -# Set MASTER from spark-env if possible -if [ -z "$MASTER" ]; then - if [ -e "$FWDIR/conf/spark-env.sh" ]; then - . "$FWDIR/conf/spark-env.sh" - fi - if [[ "x" != "x$SPARK_MASTER_IP" && "y" != "y$SPARK_MASTER_PORT" ]]; then - MASTER="spark://${SPARK_MASTER_IP}:${SPARK_MASTER_PORT}" - export MASTER - fi -fi - -# Copy restore-TTY-on-exit functions from Scala script so spark-shell exits properly even in -# binary distribution of Spark where Scala is not installed -exit_status=127 -saved_stty="" - -# restore stty settings (echo in particular) -function restoreSttySettings() { - stty $saved_stty - saved_stty="" -} - -function onExit() { - if [[ "$saved_stty" != "" ]]; then - restoreSttySettings - fi - exit $exit_status -} - -# to reenable echo if we are interrupted before completing. -trap onExit INT - -# save terminal settings -saved_stty=$(stty -g 2>/dev/null) -# clear on error so we don't later try to restore them -if [[ ! $? ]]; then - saved_stty="" -fi - -$FWDIR/spark-class $OPTIONS org.apache.spark.repl.Main "$@" - -# record the exit status lest it be overwritten: -# then reenable echo and propagate the code. -exit_status=$? -onExit From e4483582fc59330af8a43e8a152959f927103c79 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Thu, 9 Jan 2014 10:23:35 -0800 Subject: [PATCH 354/531] Add docs/graphx-programming-guide.md from 7210257ba3038d5e22d4b60fe9c3113dc45c3dff:README.md --- docs/graphx-programming-guide.md | 197 +++++++++++++++++++++++++++++++ 1 file changed, 197 insertions(+) create mode 100644 docs/graphx-programming-guide.md diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md new file mode 100644 index 0000000000000..5b06d82225229 --- /dev/null +++ b/docs/graphx-programming-guide.md @@ -0,0 +1,197 @@ +# GraphX: Unifying Graphs and Tables + + +GraphX extends the distributed fault-tolerant collections API and +interactive console of [Spark](http://spark.incubator.apache.org) 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. + + +## Motivation + +From social networks and targeted advertising to protein modeling and +astrophysics, big graphs capture the structure in data and are central +to the recent advances in machine learning and data mining. Directly +applying existing *data-parallel* tools (e.g., +[Hadoop](http://hadoop.apache.org) and +[Spark](http://spark.incubator.apache.org)) to graph computation tasks +can be cumbersome and inefficient. The need for intuitive, scalable +tools for graph computation has lead to the development of new +*graph-parallel* systems (e.g., +[Pregel](http://http://giraph.apache.org) and +[GraphLab](http://graphlab.org)) which are designed to efficiently +execute graph algorithms. Unfortunately, these systems do not address +the challenges of graph construction and transformation and provide +limited fault-tolerance and support for interactive analysis. + +

+ +

+ + + +## Solution + +The GraphX project combines the advantages of both data-parallel and +graph-parallel systems by efficiently expressing graph computation +within the [Spark](http://spark.incubator.apache.org) framework. We +leverage new ideas in distributed graph representation to efficiently +distribute graphs as tabular data-structures. Similarly, we leverage +advances in data-flow systems to exploit in-memory computation and +fault-tolerance. We provide powerful new operations to simplify graph +construction and transformation. Using these primitives we implement +the PowerGraph and Pregel abstractions in less than 20 lines of code. +Finally, by exploiting the Scala foundation of Spark, we enable users +to interactively load, transform, and compute on massive graphs. + +

+ +

+ +## Examples + +Suppose I want to build a graph from some text files, restrict the graph +to important relationships and users, run page-rank on the sub-graph, and +then finally return attributes associated with the top users. I can do +all of this in just a few lines with GraphX: + +```scala +// 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) ) + +// Parse the edge data which is already in userId -> userId format +val followerGraph = Graph.textFile(sc, "hdfs://followers.tsv") + +// Attach the user attributes +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) + +// Compute the PageRank +val pagerankGraph = Analytics.pagerank(subgraph) + +// 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) + } + +println(userInfoWithPageRank.top(5)) + +``` + + +## Online Documentation + +You can find the latest Spark documentation, including a programming +guide, on the project webpage at +. This README +file only contains basic setup instructions. + + +## Building + +Spark requires Scala 2.9.3 (Scala 2.10 is not yet supported). The +project is built using Simple Build Tool (SBT), which is packaged with +it. To build Spark and its example programs, run: + + sbt/sbt assembly + +Once you've built Spark, the easiest way to start using it is the +shell: + + ./spark-shell + +Or, for the Python API, the Python shell (`./pyspark`). + +Spark also comes with several sample programs in the `examples` +directory. To run one of them, use `./run-example +`. For example: + + ./run-example org.apache.spark.examples.SparkLR local[2] + +will run the Logistic Regression example locally on 2 CPUs. + +Each of the example programs prints usage help if no params are given. + +All of the Spark samples take a `` parameter that is the +cluster URL to connect to. This can be a mesos:// or spark:// URL, or +"local" to run locally with one thread, or "local[N]" to run locally +with N threads. + + +## A Note About Hadoop Versions + +Spark uses the Hadoop core library to talk to HDFS and other +Hadoop-supported storage systems. Because the protocols have changed +in different versions of Hadoop, you must build Spark against the same +version that your cluster runs. You can change the version by setting +the `SPARK_HADOOP_VERSION` environment when building Spark. + +For Apache Hadoop versions 1.x, Cloudera CDH MRv1, and other Hadoop +versions without YARN, use: + + # Apache Hadoop 1.2.1 + $ SPARK_HADOOP_VERSION=1.2.1 sbt/sbt assembly + + # Cloudera CDH 4.2.0 with MapReduce v1 + $ SPARK_HADOOP_VERSION=2.0.0-mr1-cdh4.2.0 sbt/sbt assembly + +For Apache Hadoop 2.x, 0.23.x, Cloudera CDH MRv2, and other Hadoop versions +with YARN, also set `SPARK_YARN=true`: + + # Apache Hadoop 2.0.5-alpha + $ SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_YARN=true sbt/sbt assembly + + # Cloudera CDH 4.2.0 with MapReduce v2 + $ SPARK_HADOOP_VERSION=2.0.0-cdh4.2.0 SPARK_YARN=true sbt/sbt assembly + +For convenience, these variables may also be set through the +`conf/spark-env.sh` file described below. + +When developing a Spark application, specify the Hadoop version by adding the +"hadoop-client" artifact to your project's dependencies. For example, if you're +using Hadoop 1.2.1 and build your application using SBT, add this entry to +`libraryDependencies`: + + "org.apache.hadoop" % "hadoop-client" % "1.2.1" + +If your project is built with Maven, add this to your POM file's +`` section: + + + org.apache.hadoop + hadoop-client + 1.2.1 + + + +## Configuration + +Please refer to the [Configuration +guide](http://spark.incubator.apache.org/docs/latest/configuration.html) +in the online documentation for an overview on how to configure Spark. + + +## Contributing to GraphX + +Contributions via GitHub pull requests are gladly accepted from their +original author. Along with any pull requests, please state that the +contribution is your original work and that you license the work to +the project under the project's open source license. Whether or not +you state this explicitly, by submitting any copyrighted material via +pull request, email, or other means you agree to license the material +under the project's open source license and warrant that you have the +legal authority to do so. + From b5b0de2de53563c43e1c5844a52b4eeeb2542ea5 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Thu, 9 Jan 2014 13:24:25 -0800 Subject: [PATCH 355/531] Start fixing formatting of graphx-programming-guide --- docs/graphx-programming-guide.md | 13 ++++++------- 1 file changed, 6 insertions(+), 7 deletions(-) diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md index 5b06d82225229..ebc47f5d1c43c 100644 --- a/docs/graphx-programming-guide.md +++ b/docs/graphx-programming-guide.md @@ -1,4 +1,7 @@ -# GraphX: Unifying Graphs and Tables +--- +layout: global +title: "GraphX: Unifying Graphs and Tables" +--- GraphX extends the distributed fault-tolerant collections API and @@ -26,11 +29,8 @@ execute graph algorithms. Unfortunately, these systems do not address the challenges of graph construction and transformation and provide limited fault-tolerance and support for interactive analysis. -

- -

- - +{:.pagination-centered} +![Data-parallel vs. graph-parallel]({{ site.url }}/img/data_parallel_vs_graph_parallel.png) ## Solution @@ -194,4 +194,3 @@ you state this explicitly, by submitting any copyrighted material via pull request, email, or other means you agree to license the material under the project's open source license and warrant that you have the legal authority to do so. - From 4aa9be16333b61139fbe81a079f46ef32df0b1fd Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Thu, 9 Jan 2014 13:26:17 -0800 Subject: [PATCH 356/531] Make GraphImpl serializable to work around capture --- .../src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index 4d35755e7e5cc..2ce5404e94744 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -30,7 +30,7 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( @transient val edges: EdgeRDD[ED], @transient val routingTable: RoutingTable, @transient val replicatedVertexView: ReplicatedVertexView[VD]) - extends Graph[VD, ED] { + extends Graph[VD, ED] with Serializable { def this( vertices: VertexRDD[VD], From 3b2e22e2c343ce4615f31c3d94f9af568ea0ea42 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Thu, 9 Jan 2014 13:27:40 -0800 Subject: [PATCH 357/531] Revert changes to examples/.../PageRankUtils.scala Reverts to 04d83fc37f9eef89c20331c85291a0a169f75e6d:examples/src/main/scala/org/apache/spark/examples/bagel/PageRankUtils.scala. --- .../org/apache/spark/examples/bagel/PageRankUtils.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/bagel/PageRankUtils.scala b/examples/src/main/scala/org/apache/spark/examples/bagel/PageRankUtils.scala index 8dd7fb40e86aa..cfafbaf23e4c2 100644 --- a/examples/src/main/scala/org/apache/spark/examples/bagel/PageRankUtils.scala +++ b/examples/src/main/scala/org/apache/spark/examples/bagel/PageRankUtils.scala @@ -31,16 +31,16 @@ import java.io.{InputStream, OutputStream, DataInputStream, DataOutputStream} import com.esotericsoftware.kryo._ class PageRankUtils extends Serializable { - def computeWithCombiner(numVertices: Long, epsilon: Double, terminateSteps: Int = 10)( + def computeWithCombiner(numVertices: Long, epsilon: Double)( self: PRVertex, messageSum: Option[Double], superstep: Int ): (PRVertex, Array[PRMessage]) = { val newValue = messageSum match { case Some(msgSum) if msgSum != 0 => - 0.15 + 0.85 * msgSum + 0.15 / numVertices + 0.85 * msgSum case _ => self.value } - val terminate = superstep >= terminateSteps + val terminate = superstep >= 10 val outbox: Array[PRMessage] = if (!terminate) From ec12c63409c2db85c27a87813a8d0505ea8f6c21 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Thu, 9 Jan 2014 13:29:20 -0800 Subject: [PATCH 358/531] Unwrap Graph.mapEdges signature --- graph/src/main/scala/org/apache/spark/graph/Graph.scala | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/Graph.scala b/graph/src/main/scala/org/apache/spark/graph/Graph.scala index 9dd26f7679a2d..420d01b426180 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Graph.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Graph.scala @@ -169,9 +169,7 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] { * @tparam ED2 the new edge data type * */ - def mapEdges[ED2: ClassTag]( - map: (Pid, Iterator[Edge[ED]]) => Iterator[ED2]): - Graph[VD, ED2] + def mapEdges[ED2: ClassTag](map: (Pid, Iterator[Edge[ED]]) => Iterator[ED2]): Graph[VD, ED2] /** * Construct a new graph where the value of each edge is From da83038234de1a16de38a24633c73fd950d4a85f Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Thu, 9 Jan 2014 13:52:07 -0800 Subject: [PATCH 359/531] Vid -> VertexID --- .../scala/org/apache/spark/graph/Edge.scala | 10 ++-- .../org/apache/spark/graph/EdgeRDD.scala | 4 +- .../org/apache/spark/graph/EdgeTriplet.scala | 4 +- .../scala/org/apache/spark/graph/Graph.scala | 18 +++---- .../spark/graph/GraphKryoRegistrator.scala | 2 +- .../org/apache/spark/graph/GraphLab.scala | 24 +++++----- .../org/apache/spark/graph/GraphOps.scala | 30 +++++++----- .../spark/graph/PartitionStrategy.scala | 14 +++--- .../scala/org/apache/spark/graph/Pregel.scala | 8 ++-- .../org/apache/spark/graph/VertexRDD.scala | 42 ++++++++--------- .../algorithms/ConnectedComponents.scala | 4 +- .../spark/graph/algorithms/PageRank.scala | 4 +- .../StronglyConnectedComponents.scala | 6 +-- .../apache/spark/graph/algorithms/Svdpp.scala | 12 ++--- .../graph/algorithms/TriangleCount.scala | 2 +- .../spark/graph/impl/EdgePartition.scala | 16 +++---- .../graph/impl/EdgePartitionBuilder.scala | 10 ++-- .../graph/impl/EdgeTripletIterator.scala | 2 +- .../apache/spark/graph/impl/GraphImpl.scala | 32 ++++++------- .../spark/graph/impl/MessageToPartition.scala | 12 ++--- .../graph/impl/ReplicatedVertexView.scala | 29 ++++++------ .../spark/graph/impl/RoutingTable.scala | 16 +++---- .../apache/spark/graph/impl/Serializers.scala | 10 ++-- .../spark/graph/impl/VertexPartition.scala | 47 ++++++++++--------- .../org/apache/spark/graph/package.scala | 6 +-- .../spark/graph/util/GraphGenerators.scala | 12 ++--- .../apache/spark/graph/GraphOpsSuite.scala | 18 +++---- .../org/apache/spark/graph/GraphSuite.scala | 31 ++++++------ .../org/apache/spark/graph/PregelSuite.scala | 10 ++-- .../apache/spark/graph/SerializerSuite.scala | 18 +++---- .../spark/graph/impl/EdgePartitionSuite.scala | 2 +- 31 files changed, 234 insertions(+), 221 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/Edge.scala b/graph/src/main/scala/org/apache/spark/graph/Edge.scala index 5ac77839eb866..19c28bea685ba 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Edge.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Edge.scala @@ -11,11 +11,11 @@ case class Edge[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED] /** * The vertex id of the source vertex */ - var srcId: Vid = 0, + var srcId: VertexID = 0, /** * The vertex id of the target vertex. */ - var dstId: Vid = 0, + var dstId: VertexID = 0, /** * The attribute associated with the edge. */ @@ -27,7 +27,7 @@ case class Edge[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED] * @param vid the id one of the two vertices on the edge. * @return the id of the other vertex on the edge. */ - def otherVertexId(vid: Vid): Vid = + def otherVertexId(vid: VertexID): VertexID = if (srcId == vid) dstId else { assert(dstId == vid); srcId } /** @@ -38,13 +38,13 @@ case class Edge[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED] * @return the relative direction of the edge to the corresponding * vertex. */ - def relativeDirection(vid: Vid): EdgeDirection = + def relativeDirection(vid: VertexID): EdgeDirection = if (vid == srcId) EdgeDirection.Out else { assert(vid == dstId); EdgeDirection.In } } object Edge { def lexicographicOrdering[ED] = new Ordering[Edge[ED]] { override def compare(a: Edge[ED], b: Edge[ED]): Int = - Ordering[(Vid, Vid)].compare((a.srcId, a.dstId), (b.srcId, b.dstId)) + Ordering[(VertexID, VertexID)].compare((a.srcId, a.dstId), (b.srcId, b.dstId)) } } diff --git a/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala b/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala index 230202d6b0a6f..fd933593523a8 100644 --- a/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala @@ -55,7 +55,7 @@ class EdgeRDD[@specialized ED: ClassTag]( def innerJoin[ED2: ClassTag, ED3: ClassTag] (other: EdgeRDD[ED2]) - (f: (Vid, Vid, ED, ED2) => ED3): EdgeRDD[ED3] = { + (f: (VertexID, VertexID, ED, ED2) => ED3): EdgeRDD[ED3] = { val ed2Tag = classTag[ED2] val ed3Tag = classTag[ED3] new EdgeRDD[ED3](partitionsRDD.zipPartitions(other.partitionsRDD, true) { @@ -66,7 +66,7 @@ class EdgeRDD[@specialized ED: ClassTag]( }) } - def collectVids(): RDD[Vid] = { + def collectVertexIDs(): RDD[VertexID] = { partitionsRDD.flatMap { case (_, p) => Array.concat(p.srcIds, p.dstIds) } } diff --git a/graph/src/main/scala/org/apache/spark/graph/EdgeTriplet.scala b/graph/src/main/scala/org/apache/spark/graph/EdgeTriplet.scala index 5a384a5f84dd9..a5103ed3cbffc 100644 --- a/graph/src/main/scala/org/apache/spark/graph/EdgeTriplet.scala +++ b/graph/src/main/scala/org/apache/spark/graph/EdgeTriplet.scala @@ -47,7 +47,7 @@ class EdgeTriplet[VD, ED] extends Edge[ED] { * @param vid the id one of the two vertices on the edge. * @return the attribute for the other vertex on the edge. */ - def otherVertexAttr(vid: Vid): VD = + def otherVertexAttr(vid: VertexID): VD = if (srcId == vid) dstAttr else { assert(dstId == vid); srcAttr } /** @@ -56,7 +56,7 @@ class EdgeTriplet[VD, ED] extends Edge[ED] { * @param vid the id of one of the two vertices on the edge * @return the attr for the vertex with that id. */ - def vertexAttr(vid: Vid): VD = + def vertexAttr(vid: VertexID): VD = if (srcId == vid) srcAttr else { assert(dstId == vid); dstAttr } override def toString() = ((srcId, srcAttr), (dstId, dstAttr), attr).toString() diff --git a/graph/src/main/scala/org/apache/spark/graph/Graph.scala b/graph/src/main/scala/org/apache/spark/graph/Graph.scala index 420d01b426180..dd0799142ef74 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Graph.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Graph.scala @@ -125,7 +125,7 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] { * }}} * */ - def mapVertices[VD2: ClassTag](map: (Vid, VD) => VD2): Graph[VD2, ED] + def mapVertices[VD2: ClassTag](map: (VertexID, VD) => VD2): Graph[VD2, ED] /** * Construct a new graph where the value of each edge is @@ -253,7 +253,7 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] { * satisfy the predicates. */ def subgraph(epred: EdgeTriplet[VD,ED] => Boolean = (x => true), - vpred: (Vid, VD) => Boolean = ((v,d) => true) ): Graph[VD, ED] + vpred: (VertexID, VD) => Boolean = ((v,d) => true) ): Graph[VD, ED] /** * Subgraph of this graph with only vertices and edges from the other graph. @@ -302,7 +302,7 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] { * vertex * {{{ * val rawGraph: Graph[(),()] = Graph.textFile("twittergraph") - * val inDeg: RDD[(Vid, Int)] = + * val inDeg: RDD[(VertexID, Int)] = * mapReduceTriplets[Int](et => Array((et.dst.id, 1)), _ + _) * }}} * @@ -314,7 +314,7 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] { * */ def mapReduceTriplets[A: ClassTag]( - mapFunc: EdgeTriplet[VD, ED] => Iterator[(Vid, A)], + mapFunc: EdgeTriplet[VD, ED] => Iterator[(VertexID, A)], reduceFunc: (A, A) => A, activeSetOpt: Option[(VertexRDD[_], EdgeDirection)] = None) : VertexRDD[A] @@ -341,15 +341,15 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] { * * {{{ * val rawGraph: Graph[(),()] = Graph.textFile("webgraph") - * val outDeg: RDD[(Vid, Int)] = rawGraph.outDegrees() + * val outDeg: RDD[(VertexID, Int)] = rawGraph.outDegrees() * val graph = rawGraph.outerJoinVertices(outDeg) { * (vid, data, optDeg) => optDeg.getOrElse(0) * } * }}} * */ - def outerJoinVertices[U: ClassTag, VD2: ClassTag](table: RDD[(Vid, U)]) - (mapFunc: (Vid, VD, Option[U]) => VD2) + def outerJoinVertices[U: ClassTag, VD2: ClassTag](table: RDD[(VertexID, U)]) + (mapFunc: (VertexID, VD, Option[U]) => VD2) : Graph[VD2, ED] // Save a copy of the GraphOps object so there is always one unique GraphOps object @@ -377,7 +377,7 @@ object Graph { * (if `uniqueEdges=None`) and vertex attributes containing the total degree of each vertex. */ def fromEdgeTuples[VD: ClassTag]( - rawEdges: RDD[(Vid, Vid)], + rawEdges: RDD[(VertexID, VertexID)], defaultValue: VD, uniqueEdges: Option[PartitionStrategy] = None): Graph[VD, Int] = { val edges = rawEdges.map(p => Edge(p._1, p._2, 1)) @@ -419,7 +419,7 @@ object Graph { * partitioning the edges. */ def apply[VD: ClassTag, ED: ClassTag]( - vertices: RDD[(Vid, VD)], + vertices: RDD[(VertexID, VD)], edges: RDD[Edge[ED]], defaultVertexAttr: VD = null.asInstanceOf[VD]): Graph[VD, ED] = { GraphImpl(vertices, edges, defaultVertexAttr) diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphKryoRegistrator.scala b/graph/src/main/scala/org/apache/spark/graph/GraphKryoRegistrator.scala index b8c1b5b0f032d..296f3848f1c61 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphKryoRegistrator.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphKryoRegistrator.scala @@ -14,7 +14,7 @@ class GraphKryoRegistrator extends KryoRegistrator { kryo.register(classOf[Edge[Object]]) kryo.register(classOf[MessageToPartition[Object]]) kryo.register(classOf[VertexBroadcastMsg[Object]]) - kryo.register(classOf[(Vid, Object)]) + kryo.register(classOf[(VertexID, Object)]) kryo.register(classOf[EdgePartition[Object]]) kryo.register(classOf[BitSet]) kryo.register(classOf[VertexIdToIndexMap]) diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala b/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala index c1ce5cd9ccada..22f48540193ea 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala @@ -42,11 +42,12 @@ object GraphLab extends Logging { (graph: Graph[VD, ED], numIter: Int, gatherDirection: EdgeDirection = EdgeDirection.In, scatterDirection: EdgeDirection = EdgeDirection.Out) - (gatherFunc: (Vid, EdgeTriplet[VD, ED]) => A, + (gatherFunc: (VertexID, EdgeTriplet[VD, ED]) => A, mergeFunc: (A, A) => A, - applyFunc: (Vid, VD, Option[A]) => VD, - scatterFunc: (Vid, EdgeTriplet[VD, ED]) => Boolean, - startVertices: (Vid, VD) => Boolean = (vid: Vid, data: VD) => true): Graph[VD, ED] = { + applyFunc: (VertexID, VD, Option[A]) => VD, + scatterFunc: (VertexID, EdgeTriplet[VD, ED]) => Boolean, + startVertices: (VertexID, VD) => Boolean = (vid: VertexID, data: VD) => true) + : Graph[VD, ED] = { // Add an active attribute to all vertices to track convergence. @@ -56,7 +57,7 @@ object GraphLab extends Logging { // The gather function wrapper strips the active attribute and // only invokes the gather function on active vertices - def gather(vid: Vid, e: EdgeTriplet[(Boolean, VD), ED]): Option[A] = { + def gather(vid: VertexID, e: EdgeTriplet[(Boolean, VD), ED]): Option[A] = { if (e.vertexAttr(vid)._1) { val edgeTriplet = new EdgeTriplet[VD,ED] edgeTriplet.set(e) @@ -70,7 +71,7 @@ object GraphLab extends Logging { // The apply function wrapper strips the vertex of the active attribute // and only invokes the apply function on active vertices - def apply(vid: Vid, data: (Boolean, VD), accum: Option[A]): (Boolean, VD) = { + def apply(vid: VertexID, data: (Boolean, VD), accum: Option[A]): (Boolean, VD) = { val (active, vData) = data if (active) (true, applyFunc(vid, vData, accum)) else (false, vData) @@ -78,8 +79,8 @@ object GraphLab extends Logging { // The scatter function wrapper strips the vertex of the active attribute // and only invokes the scatter function on active vertices - def scatter(rawVid: Vid, e: EdgeTriplet[(Boolean, VD), ED]): Option[Boolean] = { - val vid = e.otherVertexId(rawVid) + def scatter(rawVertexID: VertexID, e: EdgeTriplet[(Boolean, VD), ED]): Option[Boolean] = { + val vid = e.otherVertexId(rawVertexID) if (e.vertexAttr(vid)._1) { val edgeTriplet = new EdgeTriplet[VD,ED] edgeTriplet.set(e) @@ -92,7 +93,8 @@ object GraphLab extends Logging { } // Used to set the active status of vertices for the next round - def applyActive(vid: Vid, data: (Boolean, VD), newActiveOpt: Option[Boolean]): (Boolean, VD) = { + def applyActive( + vid: VertexID, data: (Boolean, VD), newActiveOpt: Option[Boolean]): (Boolean, VD) = { val (prevActive, vData) = data (newActiveOpt.getOrElse(false), vData) } @@ -103,7 +105,7 @@ object GraphLab extends Logging { while (i < numIter && numActive > 0) { // Gather - val gathered: RDD[(Vid, A)] = + val gathered: RDD[(VertexID, A)] = activeGraph.aggregateNeighbors(gather, mergeFunc, gatherDirection) // Apply @@ -113,7 +115,7 @@ object GraphLab extends Logging { // Scatter is basically a gather in the opposite direction so we reverse the edge direction // activeGraph: Graph[(Boolean, VD), ED] - val scattered: RDD[(Vid, Boolean)] = + val scattered: RDD[(VertexID, Boolean)] = activeGraph.aggregateNeighbors(scatter, _ || _, scatterDirection.reverse) activeGraph = activeGraph.outerJoinVertices(scattered)(applyActive).cache() diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala b/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala index 11c6120beb414..e41287c1ed4f1 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala +++ b/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala @@ -112,7 +112,7 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) { * */ def aggregateNeighbors[A: ClassTag]( - mapFunc: (Vid, EdgeTriplet[VD, ED]) => Option[A], + mapFunc: (VertexID, EdgeTriplet[VD, ED]) => Option[A], reduceFunc: (A, A) => A, dir: EdgeDirection) : VertexRDD[A] = { @@ -151,25 +151,27 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) { * @return the vertex set of neighboring ids for each vertex. */ def collectNeighborIds(edgeDirection: EdgeDirection) : - VertexRDD[Array[Vid]] = { + VertexRDD[Array[VertexID]] = { val nbrs = if (edgeDirection == EdgeDirection.Both) { - graph.mapReduceTriplets[Array[Vid]]( + graph.mapReduceTriplets[Array[VertexID]]( mapFunc = et => Iterator((et.srcId, Array(et.dstId)), (et.dstId, Array(et.srcId))), reduceFunc = _ ++ _ ) } else if (edgeDirection == EdgeDirection.Out) { - graph.mapReduceTriplets[Array[Vid]]( + graph.mapReduceTriplets[Array[VertexID]]( mapFunc = et => Iterator((et.srcId, Array(et.dstId))), reduceFunc = _ ++ _) } else if (edgeDirection == EdgeDirection.In) { - graph.mapReduceTriplets[Array[Vid]]( + graph.mapReduceTriplets[Array[VertexID]]( 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.") } - graph.vertices.leftZipJoin(nbrs) { (vid, vdata, nbrsOpt) => nbrsOpt.getOrElse(Array.empty[Vid]) } + graph.vertices.leftZipJoin(nbrs) { (vid, vdata, nbrsOpt) => + nbrsOpt.getOrElse(Array.empty[VertexID]) + } } // end of collectNeighborIds @@ -187,14 +189,16 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) { * vertex. */ def collectNeighbors(edgeDirection: EdgeDirection) : - VertexRDD[ Array[(Vid, VD)] ] = { - val nbrs = graph.aggregateNeighbors[Array[(Vid,VD)]]( + VertexRDD[ Array[(VertexID, VD)] ] = { + val nbrs = graph.aggregateNeighbors[Array[(VertexID,VD)]]( (vid, edge) => Some(Array( (edge.otherVertexId(vid), edge.otherVertexAttr(vid)) )), (a, b) => a ++ b, edgeDirection) - graph.vertices.leftZipJoin(nbrs) { (vid, vdata, nbrsOpt) => nbrsOpt.getOrElse(Array.empty[(Vid, VD)]) } + graph.vertices.leftZipJoin(nbrs) { (vid, vdata, nbrsOpt) => + nbrsOpt.getOrElse(Array.empty[(VertexID, VD)]) + } } // end of collectNeighbor @@ -228,9 +232,9 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) { * }}} * */ - def joinVertices[U: ClassTag](table: RDD[(Vid, U)])(mapFunc: (Vid, VD, U) => VD) + def joinVertices[U: ClassTag](table: RDD[(VertexID, U)])(mapFunc: (VertexID, VD, U) => VD) : Graph[VD, ED] = { - val uf = (id: Vid, data: VD, o: Option[U]) => { + val uf = (id: VertexID, data: VD, o: Option[U]) => { o match { case Some(u) => mapFunc(id, data, u) case None => data @@ -259,7 +263,7 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) { * val degrees: VertexSetRDD[Int] = graph.outDegrees * graph.outerJoinVertices(degrees) {(vid, data, deg) => deg.getOrElse(0)} * }, - * vpred = (vid: Vid, deg:Int) => deg > 0 + * vpred = (vid: VertexID, deg:Int) => deg > 0 * ) * }}} * @@ -267,7 +271,7 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) { def filter[VD2: ClassTag, ED2: ClassTag]( preprocess: Graph[VD, ED] => Graph[VD2, ED2], epred: (EdgeTriplet[VD2, ED2]) => Boolean = (x: EdgeTriplet[VD2, ED2]) => true, - vpred: (Vid, VD2) => Boolean = (v:Vid, d:VD2) => true): Graph[VD, ED] = { + vpred: (VertexID, VD2) => Boolean = (v:VertexID, d:VD2) => true): Graph[VD, ED] = { graph.mask(preprocess(graph).subgraph(epred, vpred)) } } // end of GraphOps diff --git a/graph/src/main/scala/org/apache/spark/graph/PartitionStrategy.scala b/graph/src/main/scala/org/apache/spark/graph/PartitionStrategy.scala index 293a9d588afda..c01b4b9439589 100644 --- a/graph/src/main/scala/org/apache/spark/graph/PartitionStrategy.scala +++ b/graph/src/main/scala/org/apache/spark/graph/PartitionStrategy.scala @@ -2,7 +2,7 @@ package org.apache.spark.graph sealed trait PartitionStrategy extends Serializable { - def getPartition(src: Vid, dst: Vid, numParts: Pid): Pid + def getPartition(src: VertexID, dst: VertexID, numParts: Pid): Pid } @@ -51,9 +51,9 @@ sealed trait PartitionStrategy extends Serializable { * */ case object EdgePartition2D extends PartitionStrategy { - override def getPartition(src: Vid, dst: Vid, numParts: Pid): Pid = { + override def getPartition(src: VertexID, dst: VertexID, numParts: Pid): Pid = { val ceilSqrtNumParts: Pid = math.ceil(math.sqrt(numParts)).toInt - val mixingPrime: Vid = 1125899906842597L + val mixingPrime: VertexID = 1125899906842597L val col: Pid = ((math.abs(src) * mixingPrime) % ceilSqrtNumParts).toInt val row: Pid = ((math.abs(dst) * mixingPrime) % ceilSqrtNumParts).toInt (col * ceilSqrtNumParts + row) % numParts @@ -62,8 +62,8 @@ case object EdgePartition2D extends PartitionStrategy { case object EdgePartition1D extends PartitionStrategy { - override def getPartition(src: Vid, dst: Vid, numParts: Pid): Pid = { - val mixingPrime: Vid = 1125899906842597L + override def getPartition(src: VertexID, dst: VertexID, numParts: Pid): Pid = { + val mixingPrime: VertexID = 1125899906842597L (math.abs(src) * mixingPrime).toInt % numParts } } @@ -74,7 +74,7 @@ case object EdgePartition1D extends PartitionStrategy { * random vertex cut. */ case object RandomVertexCut extends PartitionStrategy { - override def getPartition(src: Vid, dst: Vid, numParts: Pid): Pid = { + override def getPartition(src: VertexID, dst: VertexID, numParts: Pid): Pid = { math.abs((src, dst).hashCode()) % numParts } } @@ -86,7 +86,7 @@ case object RandomVertexCut extends PartitionStrategy { * will end up on the same partition. */ case object CanonicalRandomVertexCut extends PartitionStrategy { - override def getPartition(src: Vid, dst: Vid, numParts: Pid): Pid = { + override def getPartition(src: VertexID, dst: VertexID, numParts: Pid): Pid = { val lower = math.min(src, dst) val higher = math.max(src, dst) math.abs((lower, higher).hashCode()) % numParts diff --git a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala index 4664091b5714b..3b84e2e5e4195 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Pregel.scala @@ -25,9 +25,9 @@ import scala.reflect.ClassTag * // Set the vertex attributes to the initial pagerank values * .mapVertices( (id, attr) => 1.0 ) * - * def vertexProgram(id: Vid, attr: Double, msgSum: Double): Double = + * def vertexProgram(id: VertexID, attr: Double, msgSum: Double): Double = * resetProb + (1.0 - resetProb) * msgSum - * def sendMessage(id: Vid, edge: EdgeTriplet[Double, Double]): Option[Double] = + * def sendMessage(id: VertexID, edge: EdgeTriplet[Double, Double]): Option[Double] = * Some(edge.srcAttr * edge.attr) * def messageCombiner(a: Double, b: Double): Double = a + b * val initialMessage = 0.0 @@ -88,8 +88,8 @@ object Pregel { */ def apply[VD: ClassTag, ED: ClassTag, A: ClassTag] (graph: Graph[VD, ED], initialMsg: A, maxIterations: Int = Int.MaxValue)( - vprog: (Vid, VD, A) => VD, - sendMsg: EdgeTriplet[VD, ED] => Iterator[(Vid,A)], + vprog: (VertexID, VD, A) => VD, + sendMsg: EdgeTriplet[VD, ED] => Iterator[(VertexID,A)], mergeMsg: (A, A) => A) : Graph[VD, ED] = { diff --git a/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala b/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala index c5fb4aeca73a0..25b0aed85aae9 100644 --- a/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala @@ -29,7 +29,7 @@ import org.apache.spark.graph.impl.VertexPartition /** - * A `VertexRDD[VD]` extends the `RDD[(Vid, VD)]` by ensuring that there is + * A `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. * @@ -40,12 +40,12 @@ import org.apache.spark.graph.impl.VertexPartition * @example Construct a `VertexRDD` from a plain RDD * {{{ * // Construct an intial vertex set - * val someData: RDD[(Vid, SomeType)] = loadData(someFile) + * val someData: RDD[(VertexID, SomeType)] = loadData(someFile) * val vset = VertexRDD(someData) * // If there were redundant values in someData we would use a reduceFunc * val vset2 = VertexRDD(someData, reduceFunc) * // Finally we can use the VertexRDD to index another dataset - * val otherData: RDD[(Vid, OtherType)] = loadData(otherFile) + * val otherData: RDD[(VertexID, OtherType)] = loadData(otherFile) * val vset3 = VertexRDD(otherData, vset.index) * // Now we can construct very fast joins between the two sets * val vset4: VertexRDD[(SomeType, OtherType)] = vset.leftJoin(vset3) @@ -54,7 +54,7 @@ import org.apache.spark.graph.impl.VertexPartition */ class VertexRDD[@specialized VD: ClassTag]( val partitionsRDD: RDD[VertexPartition[VD]]) - extends RDD[(Vid, VD)](partitionsRDD.context, List(new OneToOneDependency(partitionsRDD))) { + extends RDD[(VertexID, VD)](partitionsRDD.context, List(new OneToOneDependency(partitionsRDD))) { require(partitionsRDD.partitioner.isDefined) @@ -104,9 +104,9 @@ class VertexRDD[@specialized VD: ClassTag]( } /** - * Provide the `RDD[(Vid, VD)]` equivalent output. + * Provide the `RDD[(VertexID, VD)]` equivalent output. */ - override def compute(part: Partition, context: TaskContext): Iterator[(Vid, VD)] = { + override def compute(part: Partition, context: TaskContext): Iterator[(VertexID, VD)] = { firstParent[VertexPartition[VD]].iterator(part, context).next.iterator } @@ -125,14 +125,14 @@ class VertexRDD[@specialized VD: ClassTag]( * given predicate. * * @param pred the user defined predicate, which takes a tuple to conform to - * the RDD[(Vid, VD)] interface + * the RDD[(VertexID, VD)] interface * * @note The vertex set preserves the original index structure * which means that the returned RDD can be easily joined with * the original vertex-set. Furthermore, the filter only * modifies the bitmap index and so no new values are allocated. */ - override def filter(pred: Tuple2[Vid, VD] => Boolean): VertexRDD[VD] = + override def filter(pred: Tuple2[VertexID, VD] => Boolean): VertexRDD[VD] = this.mapVertexPartitions(_.filter(Function.untupled(pred))) /** @@ -160,7 +160,7 @@ class VertexRDD[@specialized VD: ClassTag]( * each of the entries in the original VertexRDD. The resulting * VertexRDD retains the same index. */ - def mapValues[VD2: ClassTag](f: (Vid, VD) => VD2): VertexRDD[VD2] = + def mapValues[VD2: ClassTag](f: (VertexID, VD) => VD2): VertexRDD[VD2] = this.mapVertexPartitions(_.map(f)) /** @@ -197,7 +197,7 @@ class VertexRDD[@specialized VD: ClassTag]( * */ def leftZipJoin[VD2: ClassTag, VD3: ClassTag] - (other: VertexRDD[VD2])(f: (Vid, VD, Option[VD2]) => VD3): VertexRDD[VD3] = { + (other: VertexRDD[VD2])(f: (VertexID, VD, Option[VD2]) => VD3): VertexRDD[VD3] = { val newPartitionsRDD = partitionsRDD.zipPartitions( other.partitionsRDD, preservesPartitioning = true ) { (thisIter, otherIter) => @@ -228,8 +228,8 @@ class VertexRDD[@specialized VD: ClassTag]( * VertexRDD with the attribute emitted by f. */ def leftJoin[VD2: ClassTag, VD3: ClassTag] - (other: RDD[(Vid, VD2)]) - (f: (Vid, VD, Option[VD2]) => VD3) + (other: RDD[(VertexID, VD2)]) + (f: (VertexID, VD, Option[VD2]) => VD3) : VertexRDD[VD3] = { // Test if the other vertex is a VertexRDD to choose the optimal join strategy. @@ -254,7 +254,7 @@ class VertexRDD[@specialized VD: ClassTag]( * must have the same index. */ def innerZipJoin[U: ClassTag, VD2: ClassTag](other: VertexRDD[U]) - (f: (Vid, VD, U) => VD2): VertexRDD[VD2] = { + (f: (VertexID, VD, U) => VD2): VertexRDD[VD2] = { val newPartitionsRDD = partitionsRDD.zipPartitions( other.partitionsRDD, preservesPartitioning = true ) { (thisIter, otherIter) => @@ -269,8 +269,8 @@ class VertexRDD[@specialized VD: ClassTag]( * Replace vertices with corresponding vertices in `other`, and drop vertices without a * corresponding vertex in `other`. */ - def innerJoin[U: ClassTag, VD2: ClassTag](other: RDD[(Vid, U)]) - (f: (Vid, VD, U) => VD2): VertexRDD[VD2] = { + def innerJoin[U: ClassTag, VD2: ClassTag](other: RDD[(VertexID, U)]) + (f: (VertexID, VD, U) => VD2): VertexRDD[VD2] = { // Test if the other vertex is a VertexRDD to choose the optimal join strategy. // If the other set is a VertexRDD then we use the much more efficient innerZipJoin other match { @@ -293,7 +293,7 @@ class VertexRDD[@specialized VD: ClassTag]( * co-indexed with this one. */ def aggregateUsingIndex[VD2: ClassTag]( - messages: RDD[(Vid, VD2)], reduceFunc: (VD2, VD2) => VD2): VertexRDD[VD2] = + messages: RDD[(VertexID, VD2)], reduceFunc: (VD2, VD2) => VD2): VertexRDD[VD2] = { val shuffled = MsgRDDFunctions.partitionForAggregation(messages, this.partitioner.get) val parts = partitionsRDD.zipPartitions(shuffled, true) { (thisIter, msgIter) => @@ -319,8 +319,8 @@ object VertexRDD { * * @param rdd the collection of vertex-attribute pairs */ - def apply[VD: ClassTag](rdd: RDD[(Vid, VD)]): VertexRDD[VD] = { - val partitioned: RDD[(Vid, VD)] = rdd.partitioner match { + def apply[VD: ClassTag](rdd: RDD[(VertexID, VD)]): VertexRDD[VD] = { + val partitioned: RDD[(VertexID, VD)] = rdd.partitioner match { case Some(p) => rdd case None => rdd.partitionBy(new HashPartitioner(rdd.partitions.size)) } @@ -339,9 +339,9 @@ object VertexRDD { * @param rdd the collection of vertex-attribute pairs * @param mergeFunc the associative, commutative merge function. */ - def apply[VD: ClassTag](rdd: RDD[(Vid, VD)], mergeFunc: (VD, VD) => VD): VertexRDD[VD] = + def apply[VD: ClassTag](rdd: RDD[(VertexID, VD)], mergeFunc: (VD, VD) => VD): VertexRDD[VD] = { - val partitioned: RDD[(Vid, VD)] = rdd.partitioner match { + val partitioned: RDD[(VertexID, VD)] = rdd.partitioner match { case Some(p) => rdd case None => rdd.partitionBy(new HashPartitioner(rdd.partitions.size)) } @@ -351,7 +351,7 @@ object VertexRDD { new VertexRDD(vertexPartitions) } - def apply[VD: ClassTag](vids: RDD[Vid], rdd: RDD[(Vid, VD)], defaultVal: VD) + def apply[VD: ClassTag](vids: RDD[VertexID], rdd: RDD[(VertexID, VD)], defaultVal: VD) : VertexRDD[VD] = { VertexRDD(vids.map(vid => (vid, defaultVal))).leftJoin(rdd) { (vid, default, value) => diff --git a/graph/src/main/scala/org/apache/spark/graph/algorithms/ConnectedComponents.scala b/graph/src/main/scala/org/apache/spark/graph/algorithms/ConnectedComponents.scala index 7cd947d2ba7df..2a6b8c0999ab0 100644 --- a/graph/src/main/scala/org/apache/spark/graph/algorithms/ConnectedComponents.scala +++ b/graph/src/main/scala/org/apache/spark/graph/algorithms/ConnectedComponents.scala @@ -16,10 +16,10 @@ object ConnectedComponents { * @return a graph with vertex attributes containing the smallest vertex in each * connected component */ - def run[VD: Manifest, ED: Manifest](graph: Graph[VD, ED]): Graph[Vid, ED] = { + def run[VD: Manifest, ED: Manifest](graph: Graph[VD, ED]): Graph[VertexID, ED] = { val ccGraph = graph.mapVertices { case (vid, _) => vid } - def sendMessage(edge: EdgeTriplet[Vid, ED]) = { + def sendMessage(edge: EdgeTriplet[VertexID, ED]) = { if (edge.srcAttr < edge.dstAttr) { Iterator((edge.dstId, edge.srcAttr)) } else if (edge.srcAttr > edge.dstAttr) { diff --git a/graph/src/main/scala/org/apache/spark/graph/algorithms/PageRank.scala b/graph/src/main/scala/org/apache/spark/graph/algorithms/PageRank.scala index f77dffd7b4156..26b8dc5ab60c6 100644 --- a/graph/src/main/scala/org/apache/spark/graph/algorithms/PageRank.scala +++ b/graph/src/main/scala/org/apache/spark/graph/algorithms/PageRank.scala @@ -65,7 +65,7 @@ object PageRank extends Logging { // Define the three functions needed to implement PageRank in the GraphX // version of Pregel - def vertexProgram(id: Vid, attr: Double, msgSum: Double): Double = + def vertexProgram(id: VertexID, attr: Double, msgSum: Double): Double = resetProb + (1.0 - resetProb) * msgSum def sendMessage(edge: EdgeTriplet[Double, Double]) = Iterator((edge.dstId, edge.srcAttr * edge.attr)) @@ -129,7 +129,7 @@ object PageRank extends Logging { // Define the three functions needed to implement PageRank in the GraphX // version of Pregel - def vertexProgram(id: Vid, attr: (Double, Double), msgSum: Double): (Double, Double) = { + def vertexProgram(id: VertexID, attr: (Double, Double), msgSum: Double): (Double, Double) = { val (oldPR, lastDelta) = attr val newPR = oldPR + (1.0 - resetProb) * msgSum (newPR, newPR - oldPR) diff --git a/graph/src/main/scala/org/apache/spark/graph/algorithms/StronglyConnectedComponents.scala b/graph/src/main/scala/org/apache/spark/graph/algorithms/StronglyConnectedComponents.scala index c324c984d7692..8031aa10ce3ea 100644 --- a/graph/src/main/scala/org/apache/spark/graph/algorithms/StronglyConnectedComponents.scala +++ b/graph/src/main/scala/org/apache/spark/graph/algorithms/StronglyConnectedComponents.scala @@ -15,7 +15,7 @@ object StronglyConnectedComponents { * * @return a graph with vertex attributes containing the smallest vertex id in each SCC */ - def run[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], numIter: Int): Graph[Vid, ED] = { + def run[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], numIter: Int): Graph[VertexID, ED] = { // the graph we update with final SCC ids, and the graph we return at the end var sccGraph = graph.mapVertices { case (vid, _) => vid } @@ -52,7 +52,7 @@ 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 = GraphLab[(Vid, Boolean), ED, Vid](sccWorkGraph, Integer.MAX_VALUE)( + sccWorkGraph = GraphLab[(VertexID, Boolean), ED, VertexID](sccWorkGraph, Integer.MAX_VALUE)( (vid, e) => e.otherVertexAttr(vid)._1, (vid1, vid2) => math.min(vid1, vid2), (vid, scc, optScc) => @@ -62,7 +62,7 @@ object StronglyConnectedComponents { // start at root of SCCs. Traverse values in reverse, notify all my neighbors // do not propagate if colors do not match! - sccWorkGraph = GraphLab[(Vid, Boolean), ED, Boolean]( + sccWorkGraph = GraphLab[(VertexID, Boolean), ED, Boolean]( sccWorkGraph, Integer.MAX_VALUE, EdgeDirection.Out, diff --git a/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala b/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala index 18395bdc5f230..85fa23d309469 100644 --- a/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala +++ b/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala @@ -51,12 +51,12 @@ object Svdpp { // 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: Vid, vd: (RealVector, RealVector, Double, Double), msg: Option[(Long, Double)]) => + 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: SvdppConf, u: Double)(et: EdgeTriplet[(RealVector, RealVector, Double, Double), Double]) - : Iterator[(Vid, (RealVector, RealVector, Double))] = { + : Iterator[(VertexID, (RealVector, RealVector, 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) @@ -73,19 +73,19 @@ object Svdpp { for (i <- 0 until conf.maxIters) { // phase 1, calculate pu + |N(u)|^(-0.5)*sum(y) for user nodes var t1 = g.mapReduceTriplets(et => Iterator((et.srcId, et.dstAttr._2)), (g1: RealVector, g2: RealVector) => g1.add(g2)) - g = g.outerJoinVertices(t1) { (vid: Vid, vd: (RealVector, RealVector, Double, Double), msg: Option[RealVector]) => + 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 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: Vid, vd: (RealVector, RealVector, Double, Double), msg: Option[(RealVector, RealVector, Double)]) => + 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: SvdppConf, u: Double)(et: EdgeTriplet[(RealVector, RealVector, Double, Double), Double]): Iterator[(Vid, Double)] = { + def mapTestF(conf: SvdppConf, 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) @@ -95,7 +95,7 @@ object Svdpp { Iterator((et.dstId, err)) } val t3 = g.mapReduceTriplets(mapTestF(conf, u), (g1: Double, g2: Double) => g1 + g2) - g = g.outerJoinVertices(t3) { (vid: Vid, vd: (RealVector, RealVector, Double, Double), msg: Option[Double]) => + 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) diff --git a/graph/src/main/scala/org/apache/spark/graph/algorithms/TriangleCount.scala b/graph/src/main/scala/org/apache/spark/graph/algorithms/TriangleCount.scala index a6384320bab6b..81774d52e4291 100644 --- a/graph/src/main/scala/org/apache/spark/graph/algorithms/TriangleCount.scala +++ b/graph/src/main/scala/org/apache/spark/graph/algorithms/TriangleCount.scala @@ -46,7 +46,7 @@ object TriangleCount { (vid, _, optSet) => optSet.getOrElse(null) } // Edge function computes intersection of smaller vertex with larger vertex - def edgeFunc(et: EdgeTriplet[VertexSet, ED]): Iterator[(Vid, Int)] = { + def edgeFunc(et: EdgeTriplet[VertexSet, ED]): Iterator[(VertexID, Int)] = { assert(et.srcAttr != null) assert(et.dstAttr != null) val (smallSet, largeSet) = if (et.srcAttr.size < et.dstAttr.size) { diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala b/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala index 7ae4d7df43a2f..b4311fa9f894e 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala @@ -16,10 +16,10 @@ import org.apache.spark.util.collection.PrimitiveKeyOpenHashMap * @tparam ED the edge attribute type. */ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED: ClassTag]( - val srcIds: Array[Vid], - val dstIds: Array[Vid], + val srcIds: Array[VertexID], + val dstIds: Array[VertexID], val data: Array[ED], - val index: PrimitiveKeyOpenHashMap[Vid, Int]) extends Serializable { + val index: PrimitiveKeyOpenHashMap[VertexID, Int]) extends Serializable { /** * Reverse all the edges in this partition. @@ -101,8 +101,8 @@ 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: Vid = nullValue[Vid] - var currDstId: Vid = nullValue[Vid] + var currSrcId: VertexID = nullValue[VertexID] + var currDstId: VertexID = nullValue[VertexID] var currAttr: ED = nullValue[ED] var i = 0 while (i < size) { @@ -136,7 +136,7 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) */ def innerJoin[ED2: ClassTag, ED3: ClassTag] (other: EdgePartition[ED2]) - (f: (Vid, Vid, ED, ED2) => ED3): EdgePartition[ED3] = { + (f: (VertexID, VertexID, ED, ED2) => ED3): EdgePartition[ED3] = { val builder = new EdgePartitionBuilder[ED3] var i = 0 var j = 0 @@ -193,14 +193,14 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) * iterator is generated using an index scan, so it is efficient at skipping edges that don't * match srcIdPred. */ - def indexIterator(srcIdPred: Vid => Boolean): Iterator[Edge[ED]] = + def indexIterator(srcIdPred: VertexID => Boolean): Iterator[Edge[ED]] = index.iterator.filter(kv => srcIdPred(kv._1)).flatMap(Function.tupled(clusterIterator)) /** * Get an iterator over the cluster of edges in this partition with source vertex id `srcId`. The * cluster must start at position `index`. */ - private def clusterIterator(srcId: Vid, index: Int) = new Iterator[Edge[ED]] { + private def clusterIterator(srcId: VertexID, index: Int) = new Iterator[Edge[ED]] { private[this] val edge = new Edge[ED] private[this] var pos = index diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartitionBuilder.scala b/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartitionBuilder.scala index ae3f3a6d03145..56624ef60adc0 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartitionBuilder.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartitionBuilder.scala @@ -13,22 +13,22 @@ class EdgePartitionBuilder[@specialized(Long, Int, Double) ED: ClassTag](size: I var edges = new PrimitiveVector[Edge[ED]](size) /** Add a new edge to the partition. */ - def add(src: Vid, dst: Vid, d: ED) { + def add(src: VertexID, dst: VertexID, d: ED) { edges += Edge(src, dst, d) } def toEdgePartition: EdgePartition[ED] = { val edgeArray = edges.trim().array Sorting.quickSort(edgeArray)(Edge.lexicographicOrdering) - val srcIds = new Array[Vid](edgeArray.size) - val dstIds = new Array[Vid](edgeArray.size) + val srcIds = new Array[VertexID](edgeArray.size) + val dstIds = new Array[VertexID](edgeArray.size) val data = new Array[ED](edgeArray.size) - val index = new PrimitiveKeyOpenHashMap[Vid, Int] + val index = new PrimitiveKeyOpenHashMap[VertexID, Int] // Copy edges into columnar structures, tracking the beginnings of source vertex id clusters and // adding them to the index if (edgeArray.length > 0) { index.update(srcIds(0), 0) - var currSrcId: Vid = srcIds(0) + var currSrcId: VertexID = srcIds(0) var i = 0 while (i < edgeArray.size) { srcIds(i) = edgeArray(i).srcId diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/EdgeTripletIterator.scala b/graph/src/main/scala/org/apache/spark/graph/impl/EdgeTripletIterator.scala index 4d5eb240a91b7..e95d79e3d63d6 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/EdgeTripletIterator.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/EdgeTripletIterator.scala @@ -25,7 +25,7 @@ class EdgeTripletIterator[VD: ClassTag, ED: ClassTag]( // allocating too many temporary Java objects. private val triplet = new EdgeTriplet[VD, ED] - private val vmap = new PrimitiveKeyOpenHashMap[Vid, VD](vidToIndex, vertexArray) + private val vmap = new PrimitiveKeyOpenHashMap[VertexID, VD](vidToIndex, vertexArray) override def hasNext: Boolean = pos < edgePartition.size diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index 2ce5404e94744..6eb401b3b57ac 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -89,7 +89,7 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( 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[Vid]]]): Double = { + def numReplicatedVertices(vid2pids: RDD[Array[Array[VertexID]]]): Double = { vid2pids.map(_.map(_.size).sum.toLong).reduce(_ + _).toDouble } @@ -157,7 +157,7 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( new GraphImpl(vertices, newETable, routingTable, replicatedVertexView) } - override def mapVertices[VD2: ClassTag](f: (Vid, VD) => VD2): Graph[VD2, ED] = { + override def mapVertices[VD2: ClassTag](f: (VertexID, VD) => VD2): Graph[VD2, ED] = { if (classTag[VD] equals classTag[VD2]) { // The map preserves type, so we can use incremental replication val newVerts = vertices.mapVertexPartitions(_.map(f)) @@ -208,7 +208,7 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( override def subgraph( epred: EdgeTriplet[VD, ED] => Boolean = x => true, - vpred: (Vid, VD) => Boolean = (a, b) => true): Graph[VD, ED] = { + vpred: (VertexID, VD) => Boolean = (a, b) => true): Graph[VD, ED] = { // Filter the vertices, reusing the partitioner and the index from this graph val newVerts = vertices.mapVertexPartitions(_.filter(vpred)) @@ -250,7 +250,7 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( ////////////////////////////////////////////////////////////////////////////////////////////////// override def mapReduceTriplets[A: ClassTag]( - mapFunc: EdgeTriplet[VD, ED] => Iterator[(Vid, A)], + mapFunc: EdgeTriplet[VD, ED] => Iterator[(VertexID, A)], reduceFunc: (A, A) => A, activeSetOpt: Option[(VertexRDD[_], EdgeDirection)] = None) = { @@ -280,14 +280,14 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( val edgeIter = activeDirectionOpt match { case Some(EdgeDirection.Both) => if (activeFraction < 0.8) { - edgePartition.indexIterator(srcVid => vPart.isActive(srcVid)) + edgePartition.indexIterator(srcVertexID => vPart.isActive(srcVertexID)) .filter(e => vPart.isActive(e.dstId)) } else { edgePartition.iterator.filter(e => vPart.isActive(e.srcId) && vPart.isActive(e.dstId)) } case Some(EdgeDirection.Out) => if (activeFraction < 0.8) { - edgePartition.indexIterator(srcVid => vPart.isActive(srcVid)) + edgePartition.indexIterator(srcVertexID => vPart.isActive(srcVertexID)) } else { edgePartition.iterator.filter(e => vPart.isActive(e.srcId)) } @@ -318,7 +318,7 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( } // end of mapReduceTriplets override def outerJoinVertices[U: ClassTag, VD2: ClassTag] - (updates: RDD[(Vid, U)])(updateF: (Vid, VD, Option[U]) => VD2): Graph[VD2, ED] = { + (updates: 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) @@ -360,7 +360,7 @@ object GraphImpl { } def apply[VD: ClassTag, ED: ClassTag]( - vertices: RDD[(Vid, VD)], + vertices: RDD[(VertexID, VD)], edges: RDD[Edge[ED]], defaultVertexAttr: VD): GraphImpl[VD, ED] = { @@ -369,7 +369,7 @@ object GraphImpl { // Get the set of all vids val partitioner = Partitioner.defaultPartitioner(vertices) val vPartitioned = vertices.partitionBy(partitioner) - val vidsFromEdges = collectVidsFromEdges(edgeRDD, partitioner) + val vidsFromEdges = collectVertexIDsFromEdges(edgeRDD, partitioner) val vids = vPartitioned.zipPartitions(vidsFromEdges) { (vertexIter, vidsFromEdgesIter) => vertexIter.map(_._1) ++ vidsFromEdgesIter.map(_._1) } @@ -381,7 +381,7 @@ object GraphImpl { /** * Create the edge RDD, which is much more efficient for Java heap storage than the normal edges - * data structure (RDD[(Vid, Vid, ED)]). + * data structure (RDD[(VertexID, VertexID, ED)]). * * The edge RDD contains multiple partitions, and each partition contains only one RDD key-value * pair: the key is the partition id, and the value is an EdgePartition object containing all the @@ -404,19 +404,19 @@ object GraphImpl { defaultVertexAttr: VD): GraphImpl[VD, ED] = { edges.cache() // Get the set of all vids - val vids = collectVidsFromEdges(edges, new HashPartitioner(edges.partitions.size)) + val vids = collectVertexIDsFromEdges(edges, new HashPartitioner(edges.partitions.size)) // Create the VertexRDD. val vertices = VertexRDD(vids.mapValues(x => defaultVertexAttr)) new GraphImpl(vertices, edges) } /** Collects all vids mentioned in edges and partitions them by partitioner. */ - private def collectVidsFromEdges( + private def collectVertexIDsFromEdges( edges: EdgeRDD[_], - partitioner: Partitioner): RDD[(Vid, Int)] = { + partitioner: Partitioner): RDD[(VertexID, Int)] = { // TODO: Consider doing map side distinct before shuffle. - new ShuffledRDD[Vid, Int, (Vid, Int)]( - edges.collectVids.map(vid => (vid, 0)), partitioner) - .setSerializer(classOf[VidMsgSerializer].getName) + new ShuffledRDD[VertexID, Int, (VertexID, Int)]( + edges.collectVertexIDs.map(vid => (vid, 0)), partitioner) + .setSerializer(classOf[VertexIDMsgSerializer].getName) } } // end of object GraphImpl diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/MessageToPartition.scala b/graph/src/main/scala/org/apache/spark/graph/impl/MessageToPartition.scala index bf033945dee24..2d03f75a28a25 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/MessageToPartition.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/MessageToPartition.scala @@ -3,15 +3,15 @@ package org.apache.spark.graph.impl import scala.reflect.{classTag, ClassTag} import org.apache.spark.Partitioner -import org.apache.spark.graph.{Pid, Vid} +import org.apache.spark.graph.{Pid, VertexID} import org.apache.spark.rdd.{ShuffledRDD, RDD} class VertexBroadcastMsg[@specialized(Int, Long, Double, Boolean) T]( @transient var partition: Pid, - var vid: Vid, + var vid: VertexID, var data: T) - extends Product2[Pid, (Vid, T)] with Serializable { + extends Product2[Pid, (VertexID, T)] with Serializable { override def _1 = partition @@ -41,7 +41,7 @@ class MessageToPartition[@specialized(Int, Long, Double, Char, Boolean/*, AnyRef class VertexBroadcastMsgRDDFunctions[T: ClassTag](self: RDD[VertexBroadcastMsg[T]]) { def partitionBy(partitioner: Partitioner): RDD[VertexBroadcastMsg[T]] = { - val rdd = new ShuffledRDD[Pid, (Vid, T), VertexBroadcastMsg[T]](self, partitioner) + val rdd = new ShuffledRDD[Pid, (VertexID, T), VertexBroadcastMsg[T]](self, partitioner) // Set a custom serializer if the data is of int or double type. if (classTag[T] == ClassTag.Int) { @@ -77,8 +77,8 @@ object MsgRDDFunctions { new VertexBroadcastMsgRDDFunctions(rdd) } - def partitionForAggregation[T: ClassTag](msgs: RDD[(Vid, T)], partitioner: Partitioner) = { - val rdd = new ShuffledRDD[Vid, T, (Vid, T)](msgs, partitioner) + def partitionForAggregation[T: ClassTag](msgs: RDD[(VertexID, T)], partitioner: Partitioner) = { + val rdd = new ShuffledRDD[VertexID, T, (VertexID, T)](msgs, partitioner) // Set a custom serializer if the data is of int or double type. if (classTag[T] == ClassTag.Int) { diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/ReplicatedVertexView.scala b/graph/src/main/scala/org/apache/spark/graph/impl/ReplicatedVertexView.scala index 970acfed27425..9d2d242ffa8d9 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/ReplicatedVertexView.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/ReplicatedVertexView.scala @@ -31,9 +31,9 @@ class ReplicatedVertexView[VD: ClassTag]( * vids from both the source and destination of edges. It must always include both source and * destination vids because some operations, such as GraphImpl.mapReduceTriplets, rely on this. */ - private val localVidMap: RDD[(Int, VertexIdToIndexMap)] = prevViewOpt match { + private val localVertexIDMap: RDD[(Int, VertexIdToIndexMap)] = prevViewOpt match { case Some(prevView) => - prevView.localVidMap + prevView.localVertexIDMap case None => edges.partitionsRDD.mapPartitions(_.map { case (pid, epart) => @@ -43,7 +43,7 @@ class ReplicatedVertexView[VD: ClassTag]( vidToIndex.add(e.dstId) } (pid, vidToIndex) - }, preservesPartitioning = true).cache().setName("ReplicatedVertexView localVidMap") + }, preservesPartitioning = true).cache().setName("ReplicatedVertexView localVertexIDMap") } private lazy val bothAttrs: RDD[(Pid, VertexPartition[VD])] = create(true, true) @@ -104,8 +104,8 @@ class ReplicatedVertexView[VD: ClassTag]( case None => // Within each edge partition, place the shipped vertex attributes into the correct - // locations specified in localVidMap - localVidMap.zipPartitions(shippedVerts) { (mapIter, shippedVertsIter) => + // locations specified in localVertexIDMap + localVertexIDMap.zipPartitions(shippedVerts) { (mapIter, shippedVertsIter) => val (pid, vidToIndex) = mapIter.next() assert(!mapIter.hasNext) // Populate the vertex array using the vidToIndex map @@ -128,15 +128,15 @@ class ReplicatedVertexView[VD: ClassTag]( object ReplicatedVertexView { protected def buildBuffer[VD: ClassTag]( - pid2vidIter: Iterator[Array[Array[Vid]]], + pid2vidIter: Iterator[Array[Array[VertexID]]], vertexPartIter: Iterator[VertexPartition[VD]]) = { - val pid2vid: Array[Array[Vid]] = pid2vidIter.next() + val pid2vid: Array[Array[VertexID]] = pid2vidIter.next() val vertexPart: VertexPartition[VD] = vertexPartIter.next() Iterator.tabulate(pid2vid.size) { pid => val vidsCandidate = pid2vid(pid) val size = vidsCandidate.length - val vids = new PrimitiveVector[Vid](pid2vid(pid).size) + val vids = new PrimitiveVector[VertexID](pid2vid(pid).size) val attrs = new PrimitiveVector[VD](pid2vid(pid).size) var i = 0 while (i < size) { @@ -152,16 +152,16 @@ object ReplicatedVertexView { } protected def buildActiveBuffer( - pid2vidIter: Iterator[Array[Array[Vid]]], + pid2vidIter: Iterator[Array[Array[VertexID]]], activePartIter: Iterator[VertexPartition[_]]) - : Iterator[(Int, Array[Vid])] = { - val pid2vid: Array[Array[Vid]] = pid2vidIter.next() + : Iterator[(Int, Array[VertexID])] = { + val pid2vid: Array[Array[VertexID]] = pid2vidIter.next() val activePart: VertexPartition[_] = activePartIter.next() Iterator.tabulate(pid2vid.size) { pid => val vidsCandidate = pid2vid(pid) val size = vidsCandidate.length - val actives = new PrimitiveVector[Vid](vidsCandidate.size) + val actives = new PrimitiveVector[VertexID](vidsCandidate.size) var i = 0 while (i < size) { val vid = vidsCandidate(i) @@ -175,7 +175,8 @@ object ReplicatedVertexView { } } -class VertexAttributeBlock[VD: ClassTag](val vids: Array[Vid], val attrs: Array[VD]) +class VertexAttributeBlock[VD: ClassTag](val vids: Array[VertexID], val attrs: Array[VD]) extends Serializable { - def iterator: Iterator[(Vid, VD)] = (0 until vids.size).iterator.map { i => (vids(i), attrs(i)) } + def iterator: Iterator[(VertexID, VD)] = + (0 until vids.size).iterator.map { i => (vids(i), attrs(i)) } } diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/RoutingTable.scala b/graph/src/main/scala/org/apache/spark/graph/impl/RoutingTable.scala index b6cd048b33bb1..9e6f78197e7f9 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/RoutingTable.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/RoutingTable.scala @@ -14,12 +14,12 @@ import org.apache.spark.util.collection.PrimitiveVector */ class RoutingTable(edges: EdgeRDD[_], vertices: VertexRDD[_]) { - val bothAttrs: RDD[Array[Array[Vid]]] = createPid2Vid(true, true) - val srcAttrOnly: RDD[Array[Array[Vid]]] = createPid2Vid(true, false) - val dstAttrOnly: RDD[Array[Array[Vid]]] = createPid2Vid(false, true) - val noAttrs: RDD[Array[Array[Vid]]] = createPid2Vid(false, false) + val bothAttrs: RDD[Array[Array[VertexID]]] = createPid2Vid(true, true) + val srcAttrOnly: RDD[Array[Array[VertexID]]] = createPid2Vid(true, false) + val dstAttrOnly: RDD[Array[Array[VertexID]]] = createPid2Vid(false, true) + val noAttrs: RDD[Array[Array[VertexID]]] = createPid2Vid(false, false) - def get(includeSrcAttr: Boolean, includeDstAttr: Boolean): RDD[Array[Array[Vid]]] = + def get(includeSrcAttr: Boolean, includeDstAttr: Boolean): RDD[Array[Array[VertexID]]] = (includeSrcAttr, includeDstAttr) match { case (true, true) => bothAttrs case (true, false) => srcAttrOnly @@ -28,9 +28,9 @@ class RoutingTable(edges: EdgeRDD[_], vertices: VertexRDD[_]) { } private def createPid2Vid( - includeSrcAttr: Boolean, includeDstAttr: Boolean): RDD[Array[Array[Vid]]] = { + includeSrcAttr: Boolean, includeDstAttr: Boolean): RDD[Array[Array[VertexID]]] = { // Determine which vertices each edge partition needs by creating a mapping from vid to pid. - val vid2pid: RDD[(Vid, Pid)] = edges.partitionsRDD.mapPartitions { iter => + val vid2pid: RDD[(VertexID, Pid)] = edges.partitionsRDD.mapPartitions { iter => val (pid: Pid, edgePartition: EdgePartition[_]) = iter.next() val numEdges = edgePartition.size val vSet = new VertexSet @@ -53,7 +53,7 @@ class RoutingTable(edges: EdgeRDD[_], vertices: VertexRDD[_]) { val numPartitions = vertices.partitions.size vid2pid.partitionBy(vertices.partitioner.get).mapPartitions { iter => - val pid2vid = Array.fill(numPartitions)(new PrimitiveVector[Vid]) + val pid2vid = Array.fill(numPartitions)(new PrimitiveVector[VertexID]) for ((vid, pid) <- iter) { pid2vid(pid) += vid } diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala b/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala index dcf619fa85e8a..a3b0ea7689fc7 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala @@ -7,12 +7,12 @@ import org.apache.spark.SparkConf import org.apache.spark.graph._ import org.apache.spark.serializer._ -class VidMsgSerializer(conf: SparkConf) extends Serializer { +class VertexIDMsgSerializer(conf: SparkConf) extends Serializer { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { def writeObject[T](t: T) = { - val msg = t.asInstanceOf[(Vid, _)] + val msg = t.asInstanceOf[(VertexID, _)] writeVarLong(msg._1, optimizePositive = false) this } @@ -101,7 +101,7 @@ class IntAggMsgSerializer(conf: SparkConf) extends Serializer { override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { def writeObject[T](t: T) = { - val msg = t.asInstanceOf[(Vid, Int)] + val msg = t.asInstanceOf[(VertexID, Int)] writeVarLong(msg._1, optimizePositive = false) writeUnsignedVarInt(msg._2) this @@ -124,7 +124,7 @@ class LongAggMsgSerializer(conf: SparkConf) extends Serializer { override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { def writeObject[T](t: T) = { - val msg = t.asInstanceOf[(Vid, Long)] + val msg = t.asInstanceOf[(VertexID, Long)] writeVarLong(msg._1, optimizePositive = false) writeVarLong(msg._2, optimizePositive = true) this @@ -147,7 +147,7 @@ class DoubleAggMsgSerializer(conf: SparkConf) extends Serializer { override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { def writeObject[T](t: T) = { - val msg = t.asInstanceOf[(Vid, Double)] + val msg = t.asInstanceOf[(VertexID, Double)] writeVarLong(msg._1, optimizePositive = false) writeDouble(msg._2) this diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala index 7048a40f42364..91244daa54a57 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala @@ -10,18 +10,18 @@ import org.apache.spark.graph._ private[graph] object VertexPartition { - def apply[VD: ClassTag](iter: Iterator[(Vid, VD)]): VertexPartition[VD] = { - val map = new PrimitiveKeyOpenHashMap[Vid, VD] + def apply[VD: ClassTag](iter: Iterator[(VertexID, VD)]): VertexPartition[VD] = { + val map = new PrimitiveKeyOpenHashMap[VertexID, VD] iter.foreach { case (k, v) => map(k) = v } new VertexPartition(map.keySet, map._values, map.keySet.getBitSet) } - def apply[VD: ClassTag](iter: Iterator[(Vid, VD)], mergeFunc: (VD, VD) => VD) + def apply[VD: ClassTag](iter: Iterator[(VertexID, VD)], mergeFunc: (VD, VD) => VD) : VertexPartition[VD] = { - val map = new PrimitiveKeyOpenHashMap[Vid, VD] + val map = new PrimitiveKeyOpenHashMap[VertexID, VD] iter.foreach { case (k, v) => map.setMerge(k, v, mergeFunc) } @@ -44,15 +44,15 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassTag]( def size: Int = mask.cardinality() /** Return the vertex attribute for the given vertex ID. */ - def apply(vid: Vid): VD = values(index.getPos(vid)) + def apply(vid: VertexID): VD = values(index.getPos(vid)) - def isDefined(vid: Vid): Boolean = { + def isDefined(vid: VertexID): Boolean = { val pos = index.getPos(vid) pos >= 0 && mask.get(pos) } /** Look up vid in activeSet, throwing an exception if it is None. */ - def isActive(vid: Vid): Boolean = { + def isActive(vid: VertexID): Boolean = { activeSet.get.contains(vid) } @@ -72,7 +72,7 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassTag]( * each of the entries in the original VertexRDD. The resulting * VertexPartition retains the same index. */ - def map[VD2: ClassTag](f: (Vid, VD) => VD2): VertexPartition[VD2] = { + def map[VD2: ClassTag](f: (VertexID, VD) => VD2): VertexPartition[VD2] = { // Construct a view of the map transformation val newValues = new Array[VD2](capacity) var i = mask.nextSetBit(0) @@ -92,7 +92,7 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassTag]( * RDD can be easily joined with the original vertex-set. Furthermore, the filter only * modifies the bitmap index and so no new values are allocated. */ - def filter(pred: (Vid, VD) => Boolean): VertexPartition[VD] = { + def filter(pred: (VertexID, VD) => Boolean): VertexPartition[VD] = { // Allocate the array to store the results into val newMask = new BitSet(capacity) // Iterate over the active bits in the old mask and evaluate the predicate @@ -130,7 +130,7 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassTag]( /** Left outer join another VertexPartition. */ def leftJoin[VD2: ClassTag, VD3: ClassTag] (other: VertexPartition[VD2]) - (f: (Vid, VD, Option[VD2]) => VD3): VertexPartition[VD3] = { + (f: (VertexID, VD, Option[VD2]) => VD3): VertexPartition[VD3] = { if (index != other.index) { logWarning("Joining two VertexPartitions with different indexes is slow.") leftJoin(createUsingIndex(other.iterator))(f) @@ -149,14 +149,14 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassTag]( /** Left outer join another iterator of messages. */ def leftJoin[VD2: ClassTag, VD3: ClassTag] - (other: Iterator[(Vid, VD2)]) - (f: (Vid, VD, Option[VD2]) => VD3): VertexPartition[VD3] = { + (other: Iterator[(VertexID, VD2)]) + (f: (VertexID, VD, Option[VD2]) => VD3): VertexPartition[VD3] = { leftJoin(createUsingIndex(other))(f) } /** Inner join another VertexPartition. */ def innerJoin[U: ClassTag, VD2: ClassTag](other: VertexPartition[U]) - (f: (Vid, VD, U) => VD2): VertexPartition[VD2] = { + (f: (VertexID, VD, U) => VD2): VertexPartition[VD2] = { if (index != other.index) { logWarning("Joining two VertexPartitions with different indexes is slow.") innerJoin(createUsingIndex(other.iterator))(f) @@ -176,15 +176,15 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassTag]( * Inner join an iterator of messages. */ def innerJoin[U: ClassTag, VD2: ClassTag] - (iter: Iterator[Product2[Vid, U]]) - (f: (Vid, VD, U) => VD2): VertexPartition[VD2] = { + (iter: Iterator[Product2[VertexID, U]]) + (f: (VertexID, VD, U) => VD2): VertexPartition[VD2] = { innerJoin(createUsingIndex(iter))(f) } /** * Similar effect as aggregateUsingIndex((a, b) => a) */ - def createUsingIndex[VD2: ClassTag](iter: Iterator[Product2[Vid, VD2]]) + def createUsingIndex[VD2: ClassTag](iter: Iterator[Product2[VertexID, VD2]]) : VertexPartition[VD2] = { val newMask = new BitSet(capacity) val newValues = new Array[VD2](capacity) @@ -202,7 +202,7 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassTag]( * Similar to innerJoin, but vertices from the left side that don't appear in iter will remain in * the partition, hidden by the bitmask. */ - def innerJoinKeepLeft(iter: Iterator[Product2[Vid, VD]]): VertexPartition[VD] = { + def innerJoinKeepLeft(iter: Iterator[Product2[VertexID, VD]]): VertexPartition[VD] = { val newMask = new BitSet(capacity) val newValues = new Array[VD](capacity) System.arraycopy(values, 0, newValues, 0, newValues.length) @@ -217,8 +217,8 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassTag]( } def aggregateUsingIndex[VD2: ClassTag]( - iter: Iterator[Product2[Vid, VD2]], reduceFunc: (VD2, VD2) => VD2): VertexPartition[VD2] = - { + iter: Iterator[Product2[VertexID, VD2]], + reduceFunc: (VD2, VD2) => VD2): VertexPartition[VD2] = { val newMask = new BitSet(capacity) val newValues = new Array[VD2](capacity) iter.foreach { product => @@ -237,7 +237,7 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassTag]( new VertexPartition[VD2](index, newValues, newMask) } - def replaceActives(iter: Iterator[Vid]): VertexPartition[VD] = { + def replaceActives(iter: Iterator[VertexID]): VertexPartition[VD] = { val newActiveSet = new VertexSet iter.foreach(newActiveSet.add(_)) new VertexPartition(index, values, mask, Some(newActiveSet)) @@ -247,7 +247,7 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassTag]( * Construct a new VertexPartition whose index contains only the vertices in the mask. */ def reindex(): VertexPartition[VD] = { - val hashMap = new PrimitiveKeyOpenHashMap[Vid, VD] + val hashMap = new PrimitiveKeyOpenHashMap[VertexID, VD] val arbitraryMerge = (a: VD, b: VD) => a for ((k, v) <- this.iterator) { hashMap.setMerge(k, v, arbitraryMerge) @@ -255,7 +255,8 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassTag]( new VertexPartition(hashMap.keySet, hashMap._values, hashMap.keySet.getBitSet) } - def iterator: Iterator[(Vid, VD)] = mask.iterator.map(ind => (index.getValue(ind), values(ind))) + def iterator: Iterator[(VertexID, VD)] = + mask.iterator.map(ind => (index.getValue(ind), values(ind))) - def vidIterator: Iterator[Vid] = mask.iterator.map(ind => index.getValue(ind)) + def vidIterator: Iterator[VertexID] = mask.iterator.map(ind => index.getValue(ind)) } diff --git a/graph/src/main/scala/org/apache/spark/graph/package.scala b/graph/src/main/scala/org/apache/spark/graph/package.scala index 655ae53bf8bc5..823d47c359b09 100644 --- a/graph/src/main/scala/org/apache/spark/graph/package.scala +++ b/graph/src/main/scala/org/apache/spark/graph/package.scala @@ -5,15 +5,15 @@ import org.apache.spark.util.collection.OpenHashSet package object graph { - type Vid = Long + type VertexID = Long // TODO: Consider using Char. type Pid = Int - type VertexSet = OpenHashSet[Vid] + type VertexSet = OpenHashSet[VertexID] // type VertexIdToIndexMap = it.unimi.dsi.fastutil.longs.Long2IntOpenHashMap - type VertexIdToIndexMap = OpenHashSet[Vid] + type VertexIdToIndexMap = OpenHashSet[VertexID] /** * Return the default null-like value for a data type T. diff --git a/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala b/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala index d61f358bb08bd..51f45cb8922b3 100644 --- a/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala +++ b/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala @@ -70,7 +70,7 @@ object GraphGenerators { val sigma = 1.3 //val vertsAndEdges = (0 until numVertices).flatMap { src => { - val vertices: RDD[(Vid, Int)] = sc.parallelize(0 until numVertices).map{ + val vertices: RDD[(VertexID, Int)] = sc.parallelize(0 until numVertices).map{ src => (src, sampleLogNormal(mu, sigma, numVertices)) } @@ -92,11 +92,11 @@ object GraphGenerators { } - def generateRandomEdges(src: Int, numEdges: Int, maxVid: Int): Array[Edge[Int]] = { + 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(maxVid) + val nextDst = rand.nextInt(maxVertexID) if (nextDst != src) { dsts += nextDst } @@ -251,9 +251,9 @@ object GraphGenerators { */ def gridGraph(sc: SparkContext, rows: Int, cols: Int): Graph[(Int,Int), Double] = { // Convert row column address into vertex ids (row major order) - def sub2ind(r: Int, c: Int): Vid = r * cols + c + def sub2ind(r: Int, c: Int): VertexID = r * cols + c - val vertices: RDD[(Vid, (Int,Int))] = + val vertices: RDD[(VertexID, (Int,Int))] = sc.parallelize(0 until rows).flatMap( r => (0 until cols).map( c => (sub2ind(r,c), (r,c)) ) ) val edges: RDD[Edge[Double]] = vertices.flatMap{ case (vid, (r,c)) => @@ -273,7 +273,7 @@ object GraphGenerators { * being the center vertex. */ def starGraph(sc: SparkContext, nverts: Int): Graph[Int, Int] = { - val edges: RDD[(Vid, Vid)] = sc.parallelize(1 until nverts).map(vid => (vid, 0)) + val edges: RDD[(VertexID, VertexID)] = sc.parallelize(1 until nverts).map(vid => (vid, 0)) Graph.fromEdgeTuples(edges, 1) } // end of starGraph diff --git a/graph/src/test/scala/org/apache/spark/graph/GraphOpsSuite.scala b/graph/src/test/scala/org/apache/spark/graph/GraphOpsSuite.scala index 9e9213631ff63..132e6be24a04a 100644 --- a/graph/src/test/scala/org/apache/spark/graph/GraphOpsSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/GraphOpsSuite.scala @@ -11,7 +11,8 @@ 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: Vid, x: Vid))), 1) + val star = + Graph.fromEdgeTuples(sc.parallelize((1 to n).map(x => (0: VertexID, x: VertexID))), 1) val indegrees = star.aggregateNeighbors( (vid, edge) => Some(1), @@ -26,21 +27,22 @@ class GraphOpsSuite extends FunSuite with LocalSparkContext { assert(outdegrees.collect().toSet === Set((0, n))) val noVertexValues = star.aggregateNeighbors[Int]( - (vid: Vid, edge: EdgeTriplet[Int, Int]) => None, + (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[(Vid, Int)]) + assert(noVertexValues.collect().toSet === Set.empty[(VertexID, Int)]) } } test("joinVertices") { withSpark { sc => - val vertices = sc.parallelize(Seq[(Vid, String)]((1, "one"), (2, "two"), (3, "three")), 2) + val vertices = + sc.parallelize(Seq[(VertexID, String)]((1, "one"), (2, "two"), (3, "three")), 2) val edges = sc.parallelize((Seq(Edge(1, 2, "onetwo")))) val g: Graph[String, String] = Graph(vertices, edges) - val tbl = sc.parallelize(Seq[(Vid, Int)]((1, 10), (2, 20))) - val g1 = g.joinVertices(tbl) { (vid: Vid, attr: String, u: Int) => attr + u } + val tbl = sc.parallelize(Seq[(VertexID, Int)]((1, 10), (2, 20))) + val g1 = g.joinVertices(tbl) { (vid: VertexID, attr: String, u: Int) => attr + u } val v = g1.vertices.collect().toSet assert(v === Set((1, "one10"), (2, "two20"), (3, "three"))) @@ -67,7 +69,7 @@ class GraphOpsSuite extends FunSuite with LocalSparkContext { test ("filter") { withSpark { sc => val n = 5 - val vertices = sc.parallelize((0 to n).map(x => (x:Vid, x))) + val vertices = sc.parallelize((0 to n).map(x => (x:VertexID, x))) val edges = sc.parallelize((1 to n).map(x => Edge(0, x, x))) val graph: Graph[Int, Int] = Graph(vertices, edges) val filteredGraph = graph.filter( @@ -75,7 +77,7 @@ class GraphOpsSuite extends FunSuite with LocalSparkContext { val degrees: VertexRDD[Int] = graph.outDegrees graph.outerJoinVertices(degrees) {(vid, data, deg) => deg.getOrElse(0)} }, - vpred = (vid: Vid, deg:Int) => deg > 0 + vpred = (vid: VertexID, deg:Int) => deg > 0 ) val v = filteredGraph.vertices.collect().toSet diff --git a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala index e6c19dbc40358..41f3a8311d5a8 100644 --- a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala @@ -9,7 +9,7 @@ import org.apache.spark.rdd._ class GraphSuite extends FunSuite with LocalSparkContext { def starGraph(sc: SparkContext, n: Int): Graph[String, Int] = { - Graph.fromEdgeTuples(sc.parallelize((1 to n).map(x => (0: Vid, x: Vid)), 3), "v") + Graph.fromEdgeTuples(sc.parallelize((1 to n).map(x => (0: VertexID, x: VertexID)), 3), "v") } test("Graph.fromEdgeTuples") { @@ -39,7 +39,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { withSpark { sc => val rawEdges = (0L to 98L).zip((1L to 99L) :+ 0L) val edges: RDD[Edge[Int]] = sc.parallelize(rawEdges).map { case (s, t) => Edge(s, t, 1) } - val vertices: RDD[(Vid, Boolean)] = sc.parallelize((0L until 10L).map(id => (id, true))) + val vertices: RDD[(VertexID, Boolean)] = sc.parallelize((0L until 10L).map(id => (id, true))) val graph = Graph(vertices, edges, false) assert( graph.edges.count() === rawEdges.size ) // Vertices not explicitly provided but referenced by edges should be created automatically @@ -56,7 +56,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { val n = 5 val star = starGraph(sc, n) assert(star.triplets.map(et => (et.srcId, et.dstId, et.srcAttr, et.dstAttr)).collect.toSet === - (1 to n).map(x => (0: Vid, x: Vid, "v", "v")).toSet) + (1 to n).map(x => (0: VertexID, x: VertexID, "v", "v")).toSet) } } @@ -92,7 +92,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { val p = 100 val verts = 1 to n val graph = Graph.fromEdgeTuples(sc.parallelize(verts.flatMap(x => - verts.filter(y => y % x == 0).map(y => (x: Vid, y: Vid))), p), 0) + verts.filter(y => y % x == 0).map(y => (x: VertexID, y: VertexID))), p), 0) assert(graph.edges.partitions.length === p) val partitionedGraph = graph.partitionBy(EdgePartition2D) assert(graph.edges.partitions.length === p) @@ -118,10 +118,10 @@ class GraphSuite extends FunSuite with LocalSparkContext { val star = starGraph(sc, n) // mapVertices preserving type val mappedVAttrs = star.mapVertices((vid, attr) => attr + "2") - assert(mappedVAttrs.vertices.collect.toSet === (0 to n).map(x => (x: Vid, "v2")).toSet) + assert(mappedVAttrs.vertices.collect.toSet === (0 to n).map(x => (x: VertexID, "v2")).toSet) // mapVertices changing type val mappedVAttrs2 = star.mapVertices((vid, attr) => attr.length) - assert(mappedVAttrs2.vertices.collect.toSet === (0 to n).map(x => (x: Vid, 1)).toSet) + assert(mappedVAttrs2.vertices.collect.toSet === (0 to n).map(x => (x: VertexID, 1)).toSet) } } @@ -150,7 +150,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { withSpark { sc => val n = 5 val star = starGraph(sc, n) - assert(star.reverse.outDegrees.collect.toSet === (1 to n).map(x => (x: Vid, 1)).toSet) + assert(star.reverse.outDegrees.collect.toSet === (1 to n).map(x => (x: VertexID, 1)).toSet) } } @@ -173,7 +173,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { test("mask") { withSpark { sc => val n = 5 - val vertices = sc.parallelize((0 to n).map(x => (x:Vid, x))) + val vertices = sc.parallelize((0 to n).map(x => (x:VertexID, x))) val edges = sc.parallelize((1 to n).map(x => Edge(0, x, x))) val graph: Graph[Int, Int] = Graph(vertices, edges) @@ -199,7 +199,8 @@ class GraphSuite extends FunSuite with LocalSparkContext { val n = 5 val star = starGraph(sc, n) val doubleStar = Graph.fromEdgeTuples( - sc.parallelize((1 to n).flatMap(x => List((0: Vid, x: Vid), (0: Vid, x: Vid))), 1), "v") + sc.parallelize((1 to n).flatMap(x => + List((0: VertexID, x: VertexID), (0: VertexID, x: VertexID))), 1), "v") val star2 = doubleStar.groupEdges { (a, b) => a} assert(star2.edges.collect.toArray.sorted(Edge.lexicographicOrdering[Int]) === star.edges.collect.toArray.sorted(Edge.lexicographicOrdering[Int])) @@ -218,7 +219,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { assert(neighborDegreeSums.collect().toSet === (0 to n).map(x => (x, n)).toSet) // activeSetOpt - val allPairs = for (x <- 1 to n; y <- 1 to n) yield (x: Vid, y: Vid) + val allPairs = for (x <- 1 to n; y <- 1 to n) yield (x: VertexID, y: VertexID) val complete = Graph.fromEdgeTuples(sc.parallelize(allPairs, 3), 0) val vids = complete.mapVertices((vid, attr) => vid).cache() val active = vids.vertices.filter { case (vid, attr) => attr % 2 == 0 } @@ -229,11 +230,11 @@ class GraphSuite extends FunSuite with LocalSparkContext { } Iterator((et.srcId, 1)) }, (a: Int, b: Int) => a + b, Some((active, EdgeDirection.In))).collect.toSet - assert(numEvenNeighbors === (1 to n).map(x => (x: Vid, n / 2)).toSet) + assert(numEvenNeighbors === (1 to n).map(x => (x: VertexID, n / 2)).toSet) // outerJoinVertices followed by mapReduceTriplets(activeSetOpt) - val ring = Graph.fromEdgeTuples(sc.parallelize((0 until n).map(x => (x: Vid, (x+1) % n: Vid)), 3), 0) - .mapVertices((vid, attr) => vid).cache() + val ringEdges = sc.parallelize((0 until n).map(x => (x: VertexID, (x+1) % n: VertexID)), 3) + val ring = Graph.fromEdgeTuples(ringEdges, 0) .mapVertices((vid, attr) => vid).cache() val changed = ring.vertices.filter { case (vid, attr) => attr % 2 == 1 }.mapValues(-_) val changedGraph = ring.outerJoinVertices(changed) { (vid, old, newOpt) => newOpt.getOrElse(old) } val numOddNeighbors = changedGraph.mapReduceTriplets(et => { @@ -243,7 +244,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { } Iterator((et.dstId, 1)) }, (a: Int, b: Int) => a + b, Some(changed, EdgeDirection.Out)).collect.toSet - assert(numOddNeighbors === (2 to n by 2).map(x => (x: Vid, 1)).toSet) + assert(numOddNeighbors === (2 to n by 2).map(x => (x: VertexID, 1)).toSet) } } @@ -258,7 +259,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { val neighborDegreeSums = reverseStarDegrees.mapReduceTriplets( et => Iterator((et.srcId, et.dstAttr), (et.dstId, et.srcAttr)), (a: Int, b: Int) => a + b).collect.toSet - assert(neighborDegreeSums === Set((0: Vid, n)) ++ (1 to n).map(x => (x: Vid, 0))) + assert(neighborDegreeSums === Set((0: VertexID, n)) ++ (1 to n).map(x => (x: VertexID, 0))) // outerJoinVertices preserving type val messages = reverseStar.vertices.mapValues { (vid, attr) => vid.toString } val newReverseStar = diff --git a/graph/src/test/scala/org/apache/spark/graph/PregelSuite.scala b/graph/src/test/scala/org/apache/spark/graph/PregelSuite.scala index 44182e85eeafe..de7e3872cad6f 100644 --- a/graph/src/test/scala/org/apache/spark/graph/PregelSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/PregelSuite.scala @@ -10,7 +10,8 @@ class PregelSuite extends FunSuite with LocalSparkContext { test("1 iteration") { withSpark { sc => val n = 5 - val star = Graph.fromEdgeTuples(sc.parallelize((1 to n).map(x => (0: Vid, x: Vid)), 3), "v") + val star = + Graph.fromEdgeTuples(sc.parallelize((1 to n).map(x => (0: VertexID, x: VertexID)), 3), "v") val result = Pregel(star, 0)( (vid, attr, msg) => attr, et => Iterator.empty, @@ -23,11 +24,12 @@ class PregelSuite extends FunSuite with LocalSparkContext { withSpark { sc => val n = 5 val chain = Graph.fromEdgeTuples( - sc.parallelize((1 until n).map(x => (x: Vid, x + 1: Vid)), 3), + sc.parallelize((1 until n).map(x => (x: VertexID, x + 1: VertexID)), 3), 0).cache() - assert(chain.vertices.collect.toSet === (1 to n).map(x => (x: Vid, 0)).toSet) + assert(chain.vertices.collect.toSet === (1 to n).map(x => (x: VertexID, 0)).toSet) val chainWithSeed = chain.mapVertices { (vid, attr) => if (vid == 1) 1 else 0 } - assert(chainWithSeed.vertices.collect.toSet === Set((1: Vid, 1)) ++ (2 to n).map(x => (x: Vid, 0)).toSet) + assert(chainWithSeed.vertices.collect.toSet === + 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)), diff --git a/graph/src/test/scala/org/apache/spark/graph/SerializerSuite.scala b/graph/src/test/scala/org/apache/spark/graph/SerializerSuite.scala index 4014cbe440d8d..2864ffd1ca31e 100644 --- a/graph/src/test/scala/org/apache/spark/graph/SerializerSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/SerializerSuite.scala @@ -82,7 +82,7 @@ class SerializerSuite extends FunSuite with LocalSparkContext { test("IntAggMsgSerializer") { val conf = new SparkConf(false) - val outMsg = (4: Vid, 5) + val outMsg = (4: VertexID, 5) val bout = new ByteArrayOutputStream val outStrm = new IntAggMsgSerializer(conf).newInstance().serializeStream(bout) outStrm.writeObject(outMsg) @@ -90,8 +90,8 @@ class SerializerSuite extends FunSuite with LocalSparkContext { bout.flush() val bin = new ByteArrayInputStream(bout.toByteArray) val inStrm = new IntAggMsgSerializer(conf).newInstance().deserializeStream(bin) - val inMsg1: (Vid, Int) = inStrm.readObject() - val inMsg2: (Vid, Int) = inStrm.readObject() + val inMsg1: (VertexID, Int) = inStrm.readObject() + val inMsg2: (VertexID, Int) = inStrm.readObject() assert(outMsg === inMsg1) assert(outMsg === inMsg2) @@ -102,7 +102,7 @@ class SerializerSuite extends FunSuite with LocalSparkContext { test("LongAggMsgSerializer") { val conf = new SparkConf(false) - val outMsg = (4: Vid, 1L << 32) + val outMsg = (4: VertexID, 1L << 32) val bout = new ByteArrayOutputStream val outStrm = new LongAggMsgSerializer(conf).newInstance().serializeStream(bout) outStrm.writeObject(outMsg) @@ -110,8 +110,8 @@ class SerializerSuite extends FunSuite with LocalSparkContext { bout.flush() val bin = new ByteArrayInputStream(bout.toByteArray) val inStrm = new LongAggMsgSerializer(conf).newInstance().deserializeStream(bin) - val inMsg1: (Vid, Long) = inStrm.readObject() - val inMsg2: (Vid, Long) = inStrm.readObject() + val inMsg1: (VertexID, Long) = inStrm.readObject() + val inMsg2: (VertexID, Long) = inStrm.readObject() assert(outMsg === inMsg1) assert(outMsg === inMsg2) @@ -122,7 +122,7 @@ class SerializerSuite extends FunSuite with LocalSparkContext { test("DoubleAggMsgSerializer") { val conf = new SparkConf(false) - val outMsg = (4: Vid, 5.0) + val outMsg = (4: VertexID, 5.0) val bout = new ByteArrayOutputStream val outStrm = new DoubleAggMsgSerializer(conf).newInstance().serializeStream(bout) outStrm.writeObject(outMsg) @@ -130,8 +130,8 @@ class SerializerSuite extends FunSuite with LocalSparkContext { bout.flush() val bin = new ByteArrayInputStream(bout.toByteArray) val inStrm = new DoubleAggMsgSerializer(conf).newInstance().deserializeStream(bin) - val inMsg1: (Vid, Double) = inStrm.readObject() - val inMsg2: (Vid, Double) = inStrm.readObject() + val inMsg1: (VertexID, Double) = inStrm.readObject() + val inMsg2: (VertexID, Double) = inStrm.readObject() assert(outMsg === inMsg1) assert(outMsg === inMsg2) diff --git a/graph/src/test/scala/org/apache/spark/graph/impl/EdgePartitionSuite.scala b/graph/src/test/scala/org/apache/spark/graph/impl/EdgePartitionSuite.scala index f951fd7a82741..fd0beee2f6408 100644 --- a/graph/src/test/scala/org/apache/spark/graph/impl/EdgePartitionSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/impl/EdgePartitionSuite.scala @@ -62,7 +62,7 @@ class EdgePartitionSuite extends FunSuite { test("innerJoin") { def makeEdgePartition[A: ClassTag](xs: Iterable[(Int, Int, A)]): EdgePartition[A] = { val builder = new EdgePartitionBuilder[A] - for ((src, dst, attr) <- xs) { builder.add(src: Vid, dst: Vid, attr) } + for ((src, dst, attr) <- xs) { builder.add(src: VertexID, dst: VertexID, attr) } builder.toEdgePartition } val aList = List((0, 1, 0), (1, 0, 0), (1, 2, 0), (5, 4, 0), (5, 5, 0)) From 43e1bdc80c2b19533596df74fd7b97a2d7b84bb6 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Thu, 9 Jan 2014 13:59:48 -0800 Subject: [PATCH 360/531] Pid -> PartitionID --- .../scala/org/apache/spark/graph/EdgeRDD.scala | 8 ++++---- .../scala/org/apache/spark/graph/Graph.scala | 5 +++-- .../apache/spark/graph/PartitionStrategy.scala | 16 ++++++++-------- .../org/apache/spark/graph/impl/GraphImpl.scala | 8 ++++---- .../spark/graph/impl/MessageToPartition.scala | 14 +++++++------- .../spark/graph/impl/ReplicatedVertexView.scala | 14 +++++++------- .../apache/spark/graph/impl/RoutingTable.scala | 4 ++-- .../scala/org/apache/spark/graph/package.scala | 2 +- 8 files changed, 36 insertions(+), 35 deletions(-) diff --git a/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala b/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala index fd933593523a8..78821bf568158 100644 --- a/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala +++ b/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala @@ -9,7 +9,7 @@ import org.apache.spark.storage.StorageLevel class EdgeRDD[@specialized ED: ClassTag]( - val partitionsRDD: RDD[(Pid, EdgePartition[ED])]) + val partitionsRDD: RDD[(PartitionID, EdgePartition[ED])]) extends RDD[Edge[ED]](partitionsRDD.context, List(new OneToOneDependency(partitionsRDD))) { partitionsRDD.setName("EdgeRDD") @@ -17,7 +17,7 @@ class EdgeRDD[@specialized ED: ClassTag]( override protected def getPartitions: Array[Partition] = partitionsRDD.partitions /** - * If partitionsRDD already has a partitioner, use it. Otherwise assume that the Pids in + * If partitionsRDD already has a partitioner, use it. Otherwise assume that the PartitionIDs in * partitionsRDD correspond to the actual partitions and create a new partitioner that allows * co-partitioning with partitionsRDD. */ @@ -25,7 +25,7 @@ class EdgeRDD[@specialized ED: ClassTag]( partitionsRDD.partitioner.orElse(Some(Partitioner.defaultPartitioner(partitionsRDD))) override def compute(part: Partition, context: TaskContext): Iterator[Edge[ED]] = { - firstParent[(Pid, EdgePartition[ED])].iterator(part, context).next._2.iterator + firstParent[(PartitionID, EdgePartition[ED])].iterator(part, context).next._2.iterator } override def collect(): Array[Edge[ED]] = this.map(_.copy()).collect() @@ -44,7 +44,7 @@ class EdgeRDD[@specialized ED: ClassTag]( /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */ override def cache(): EdgeRDD[ED] = persist() - def mapEdgePartitions[ED2: ClassTag](f: (Pid, EdgePartition[ED]) => EdgePartition[ED2]) + def mapEdgePartitions[ED2: ClassTag](f: (PartitionID, EdgePartition[ED]) => EdgePartition[ED2]) : EdgeRDD[ED2] = { // iter => iter.map { case (pid, ep) => (pid, f(ep)) } new EdgeRDD[ED2](partitionsRDD.mapPartitions({ iter => diff --git a/graph/src/main/scala/org/apache/spark/graph/Graph.scala b/graph/src/main/scala/org/apache/spark/graph/Graph.scala index dd0799142ef74..86282e607ebb7 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Graph.scala +++ b/graph/src/main/scala/org/apache/spark/graph/Graph.scala @@ -169,7 +169,8 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] { * @tparam ED2 the new edge data type * */ - def mapEdges[ED2: ClassTag](map: (Pid, Iterator[Edge[ED]]) => Iterator[ED2]): Graph[VD, ED2] + def mapEdges[ED2: ClassTag]( + map: (PartitionID, Iterator[Edge[ED]]) => Iterator[ED2]): Graph[VD, ED2] /** * Construct a new graph where the value of each edge is @@ -220,7 +221,7 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] { * */ def mapTriplets[ED2: ClassTag]( - map: (Pid, Iterator[EdgeTriplet[VD, ED]]) => Iterator[ED2]): + map: (PartitionID, Iterator[EdgeTriplet[VD, ED]]) => Iterator[ED2]): Graph[VD, ED2] /** diff --git a/graph/src/main/scala/org/apache/spark/graph/PartitionStrategy.scala b/graph/src/main/scala/org/apache/spark/graph/PartitionStrategy.scala index c01b4b9439589..bc05fb812c008 100644 --- a/graph/src/main/scala/org/apache/spark/graph/PartitionStrategy.scala +++ b/graph/src/main/scala/org/apache/spark/graph/PartitionStrategy.scala @@ -2,7 +2,7 @@ package org.apache.spark.graph sealed trait PartitionStrategy extends Serializable { - def getPartition(src: VertexID, dst: VertexID, numParts: Pid): Pid + def getPartition(src: VertexID, dst: VertexID, numParts: PartitionID): PartitionID } @@ -51,18 +51,18 @@ sealed trait PartitionStrategy extends Serializable { * */ case object EdgePartition2D extends PartitionStrategy { - override def getPartition(src: VertexID, dst: VertexID, numParts: Pid): Pid = { - val ceilSqrtNumParts: Pid = math.ceil(math.sqrt(numParts)).toInt + 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: Pid = ((math.abs(src) * mixingPrime) % ceilSqrtNumParts).toInt - val row: Pid = ((math.abs(dst) * mixingPrime) % ceilSqrtNumParts).toInt + val col: PartitionID = ((math.abs(src) * mixingPrime) % ceilSqrtNumParts).toInt + val row: PartitionID = ((math.abs(dst) * mixingPrime) % ceilSqrtNumParts).toInt (col * ceilSqrtNumParts + row) % numParts } } case object EdgePartition1D extends PartitionStrategy { - override def getPartition(src: VertexID, dst: VertexID, numParts: Pid): Pid = { + override def getPartition(src: VertexID, dst: VertexID, numParts: PartitionID): PartitionID = { val mixingPrime: VertexID = 1125899906842597L (math.abs(src) * mixingPrime).toInt % numParts } @@ -74,7 +74,7 @@ case object EdgePartition1D extends PartitionStrategy { * random vertex cut. */ case object RandomVertexCut extends PartitionStrategy { - override def getPartition(src: VertexID, dst: VertexID, numParts: Pid): Pid = { + override def getPartition(src: VertexID, dst: VertexID, numParts: PartitionID): PartitionID = { math.abs((src, dst).hashCode()) % numParts } } @@ -86,7 +86,7 @@ case object RandomVertexCut extends PartitionStrategy { * will end up on the same partition. */ case object CanonicalRandomVertexCut extends PartitionStrategy { - override def getPartition(src: VertexID, dst: VertexID, numParts: Pid): Pid = { + 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 diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala index 6eb401b3b57ac..8f42e7d5929dd 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala @@ -69,7 +69,7 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( val numPartitions = edges.partitions.size val edTag = classTag[ED] val newEdges = new EdgeRDD(edges.map { e => - val part: Pid = partitionStrategy.getPartition(e.srcId, e.dstId, numPartitions) + val part: PartitionID = partitionStrategy.getPartition(e.srcId, e.dstId, numPartitions) // Should we be using 3-tuple or an optimized class new MessageToPartition(part, (e.srcId, e.dstId, e.attr)) @@ -173,13 +173,13 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( } override def mapEdges[ED2: ClassTag]( - f: (Pid, Iterator[Edge[ED]]) => Iterator[ED2]): Graph[VD, ED2] = { + f: (PartitionID, Iterator[Edge[ED]]) => Iterator[ED2]): Graph[VD, ED2] = { val newETable = edges.mapEdgePartitions((pid, part) => part.map(f(pid, part.iterator))) new GraphImpl(vertices, newETable , routingTable, replicatedVertexView) } override def mapTriplets[ED2: ClassTag]( - f: (Pid, Iterator[EdgeTriplet[VD, ED]]) => Iterator[ED2]): Graph[VD, ED2] = { + f: (PartitionID, Iterator[EdgeTriplet[VD, ED]]) => Iterator[ED2]): Graph[VD, ED2] = { // Use an explicit manifest in PrimitiveKeyOpenHashMap init so we don't pull in the implicit // manifest from GraphImpl (which would require serializing GraphImpl). val vdTag = classTag[VD] @@ -354,7 +354,7 @@ object GraphImpl { } def fromEdgePartitions[VD: ClassTag, ED: ClassTag]( - edgePartitions: RDD[(Pid, EdgePartition[ED])], + edgePartitions: RDD[(PartitionID, EdgePartition[ED])], defaultVertexAttr: VD): GraphImpl[VD, ED] = { fromEdgeRDD(new EdgeRDD(edgePartitions), defaultVertexAttr) } diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/MessageToPartition.scala b/graph/src/main/scala/org/apache/spark/graph/impl/MessageToPartition.scala index 2d03f75a28a25..b2fa7284827a4 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/MessageToPartition.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/MessageToPartition.scala @@ -3,15 +3,15 @@ package org.apache.spark.graph.impl import scala.reflect.{classTag, ClassTag} import org.apache.spark.Partitioner -import org.apache.spark.graph.{Pid, VertexID} +import org.apache.spark.graph.{PartitionID, VertexID} import org.apache.spark.rdd.{ShuffledRDD, RDD} class VertexBroadcastMsg[@specialized(Int, Long, Double, Boolean) T]( - @transient var partition: Pid, + @transient var partition: PartitionID, var vid: VertexID, var data: T) - extends Product2[Pid, (VertexID, T)] with Serializable { + extends Product2[PartitionID, (VertexID, T)] with Serializable { override def _1 = partition @@ -27,9 +27,9 @@ class VertexBroadcastMsg[@specialized(Int, Long, Double, Boolean) T]( * @param data value to send */ class MessageToPartition[@specialized(Int, Long, Double, Char, Boolean/*, AnyRef*/) T]( - @transient var partition: Pid, + @transient var partition: PartitionID, var data: T) - extends Product2[Pid, T] with Serializable { + extends Product2[PartitionID, T] with Serializable { override def _1 = partition @@ -41,7 +41,7 @@ class MessageToPartition[@specialized(Int, Long, Double, Char, Boolean/*, AnyRef class VertexBroadcastMsgRDDFunctions[T: ClassTag](self: RDD[VertexBroadcastMsg[T]]) { def partitionBy(partitioner: Partitioner): RDD[VertexBroadcastMsg[T]] = { - val rdd = new ShuffledRDD[Pid, (VertexID, T), VertexBroadcastMsg[T]](self, partitioner) + val rdd = new ShuffledRDD[PartitionID, (VertexID, T), VertexBroadcastMsg[T]](self, partitioner) // Set a custom serializer if the data is of int or double type. if (classTag[T] == ClassTag.Int) { @@ -62,7 +62,7 @@ class MsgRDDFunctions[T: ClassTag](self: RDD[MessageToPartition[T]]) { * Return a copy of the RDD partitioned using the specified partitioner. */ def partitionBy(partitioner: Partitioner): RDD[MessageToPartition[T]] = { - new ShuffledRDD[Pid, T, MessageToPartition[T]](self, partitioner) + new ShuffledRDD[PartitionID, T, MessageToPartition[T]](self, partitioner) } } diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/ReplicatedVertexView.scala b/graph/src/main/scala/org/apache/spark/graph/impl/ReplicatedVertexView.scala index 9d2d242ffa8d9..7d29861db1e3a 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/ReplicatedVertexView.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/ReplicatedVertexView.scala @@ -46,12 +46,12 @@ class ReplicatedVertexView[VD: ClassTag]( }, preservesPartitioning = true).cache().setName("ReplicatedVertexView localVertexIDMap") } - private lazy val bothAttrs: RDD[(Pid, VertexPartition[VD])] = create(true, true) - private lazy val srcAttrOnly: RDD[(Pid, VertexPartition[VD])] = create(true, false) - private lazy val dstAttrOnly: RDD[(Pid, VertexPartition[VD])] = create(false, true) - private lazy val noAttrs: RDD[(Pid, VertexPartition[VD])] = create(false, false) + private lazy val bothAttrs: RDD[(PartitionID, VertexPartition[VD])] = create(true, true) + private lazy val srcAttrOnly: RDD[(PartitionID, VertexPartition[VD])] = create(true, false) + private lazy val dstAttrOnly: RDD[(PartitionID, VertexPartition[VD])] = create(false, true) + private lazy val noAttrs: RDD[(PartitionID, VertexPartition[VD])] = create(false, false) - def get(includeSrc: Boolean, includeDst: Boolean): RDD[(Pid, VertexPartition[VD])] = { + def get(includeSrc: Boolean, includeDst: Boolean): RDD[(PartitionID, VertexPartition[VD])] = { (includeSrc, includeDst) match { case (true, true) => bothAttrs case (true, false) => srcAttrOnly @@ -63,7 +63,7 @@ class ReplicatedVertexView[VD: ClassTag]( def get( includeSrc: Boolean, includeDst: Boolean, - actives: VertexRDD[_]): RDD[(Pid, VertexPartition[VD])] = { + actives: VertexRDD[_]): RDD[(PartitionID, VertexPartition[VD])] = { // Ship active sets to edge partitions using vertexPlacement, but ignoring includeSrc and // includeDst. These flags govern attribute shipping, but the activeness of a vertex must be // shipped to all edges mentioning that vertex, regardless of whether the vertex attribute is @@ -81,7 +81,7 @@ class ReplicatedVertexView[VD: ClassTag]( } private def create(includeSrc: Boolean, includeDst: Boolean) - : RDD[(Pid, VertexPartition[VD])] = { + : RDD[(PartitionID, VertexPartition[VD])] = { val vdTag = classTag[VD] // Ship vertex attributes to edge partitions according to vertexPlacement diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/RoutingTable.scala b/graph/src/main/scala/org/apache/spark/graph/impl/RoutingTable.scala index 9e6f78197e7f9..96d9e9d7f8c8e 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/RoutingTable.scala +++ b/graph/src/main/scala/org/apache/spark/graph/impl/RoutingTable.scala @@ -30,8 +30,8 @@ class RoutingTable(edges: EdgeRDD[_], vertices: VertexRDD[_]) { private def createPid2Vid( includeSrcAttr: Boolean, includeDstAttr: Boolean): RDD[Array[Array[VertexID]]] = { // Determine which vertices each edge partition needs by creating a mapping from vid to pid. - val vid2pid: RDD[(VertexID, Pid)] = edges.partitionsRDD.mapPartitions { iter => - val (pid: Pid, edgePartition: EdgePartition[_]) = iter.next() + val vid2pid: RDD[(VertexID, PartitionID)] = edges.partitionsRDD.mapPartitions { iter => + val (pid: PartitionID, edgePartition: EdgePartition[_]) = iter.next() val numEdges = edgePartition.size val vSet = new VertexSet if (includeSrcAttr) { // Add src vertices to the set. diff --git a/graph/src/main/scala/org/apache/spark/graph/package.scala b/graph/src/main/scala/org/apache/spark/graph/package.scala index 823d47c359b09..b98a11b918a21 100644 --- a/graph/src/main/scala/org/apache/spark/graph/package.scala +++ b/graph/src/main/scala/org/apache/spark/graph/package.scala @@ -8,7 +8,7 @@ package object graph { type VertexID = Long // TODO: Consider using Char. - type Pid = Int + type PartitionID = Int type VertexSet = OpenHashSet[VertexID] From 100718bcd3f6ade1a93256458ec1528bb9142b5e Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Thu, 9 Jan 2014 14:05:09 -0800 Subject: [PATCH 361/531] Svdpp -> SVDPlusPlus --- .../algorithms/{Svdpp.scala => SVDPlusPlus.scala} | 12 ++++++------ .../{SvdppSuite.scala => SVDPlusPlusSuite.scala} | 10 +++++----- 2 files changed, 11 insertions(+), 11 deletions(-) rename graph/src/main/scala/org/apache/spark/graph/algorithms/{Svdpp.scala => SVDPlusPlus.scala} (89%) rename graph/src/test/scala/org/apache/spark/graph/algorithms/{SvdppSuite.scala => SVDPlusPlusSuite.scala} (72%) diff --git a/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala b/graph/src/main/scala/org/apache/spark/graph/algorithms/SVDPlusPlus.scala similarity index 89% rename from graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala rename to graph/src/main/scala/org/apache/spark/graph/algorithms/SVDPlusPlus.scala index 85fa23d309469..083aa305388e8 100644 --- a/graph/src/main/scala/org/apache/spark/graph/algorithms/Svdpp.scala +++ b/graph/src/main/scala/org/apache/spark/graph/algorithms/SVDPlusPlus.scala @@ -5,7 +5,7 @@ import org.apache.spark.graph._ import scala.util.Random import org.apache.commons.math.linear._ -class SvdppConf( // Svdpp parameters +class SVDPlusPlusConf( // SVDPlusPlus parameters var rank: Int, var maxIters: Int, var minVal: Double, @@ -15,7 +15,7 @@ class SvdppConf( // Svdpp parameters var gamma6: Double, var gamma7: Double) extends Serializable -object Svdpp { +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]]. @@ -23,12 +23,12 @@ object Svdpp { * * @param edges edges for constructing the graph * - * @param conf Svdpp parameters + * @param conf SVDPlusPlus parameters * * @return a graph with vertex attributes containing the trained model */ - def run(edges: RDD[Edge[Double]], conf: SvdppConf): (Graph[(RealVector, RealVector, Double, Double), Double], Double) = { + 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) = { @@ -55,7 +55,7 @@ object Svdpp { (vd._1, vd._2, msg.get._2 / msg.get._1, 1.0 / scala.math.sqrt(msg.get._1)) } - def mapTrainF(conf: SvdppConf, 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) @@ -85,7 +85,7 @@ object Svdpp { } // calculate error on training set - def mapTestF(conf: SvdppConf, 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) diff --git a/graph/src/test/scala/org/apache/spark/graph/algorithms/SvdppSuite.scala b/graph/src/test/scala/org/apache/spark/graph/algorithms/SVDPlusPlusSuite.scala similarity index 72% rename from graph/src/test/scala/org/apache/spark/graph/algorithms/SvdppSuite.scala rename to graph/src/test/scala/org/apache/spark/graph/algorithms/SVDPlusPlusSuite.scala index 411dd3d336c2a..a0a6eb33e36fc 100644 --- a/graph/src/test/scala/org/apache/spark/graph/algorithms/SvdppSuite.scala +++ b/graph/src/test/scala/org/apache/spark/graph/algorithms/SVDPlusPlusSuite.scala @@ -9,21 +9,21 @@ import org.apache.spark.graph.util.GraphGenerators import org.apache.spark.rdd._ -class SvdppSuite extends FunSuite with LocalSparkContext { +class SVDPlusPlusSuite extends FunSuite with LocalSparkContext { test("Test SVD++ with mean square error on training set") { withSpark { sc => - val SvdppErr = 8.0 + val svdppErr = 8.0 val edges = sc.textFile("mllib/data/als/test.data").map { line => val fields = line.split(",") Edge(fields(0).toLong * 2, fields(1).toLong * 2 + 1, fields(2).toDouble) } - val conf = new SvdppConf(10, 2, 0.0, 5.0, 0.007, 0.007, 0.005, 0.015) // 2 iterations - var (graph, u) = Svdpp.run(edges, conf) + val conf = new SVDPlusPlusConf(10, 2, 0.0, 5.0, 0.007, 0.007, 0.005, 0.015) // 2 iterations + var (graph, u) = SVDPlusPlus.run(edges, conf) val err = graph.vertices.collect.map{ case (vid, vd) => if (vid % 2 == 1) vd._4 else 0.0 }.reduce(_ + _) / graph.triplets.collect.size - assert(err <= SvdppErr) + assert(err <= svdppErr) } } From 731f56f309914e3fc7c22c8ef1c8cb9dd40d42c1 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Thu, 9 Jan 2014 14:31:33 -0800 Subject: [PATCH 362/531] graph -> graphx --- {graph => graphx}/pom.xml | 0 .../scala/org/apache/spark/graphx}/Analytics.scala | 8 ++++---- .../main/scala/org/apache/spark/graphx}/Edge.scala | 2 +- .../org/apache/spark/graphx}/EdgeDirection.scala | 4 ++-- .../scala/org/apache/spark/graphx}/EdgeRDD.scala | 4 ++-- .../scala/org/apache/spark/graphx}/EdgeTriplet.scala | 4 ++-- .../main/scala/org/apache/spark/graphx}/Graph.scala | 4 ++-- .../apache/spark/graphx}/GraphKryoRegistrator.scala | 4 ++-- .../scala/org/apache/spark/graphx}/GraphLab.scala | 2 +- .../scala/org/apache/spark/graphx}/GraphLoader.scala | 6 +++--- .../scala/org/apache/spark/graphx}/GraphOps.scala | 2 +- .../org/apache/spark/graphx}/PartitionStrategy.scala | 2 +- .../main/scala/org/apache/spark/graphx}/Pregel.scala | 2 +- .../scala/org/apache/spark/graphx}/VertexRDD.scala | 6 +++--- .../graphx}/algorithms/ConnectedComponents.scala | 4 ++-- .../apache/spark/graphx}/algorithms/PageRank.scala | 4 ++-- .../spark/graphx}/algorithms/SVDPlusPlus.scala | 4 ++-- .../algorithms/StronglyConnectedComponents.scala | 4 ++-- .../spark/graphx}/algorithms/TriangleCount.scala | 4 ++-- .../apache/spark/graphx}/impl/EdgePartition.scala | 4 ++-- .../spark/graphx}/impl/EdgePartitionBuilder.scala | 4 ++-- .../spark/graphx}/impl/EdgeTripletIterator.scala | 4 ++-- .../org/apache/spark/graphx}/impl/GraphImpl.scala | 10 +++++----- .../spark/graphx}/impl/MessageToPartition.scala | 4 ++-- .../spark/graphx}/impl/ReplicatedVertexView.scala | 4 ++-- .../org/apache/spark/graphx}/impl/RoutingTable.scala | 4 ++-- .../org/apache/spark/graphx}/impl/Serializers.scala | 4 ++-- .../apache/spark/graphx}/impl/VertexPartition.scala | 8 ++++---- .../scala/org/apache/spark/graphx}/package.scala | 2 +- .../org/apache/spark/graphx}/perf/BagelTest.scala | 4 ++-- .../org/apache/spark/graphx}/perf/SparkTest.scala | 4 ++-- .../apache/spark/graphx}/util/BytecodeUtils.scala | 2 +- .../apache/spark/graphx}/util/GraphGenerators.scala | 12 ++++++------ .../org/apache/spark/graphx}/util/HashUtils.scala | 2 +- .../src/test/resources/log4j.properties | 0 .../org/apache/spark/graphx}/GraphOpsSuite.scala | 6 +++--- .../scala/org/apache/spark/graphx}/GraphSuite.scala | 4 ++-- .../org/apache/spark/graphx}/LocalSparkContext.scala | 4 ++-- .../scala/org/apache/spark/graphx}/PregelSuite.scala | 2 +- .../org/apache/spark/graphx}/SerializerSuite.scala | 6 +++--- .../org/apache/spark/graphx}/VertexRDDSuite.scala | 6 +++--- .../algorithms/ConnectedComponentsSuite.scala | 6 +++--- .../spark/graphx}/algorithms/PageRankSuite.scala | 6 +++--- .../spark/graphx}/algorithms/SVDPlusPlusSuite.scala | 6 +++--- .../StronglyConnectedComponentsSuite.scala | 6 +++--- .../graphx}/algorithms/TriangleCountSuite.scala | 6 +++--- .../spark/graphx}/impl/EdgePartitionSuite.scala | 4 ++-- .../spark/graphx}/impl/VertexPartitionSuite.scala | 4 ++-- .../spark/graphx}/util/BytecodeUtilsSuite.scala | 2 +- project/SparkBuild.scala | 12 ++++++------ 50 files changed, 111 insertions(+), 111 deletions(-) rename {graph => graphx}/pom.xml (100%) rename {graph/src/main/scala/org/apache/spark/graph => graphx/src/main/scala/org/apache/spark/graphx}/Analytics.scala (99%) rename {graph/src/main/scala/org/apache/spark/graph => graphx/src/main/scala/org/apache/spark/graphx}/Edge.scala (97%) rename {graph/src/main/scala/org/apache/spark/graph => graphx/src/main/scala/org/apache/spark/graphx}/EdgeDirection.scala (88%) rename {graph/src/main/scala/org/apache/spark/graph => graphx/src/main/scala/org/apache/spark/graphx}/EdgeRDD.scala (96%) rename {graph/src/main/scala/org/apache/spark/graph => graphx/src/main/scala/org/apache/spark/graphx}/EdgeTriplet.scala (95%) rename {graph/src/main/scala/org/apache/spark/graph => graphx/src/main/scala/org/apache/spark/graphx}/Graph.scala (99%) rename {graph/src/main/scala/org/apache/spark/graph => graphx/src/main/scala/org/apache/spark/graphx}/GraphKryoRegistrator.scala (92%) rename {graph/src/main/scala/org/apache/spark/graph => graphx/src/main/scala/org/apache/spark/graphx}/GraphLab.scala (99%) rename {graph/src/main/scala/org/apache/spark/graph => graphx/src/main/scala/org/apache/spark/graphx}/GraphLoader.scala (95%) rename {graph/src/main/scala/org/apache/spark/graph => graphx/src/main/scala/org/apache/spark/graphx}/GraphOps.scala (99%) rename {graph/src/main/scala/org/apache/spark/graph => graphx/src/main/scala/org/apache/spark/graphx}/PartitionStrategy.scala (99%) rename {graph/src/main/scala/org/apache/spark/graph => graphx/src/main/scala/org/apache/spark/graphx}/Pregel.scala (99%) rename {graph/src/main/scala/org/apache/spark/graph => graphx/src/main/scala/org/apache/spark/graphx}/VertexRDD.scala (98%) rename {graph/src/main/scala/org/apache/spark/graph => graphx/src/main/scala/org/apache/spark/graphx}/algorithms/ConnectedComponents.scala (94%) rename {graph/src/main/scala/org/apache/spark/graph => graphx/src/main/scala/org/apache/spark/graphx}/algorithms/PageRank.scala (99%) rename {graph/src/main/scala/org/apache/spark/graph => graphx/src/main/scala/org/apache/spark/graphx}/algorithms/SVDPlusPlus.scala (98%) rename {graph/src/main/scala/org/apache/spark/graph => graphx/src/main/scala/org/apache/spark/graphx}/algorithms/StronglyConnectedComponents.scala (97%) rename {graph/src/main/scala/org/apache/spark/graph => graphx/src/main/scala/org/apache/spark/graphx}/algorithms/TriangleCount.scala (97%) rename {graph/src/main/scala/org/apache/spark/graph => graphx/src/main/scala/org/apache/spark/graphx}/impl/EdgePartition.scala (99%) rename {graph/src/main/scala/org/apache/spark/graph => graphx/src/main/scala/org/apache/spark/graphx}/impl/EdgePartitionBuilder.scala (95%) rename {graph/src/main/scala/org/apache/spark/graph => graphx/src/main/scala/org/apache/spark/graphx}/impl/EdgeTripletIterator.scala (94%) rename {graph/src/main/scala/org/apache/spark/graph => graphx/src/main/scala/org/apache/spark/graphx}/impl/GraphImpl.scala (98%) rename {graph/src/main/scala/org/apache/spark/graph => graphx/src/main/scala/org/apache/spark/graphx}/impl/MessageToPartition.scala (96%) rename {graph/src/main/scala/org/apache/spark/graph => graphx/src/main/scala/org/apache/spark/graphx}/impl/ReplicatedVertexView.scala (99%) rename {graph/src/main/scala/org/apache/spark/graph => graphx/src/main/scala/org/apache/spark/graphx}/impl/RoutingTable.scala (97%) rename {graph/src/main/scala/org/apache/spark/graph => graphx/src/main/scala/org/apache/spark/graphx}/impl/Serializers.scala (99%) rename {graph/src/main/scala/org/apache/spark/graph => graphx/src/main/scala/org/apache/spark/graphx}/impl/VertexPartition.scala (98%) rename {graph/src/main/scala/org/apache/spark/graph => graphx/src/main/scala/org/apache/spark/graphx}/package.scala (94%) rename {graph/src/main/scala/org/apache/spark/graph => graphx/src/main/scala/org/apache/spark/graphx}/perf/BagelTest.scala (97%) rename {graph/src/main/scala/org/apache/spark/graph => graphx/src/main/scala/org/apache/spark/graphx}/perf/SparkTest.scala (97%) rename {graph/src/main/scala/org/apache/spark/graph => graphx/src/main/scala/org/apache/spark/graphx}/util/BytecodeUtils.scala (99%) rename {graph/src/main/scala/org/apache/spark/graph => graphx/src/main/scala/org/apache/spark/graphx}/util/GraphGenerators.scala (96%) rename {graph/src/main/scala/org/apache/spark/graph => graphx/src/main/scala/org/apache/spark/graphx}/util/HashUtils.scala (91%) rename {graph => graphx}/src/test/resources/log4j.properties (100%) rename {graph/src/test/scala/org/apache/spark/graph => graphx/src/test/scala/org/apache/spark/graphx}/GraphOpsSuite.scala (96%) rename {graph/src/test/scala/org/apache/spark/graph => graphx/src/test/scala/org/apache/spark/graphx}/GraphSuite.scala (99%) rename {graph/src/test/scala/org/apache/spark/graph => graphx/src/test/scala/org/apache/spark/graphx}/LocalSparkContext.scala (92%) rename {graph/src/test/scala/org/apache/spark/graph => graphx/src/test/scala/org/apache/spark/graphx}/PregelSuite.scala (97%) rename {graph/src/test/scala/org/apache/spark/graph => graphx/src/test/scala/org/apache/spark/graphx}/SerializerSuite.scala (98%) rename {graph/src/test/scala/org/apache/spark/graph => graphx/src/test/scala/org/apache/spark/graphx}/VertexRDDSuite.scala (95%) rename {graph/src/test/scala/org/apache/spark/graph => graphx/src/test/scala/org/apache/spark/graphx}/algorithms/ConnectedComponentsSuite.scala (95%) rename {graph/src/test/scala/org/apache/spark/graph => graphx/src/test/scala/org/apache/spark/graphx}/algorithms/PageRankSuite.scala (97%) rename {graph/src/test/scala/org/apache/spark/graph => graphx/src/test/scala/org/apache/spark/graphx}/algorithms/SVDPlusPlusSuite.scala (87%) rename {graph/src/test/scala/org/apache/spark/graph => graphx/src/test/scala/org/apache/spark/graphx}/algorithms/StronglyConnectedComponentsSuite.scala (92%) rename {graph/src/test/scala/org/apache/spark/graph => graphx/src/test/scala/org/apache/spark/graphx}/algorithms/TriangleCountSuite.scala (94%) rename {graph/src/test/scala/org/apache/spark/graph => graphx/src/test/scala/org/apache/spark/graphx}/impl/EdgePartitionSuite.scala (97%) rename {graph/src/test/scala/org/apache/spark/graph => graphx/src/test/scala/org/apache/spark/graphx}/impl/VertexPartitionSuite.scala (98%) rename {graph/src/test/scala/org/apache/spark/graph => graphx/src/test/scala/org/apache/spark/graphx}/util/BytecodeUtilsSuite.scala (98%) diff --git a/graph/pom.xml b/graphx/pom.xml similarity index 100% rename from graph/pom.xml rename to graphx/pom.xml diff --git a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala b/graphx/src/main/scala/org/apache/spark/graphx/Analytics.scala similarity index 99% rename from graph/src/main/scala/org/apache/spark/graph/Analytics.scala rename to graphx/src/main/scala/org/apache/spark/graphx/Analytics.scala index 14b9be73f1651..0cafc3fdf94ce 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Analytics.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Analytics.scala @@ -1,7 +1,7 @@ -package org.apache.spark.graph +package org.apache.spark.graphx import org.apache.spark._ -import org.apache.spark.graph.algorithms._ +import org.apache.spark.graphx.algorithms._ /** @@ -49,7 +49,7 @@ object Analytics extends Logging { val serializer = "org.apache.spark.serializer.KryoSerializer" System.setProperty("spark.serializer", serializer) //System.setProperty("spark.shuffle.compress", "false") - System.setProperty("spark.kryo.registrator", "org.apache.spark.graph.GraphKryoRegistrator") + System.setProperty("spark.kryo.registrator", "org.apache.spark.graphx.GraphKryoRegistrator") taskType match { case "pagerank" => { @@ -400,7 +400,7 @@ object Analytics extends Logging { // System.setProperty("spark.serializer", "spark.KryoSerializer") // //System.setProperty("spark.shuffle.compress", "false") - // System.setProperty("spark.kryo.registrator", "spark.graph.GraphKryoRegistrator") + // System.setProperty("spark.kryo.registrator", "spark.graphx.GraphKryoRegistrator") // taskType match { // case "pagerank" => { diff --git a/graph/src/main/scala/org/apache/spark/graph/Edge.scala b/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala similarity index 97% rename from graph/src/main/scala/org/apache/spark/graph/Edge.scala rename to graphx/src/main/scala/org/apache/spark/graphx/Edge.scala index 19c28bea685ba..29b46674f149c 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Edge.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala @@ -1,4 +1,4 @@ -package org.apache.spark.graph +package org.apache.spark.graphx /** diff --git a/graph/src/main/scala/org/apache/spark/graph/EdgeDirection.scala b/graphx/src/main/scala/org/apache/spark/graphx/EdgeDirection.scala similarity index 88% rename from graph/src/main/scala/org/apache/spark/graph/EdgeDirection.scala rename to graphx/src/main/scala/org/apache/spark/graphx/EdgeDirection.scala index a1468a152baa9..785f941650139 100644 --- a/graph/src/main/scala/org/apache/spark/graph/EdgeDirection.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeDirection.scala @@ -1,4 +1,4 @@ -package org.apache.spark.graph +package org.apache.spark.graphx /** @@ -7,7 +7,7 @@ package org.apache.spark.graph */ sealed abstract class EdgeDirection { /** - * Reverse the direction of an edge. An in becomes out, + * Reverse the direction of an edge. An in becomes out, * out becomes in and both remains both. */ def reverse: EdgeDirection = this match { diff --git a/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala b/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala similarity index 96% rename from graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala rename to graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala index 78821bf568158..e4ef460e6f88a 100644 --- a/graph/src/main/scala/org/apache/spark/graph/EdgeRDD.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala @@ -1,9 +1,9 @@ -package org.apache.spark.graph +package org.apache.spark.graphx import scala.reflect.{classTag, ClassTag} import org.apache.spark.{OneToOneDependency, Partition, Partitioner, TaskContext} -import org.apache.spark.graph.impl.EdgePartition +import org.apache.spark.graphx.impl.EdgePartition import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel diff --git a/graph/src/main/scala/org/apache/spark/graph/EdgeTriplet.scala b/graphx/src/main/scala/org/apache/spark/graphx/EdgeTriplet.scala similarity index 95% rename from graph/src/main/scala/org/apache/spark/graph/EdgeTriplet.scala rename to graphx/src/main/scala/org/apache/spark/graphx/EdgeTriplet.scala index a5103ed3cbffc..b0565b7e0e07c 100644 --- a/graph/src/main/scala/org/apache/spark/graph/EdgeTriplet.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeTriplet.scala @@ -1,6 +1,6 @@ -package org.apache.spark.graph +package org.apache.spark.graphx -import org.apache.spark.graph.impl.VertexPartition +import org.apache.spark.graphx.impl.VertexPartition /** * An edge triplet represents two vertices and edge along with their diff --git a/graph/src/main/scala/org/apache/spark/graph/Graph.scala b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala similarity index 99% rename from graph/src/main/scala/org/apache/spark/graph/Graph.scala rename to graphx/src/main/scala/org/apache/spark/graphx/Graph.scala index 86282e607ebb7..2b7c0a2583f21 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Graph.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala @@ -1,8 +1,8 @@ -package org.apache.spark.graph +package org.apache.spark.graphx import scala.reflect.ClassTag -import org.apache.spark.graph.impl._ +import org.apache.spark.graphx.impl._ import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphKryoRegistrator.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala similarity index 92% rename from graph/src/main/scala/org/apache/spark/graph/GraphKryoRegistrator.scala rename to graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala index 296f3848f1c61..f8aab951f0637 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphKryoRegistrator.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala @@ -1,8 +1,8 @@ -package org.apache.spark.graph +package org.apache.spark.graphx import com.esotericsoftware.kryo.Kryo -import org.apache.spark.graph.impl._ +import org.apache.spark.graphx.impl._ import org.apache.spark.serializer.KryoRegistrator import org.apache.spark.util.collection.BitSet import org.apache.spark.util.BoundedPriorityQueue diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphLab.scala similarity index 99% rename from graph/src/main/scala/org/apache/spark/graph/GraphLab.scala rename to graphx/src/main/scala/org/apache/spark/graphx/GraphLab.scala index 22f48540193ea..437288405f307 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphLab.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphLab.scala @@ -1,4 +1,4 @@ -package org.apache.spark.graph +package org.apache.spark.graphx import scala.reflect.ClassTag diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala similarity index 95% rename from graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala rename to graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala index 7daac4fcc56c3..473cfb18cf481 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphLoader.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala @@ -1,11 +1,11 @@ -package org.apache.spark.graph +package org.apache.spark.graphx import java.util.{Arrays => JArrays} import scala.reflect.ClassTag -import org.apache.spark.graph.impl.EdgePartitionBuilder +import org.apache.spark.graphx.impl.EdgePartitionBuilder import org.apache.spark.{Logging, SparkContext} -import org.apache.spark.graph.impl.{EdgePartition, GraphImpl} +import org.apache.spark.graphx.impl.{EdgePartition, GraphImpl} import org.apache.spark.util.collection.PrimitiveVector diff --git a/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala similarity index 99% rename from graph/src/main/scala/org/apache/spark/graph/GraphOps.scala rename to graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala index e41287c1ed4f1..cacfcb1c90f49 100644 --- a/graph/src/main/scala/org/apache/spark/graph/GraphOps.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala @@ -1,4 +1,4 @@ -package org.apache.spark.graph +package org.apache.spark.graphx import scala.reflect.ClassTag diff --git a/graph/src/main/scala/org/apache/spark/graph/PartitionStrategy.scala b/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala similarity index 99% rename from graph/src/main/scala/org/apache/spark/graph/PartitionStrategy.scala rename to graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala index bc05fb812c008..5e80a535f1c81 100644 --- a/graph/src/main/scala/org/apache/spark/graph/PartitionStrategy.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala @@ -1,4 +1,4 @@ -package org.apache.spark.graph +package org.apache.spark.graphx sealed trait PartitionStrategy extends Serializable { diff --git a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala b/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala similarity index 99% rename from graph/src/main/scala/org/apache/spark/graph/Pregel.scala rename to graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala index 3b84e2e5e4195..8ddb78813557f 100644 --- a/graph/src/main/scala/org/apache/spark/graph/Pregel.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala @@ -1,4 +1,4 @@ -package org.apache.spark.graph +package org.apache.spark.graphx import scala.reflect.ClassTag diff --git a/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala similarity index 98% rename from graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala rename to graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala index 25b0aed85aae9..cfee9b089f9e4 100644 --- a/graph/src/main/scala/org/apache/spark/graph/VertexRDD.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.graph +package org.apache.spark.graphx import scala.reflect.ClassTag @@ -24,8 +24,8 @@ import org.apache.spark.SparkContext._ import org.apache.spark.rdd._ import org.apache.spark.storage.StorageLevel -import org.apache.spark.graph.impl.MsgRDDFunctions -import org.apache.spark.graph.impl.VertexPartition +import org.apache.spark.graphx.impl.MsgRDDFunctions +import org.apache.spark.graphx.impl.VertexPartition /** diff --git a/graph/src/main/scala/org/apache/spark/graph/algorithms/ConnectedComponents.scala b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/ConnectedComponents.scala similarity index 94% rename from graph/src/main/scala/org/apache/spark/graph/algorithms/ConnectedComponents.scala rename to graphx/src/main/scala/org/apache/spark/graphx/algorithms/ConnectedComponents.scala index 2a6b8c0999ab0..a0dd36da60f32 100644 --- a/graph/src/main/scala/org/apache/spark/graph/algorithms/ConnectedComponents.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/ConnectedComponents.scala @@ -1,6 +1,6 @@ -package org.apache.spark.graph.algorithms +package org.apache.spark.graphx.algorithms -import org.apache.spark.graph._ +import org.apache.spark.graphx._ object ConnectedComponents { diff --git a/graph/src/main/scala/org/apache/spark/graph/algorithms/PageRank.scala b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/PageRank.scala similarity index 99% rename from graph/src/main/scala/org/apache/spark/graph/algorithms/PageRank.scala rename to graphx/src/main/scala/org/apache/spark/graphx/algorithms/PageRank.scala index 26b8dc5ab60c6..0292b7316d392 100644 --- a/graph/src/main/scala/org/apache/spark/graph/algorithms/PageRank.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/PageRank.scala @@ -1,7 +1,7 @@ -package org.apache.spark.graph.algorithms +package org.apache.spark.graphx.algorithms import org.apache.spark.Logging -import org.apache.spark.graph._ +import org.apache.spark.graphx._ object PageRank extends Logging { diff --git a/graph/src/main/scala/org/apache/spark/graph/algorithms/SVDPlusPlus.scala b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/SVDPlusPlus.scala similarity index 98% rename from graph/src/main/scala/org/apache/spark/graph/algorithms/SVDPlusPlus.scala rename to graphx/src/main/scala/org/apache/spark/graphx/algorithms/SVDPlusPlus.scala index 083aa305388e8..8fdfa3d9077a4 100644 --- a/graph/src/main/scala/org/apache/spark/graph/algorithms/SVDPlusPlus.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/SVDPlusPlus.scala @@ -1,7 +1,7 @@ -package org.apache.spark.graph.algorithms +package org.apache.spark.graphx.algorithms import org.apache.spark.rdd._ -import org.apache.spark.graph._ +import org.apache.spark.graphx._ import scala.util.Random import org.apache.commons.math.linear._ diff --git a/graph/src/main/scala/org/apache/spark/graph/algorithms/StronglyConnectedComponents.scala b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/StronglyConnectedComponents.scala similarity index 97% rename from graph/src/main/scala/org/apache/spark/graph/algorithms/StronglyConnectedComponents.scala rename to graphx/src/main/scala/org/apache/spark/graphx/algorithms/StronglyConnectedComponents.scala index 8031aa10ce3ea..f64fc3ef0f52a 100644 --- a/graph/src/main/scala/org/apache/spark/graph/algorithms/StronglyConnectedComponents.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/StronglyConnectedComponents.scala @@ -1,6 +1,6 @@ -package org.apache.spark.graph.algorithms +package org.apache.spark.graphx.algorithms -import org.apache.spark.graph._ +import org.apache.spark.graphx._ object StronglyConnectedComponents { diff --git a/graph/src/main/scala/org/apache/spark/graph/algorithms/TriangleCount.scala b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/TriangleCount.scala similarity index 97% rename from graph/src/main/scala/org/apache/spark/graph/algorithms/TriangleCount.scala rename to graphx/src/main/scala/org/apache/spark/graphx/algorithms/TriangleCount.scala index 81774d52e4291..b5a93c1bd19bb 100644 --- a/graph/src/main/scala/org/apache/spark/graph/algorithms/TriangleCount.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/TriangleCount.scala @@ -1,8 +1,8 @@ -package org.apache.spark.graph.algorithms +package org.apache.spark.graphx.algorithms import scala.reflect.ClassTag -import org.apache.spark.graph._ +import org.apache.spark.graphx._ object TriangleCount { diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala similarity index 99% rename from graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala rename to graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala index b4311fa9f894e..4176563d22555 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala @@ -1,8 +1,8 @@ -package org.apache.spark.graph.impl +package org.apache.spark.graphx.impl import scala.reflect.ClassTag -import org.apache.spark.graph._ +import org.apache.spark.graphx._ import org.apache.spark.util.collection.PrimitiveKeyOpenHashMap /** diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartitionBuilder.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala similarity index 95% rename from graph/src/main/scala/org/apache/spark/graph/impl/EdgePartitionBuilder.scala rename to graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala index 56624ef60adc0..d4f08497a280a 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/EdgePartitionBuilder.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala @@ -1,9 +1,9 @@ -package org.apache.spark.graph.impl +package org.apache.spark.graphx.impl import scala.reflect.ClassTag import scala.util.Sorting -import org.apache.spark.graph._ +import org.apache.spark.graphx._ import org.apache.spark.util.collection.{PrimitiveKeyOpenHashMap, PrimitiveVector} diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/EdgeTripletIterator.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala similarity index 94% rename from graph/src/main/scala/org/apache/spark/graph/impl/EdgeTripletIterator.scala rename to graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala index e95d79e3d63d6..79fd962ffdc15 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/EdgeTripletIterator.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala @@ -1,8 +1,8 @@ -package org.apache.spark.graph.impl +package org.apache.spark.graphx.impl import scala.reflect.ClassTag -import org.apache.spark.graph._ +import org.apache.spark.graphx._ import org.apache.spark.util.collection.PrimitiveKeyOpenHashMap diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala similarity index 98% rename from graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala rename to graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala index 8f42e7d5929dd..be9f1881500f5 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/GraphImpl.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala @@ -1,14 +1,14 @@ -package org.apache.spark.graph.impl +package org.apache.spark.graphx.impl import scala.reflect.{classTag, ClassTag} import org.apache.spark.util.collection.PrimitiveVector import org.apache.spark.{HashPartitioner, Partitioner} import org.apache.spark.SparkContext._ -import org.apache.spark.graph._ -import org.apache.spark.graph.impl.GraphImpl._ -import org.apache.spark.graph.impl.MsgRDDFunctions._ -import org.apache.spark.graph.util.BytecodeUtils +import org.apache.spark.graphx._ +import org.apache.spark.graphx.impl.GraphImpl._ +import org.apache.spark.graphx.impl.MsgRDDFunctions._ +import org.apache.spark.graphx.util.BytecodeUtils import org.apache.spark.rdd.{ShuffledRDD, RDD} import org.apache.spark.storage.StorageLevel import org.apache.spark.util.ClosureCleaner diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/MessageToPartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala similarity index 96% rename from graph/src/main/scala/org/apache/spark/graph/impl/MessageToPartition.scala rename to graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala index b2fa7284827a4..ad5daf8f6ae32 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/MessageToPartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala @@ -1,9 +1,9 @@ -package org.apache.spark.graph.impl +package org.apache.spark.graphx.impl import scala.reflect.{classTag, ClassTag} import org.apache.spark.Partitioner -import org.apache.spark.graph.{PartitionID, VertexID} +import org.apache.spark.graphx.{PartitionID, VertexID} import org.apache.spark.rdd.{ShuffledRDD, RDD} diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/ReplicatedVertexView.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala similarity index 99% rename from graph/src/main/scala/org/apache/spark/graph/impl/ReplicatedVertexView.scala rename to graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala index 7d29861db1e3a..63180bc3af473 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/ReplicatedVertexView.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala @@ -1,4 +1,4 @@ -package org.apache.spark.graph.impl +package org.apache.spark.graphx.impl import scala.reflect.{classTag, ClassTag} @@ -6,7 +6,7 @@ import org.apache.spark.SparkContext._ import org.apache.spark.rdd.RDD import org.apache.spark.util.collection.{PrimitiveVector, OpenHashSet} -import org.apache.spark.graph._ +import org.apache.spark.graphx._ /** * A view of the vertices after they are shipped to the join sites specified in diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/RoutingTable.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTable.scala similarity index 97% rename from graph/src/main/scala/org/apache/spark/graph/impl/RoutingTable.scala rename to graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTable.scala index 96d9e9d7f8c8e..3bd8b24133244 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/RoutingTable.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTable.scala @@ -1,7 +1,7 @@ -package org.apache.spark.graph.impl +package org.apache.spark.graphx.impl import org.apache.spark.SparkContext._ -import org.apache.spark.graph._ +import org.apache.spark.graphx._ import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel import org.apache.spark.util.collection.PrimitiveVector diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala similarity index 99% rename from graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala rename to graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala index a3b0ea7689fc7..1c3c87f08dcb9 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/Serializers.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala @@ -1,10 +1,10 @@ -package org.apache.spark.graph.impl +package org.apache.spark.graphx.impl import java.io.{EOFException, InputStream, OutputStream} import java.nio.ByteBuffer import org.apache.spark.SparkConf -import org.apache.spark.graph._ +import org.apache.spark.graphx._ import org.apache.spark.serializer._ class VertexIDMsgSerializer(conf: SparkConf) extends Serializer { diff --git a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala similarity index 98% rename from graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala rename to graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala index 91244daa54a57..7c83497ca9564 100644 --- a/graph/src/main/scala/org/apache/spark/graph/impl/VertexPartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala @@ -1,14 +1,14 @@ -package org.apache.spark.graph.impl +package org.apache.spark.graphx.impl import scala.reflect.ClassTag import org.apache.spark.util.collection.{BitSet, PrimitiveKeyOpenHashMap} import org.apache.spark.Logging -import org.apache.spark.graph._ +import org.apache.spark.graphx._ -private[graph] object VertexPartition { +private[graphx] object VertexPartition { def apply[VD: ClassTag](iter: Iterator[(VertexID, VD)]): VertexPartition[VD] = { val map = new PrimitiveKeyOpenHashMap[VertexID, VD] @@ -30,7 +30,7 @@ private[graph] object VertexPartition { } -private[graph] +private[graphx] class VertexPartition[@specialized(Long, Int, Double) VD: ClassTag]( val index: VertexIdToIndexMap, val values: Array[VD], diff --git a/graph/src/main/scala/org/apache/spark/graph/package.scala b/graphx/src/main/scala/org/apache/spark/graphx/package.scala similarity index 94% rename from graph/src/main/scala/org/apache/spark/graph/package.scala rename to graphx/src/main/scala/org/apache/spark/graphx/package.scala index b98a11b918a21..96f0d91c9bcf9 100644 --- a/graph/src/main/scala/org/apache/spark/graph/package.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/package.scala @@ -3,7 +3,7 @@ package org.apache.spark import org.apache.spark.util.collection.OpenHashSet -package object graph { +package object graphx { type VertexID = Long diff --git a/graph/src/main/scala/org/apache/spark/graph/perf/BagelTest.scala b/graphx/src/main/scala/org/apache/spark/graphx/perf/BagelTest.scala similarity index 97% rename from graph/src/main/scala/org/apache/spark/graph/perf/BagelTest.scala rename to graphx/src/main/scala/org/apache/spark/graphx/perf/BagelTest.scala index eaff27a33eda5..81332e0800b94 100644 --- a/graph/src/main/scala/org/apache/spark/graph/perf/BagelTest.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/perf/BagelTest.scala @@ -1,6 +1,6 @@ ///// This file creates circular dependencies between examples bagle and graph -// package org.apache.spark.graph.perf +// package org.apache.spark.graphx.perf // import org.apache.spark._ // import org.apache.spark.SparkContext._ @@ -8,7 +8,7 @@ // import org.apache.spark.examples.bagel // //import org.apache.spark.bagel.examples._ -// import org.apache.spark.graph._ +// import org.apache.spark.graphx._ // object BagelTest { diff --git a/graph/src/main/scala/org/apache/spark/graph/perf/SparkTest.scala b/graphx/src/main/scala/org/apache/spark/graphx/perf/SparkTest.scala similarity index 97% rename from graph/src/main/scala/org/apache/spark/graph/perf/SparkTest.scala rename to graphx/src/main/scala/org/apache/spark/graphx/perf/SparkTest.scala index 01bd96855082c..24262640ab0da 100644 --- a/graph/src/main/scala/org/apache/spark/graph/perf/SparkTest.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/perf/SparkTest.scala @@ -1,13 +1,13 @@ ///// This file creates circular dependencies between examples bagle and graph -// package org.apache.spark.graph.perf +// package org.apache.spark.graphx.perf // import org.apache.spark._ // import org.apache.spark.SparkContext._ // import org.apache.spark.bagel.Bagel // import org.apache.spark.bagel.examples._ -// import org.apache.spark.graph._ +// import org.apache.spark.graphx._ // object SparkTest { diff --git a/graph/src/main/scala/org/apache/spark/graph/util/BytecodeUtils.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala similarity index 99% rename from graph/src/main/scala/org/apache/spark/graph/util/BytecodeUtils.scala rename to graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala index bc00ce215153d..ec8d534333de5 100644 --- a/graph/src/main/scala/org/apache/spark/graph/util/BytecodeUtils.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala @@ -1,4 +1,4 @@ -package org.apache.spark.graph.util +package org.apache.spark.graphx.util import java.io.{ByteArrayInputStream, ByteArrayOutputStream} diff --git a/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala similarity index 96% rename from graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala rename to graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala index 51f45cb8922b3..57117241ad818 100644 --- a/graph/src/main/scala/org/apache/spark/graph/util/GraphGenerators.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala @@ -1,4 +1,4 @@ -package org.apache.spark.graph.util +package org.apache.spark.graphx.util import scala.annotation.tailrec import scala.math._ @@ -10,10 +10,10 @@ import org.apache.spark.serializer._ import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext import org.apache.spark.SparkContext._ -import org.apache.spark.graph._ -import org.apache.spark.graph.Graph -import org.apache.spark.graph.Edge -import org.apache.spark.graph.impl.GraphImpl +import org.apache.spark.graphx._ +import org.apache.spark.graphx.Graph +import org.apache.spark.graphx.Edge +import org.apache.spark.graphx.impl.GraphImpl /** * @todo cleanup and modularize code @@ -31,7 +31,7 @@ object GraphGenerators { val serializer = "org.apache.spark.serializer.KryoSerializer" System.setProperty("spark.serializer", serializer) //System.setProperty("spark.shuffle.compress", "false") - System.setProperty("spark.kryo.registrator", "spark.graph.GraphKryoRegistrator") + System.setProperty("spark.kryo.registrator", "spark.graphx.GraphKryoRegistrator") val host = "local[4]" val sc = new SparkContext(host, "Lognormal graph generator") diff --git a/graph/src/main/scala/org/apache/spark/graph/util/HashUtils.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/HashUtils.scala similarity index 91% rename from graph/src/main/scala/org/apache/spark/graph/util/HashUtils.scala rename to graphx/src/main/scala/org/apache/spark/graphx/util/HashUtils.scala index cb18ef3d26bae..7a79d33350b1c 100644 --- a/graph/src/main/scala/org/apache/spark/graph/util/HashUtils.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/util/HashUtils.scala @@ -1,4 +1,4 @@ -package org.apache.spark.graph.util +package org.apache.spark.graphx.util object HashUtils { diff --git a/graph/src/test/resources/log4j.properties b/graphx/src/test/resources/log4j.properties similarity index 100% rename from graph/src/test/resources/log4j.properties rename to graphx/src/test/resources/log4j.properties diff --git a/graph/src/test/scala/org/apache/spark/graph/GraphOpsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala similarity index 96% rename from graph/src/test/scala/org/apache/spark/graph/GraphOpsSuite.scala rename to graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala index 132e6be24a04a..cc281fce997bf 100644 --- a/graph/src/test/scala/org/apache/spark/graph/GraphOpsSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala @@ -1,8 +1,8 @@ -package org.apache.spark.graph +package org.apache.spark.graphx import org.apache.spark.SparkContext -import org.apache.spark.graph.Graph._ -import org.apache.spark.graph.impl.EdgePartition +import org.apache.spark.graphx.Graph._ +import org.apache.spark.graphx.impl.EdgePartition import org.apache.spark.rdd._ import org.scalatest.FunSuite diff --git a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala similarity index 99% rename from graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala rename to graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala index 41f3a8311d5a8..094fa722a0030 100644 --- a/graph/src/test/scala/org/apache/spark/graph/GraphSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala @@ -1,9 +1,9 @@ -package org.apache.spark.graph +package org.apache.spark.graphx import org.scalatest.FunSuite import org.apache.spark.SparkContext -import org.apache.spark.graph.Graph._ +import org.apache.spark.graphx.Graph._ import org.apache.spark.rdd._ class GraphSuite extends FunSuite with LocalSparkContext { diff --git a/graph/src/test/scala/org/apache/spark/graph/LocalSparkContext.scala b/graphx/src/test/scala/org/apache/spark/graphx/LocalSparkContext.scala similarity index 92% rename from graph/src/test/scala/org/apache/spark/graph/LocalSparkContext.scala rename to graphx/src/test/scala/org/apache/spark/graphx/LocalSparkContext.scala index 5c20d559aa973..6aec2ea8a994f 100644 --- a/graph/src/test/scala/org/apache/spark/graph/LocalSparkContext.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/LocalSparkContext.scala @@ -1,4 +1,4 @@ -package org.apache.spark.graph +package org.apache.spark.graphx import org.scalatest.Suite import org.scalatest.BeforeAndAfterEach @@ -12,7 +12,7 @@ import org.apache.spark.SparkContext */ trait LocalSparkContext { System.setProperty("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - System.setProperty("spark.kryo.registrator", "org.apache.spark.graph.GraphKryoRegistrator") + System.setProperty("spark.kryo.registrator", "org.apache.spark.graphx.GraphKryoRegistrator") /** Runs `f` on a new SparkContext and ensures that it is stopped afterwards. */ def withSpark[T](f: SparkContext => T) = { diff --git a/graph/src/test/scala/org/apache/spark/graph/PregelSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/PregelSuite.scala similarity index 97% rename from graph/src/test/scala/org/apache/spark/graph/PregelSuite.scala rename to graphx/src/test/scala/org/apache/spark/graphx/PregelSuite.scala index de7e3872cad6f..429622357f0bd 100644 --- a/graph/src/test/scala/org/apache/spark/graph/PregelSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/PregelSuite.scala @@ -1,4 +1,4 @@ -package org.apache.spark.graph +package org.apache.spark.graphx import org.scalatest.FunSuite diff --git a/graph/src/test/scala/org/apache/spark/graph/SerializerSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/SerializerSuite.scala similarity index 98% rename from graph/src/test/scala/org/apache/spark/graph/SerializerSuite.scala rename to graphx/src/test/scala/org/apache/spark/graphx/SerializerSuite.scala index 2864ffd1ca31e..3ba412c1f84f4 100644 --- a/graph/src/test/scala/org/apache/spark/graph/SerializerSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/SerializerSuite.scala @@ -1,4 +1,4 @@ -package org.apache.spark.graph +package org.apache.spark.graphx import java.io.{EOFException, ByteArrayInputStream, ByteArrayOutputStream} @@ -7,8 +7,8 @@ import scala.util.Random import org.scalatest.FunSuite import org.apache.spark._ -import org.apache.spark.graph.impl._ -import org.apache.spark.graph.impl.MsgRDDFunctions._ +import org.apache.spark.graphx.impl._ +import org.apache.spark.graphx.impl.MsgRDDFunctions._ import org.apache.spark.serializer.SerializationStream diff --git a/graph/src/test/scala/org/apache/spark/graph/VertexRDDSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala similarity index 95% rename from graph/src/test/scala/org/apache/spark/graph/VertexRDDSuite.scala rename to graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala index e876b8e4e8bfe..573b708e89dd9 100644 --- a/graph/src/test/scala/org/apache/spark/graph/VertexRDDSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala @@ -1,8 +1,8 @@ -package org.apache.spark.graph +package org.apache.spark.graphx import org.apache.spark.SparkContext -import org.apache.spark.graph.Graph._ -import org.apache.spark.graph.impl.EdgePartition +import org.apache.spark.graphx.Graph._ +import org.apache.spark.graphx.impl.EdgePartition import org.apache.spark.rdd._ import org.scalatest.FunSuite diff --git a/graph/src/test/scala/org/apache/spark/graph/algorithms/ConnectedComponentsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/ConnectedComponentsSuite.scala similarity index 95% rename from graph/src/test/scala/org/apache/spark/graph/algorithms/ConnectedComponentsSuite.scala rename to graphx/src/test/scala/org/apache/spark/graphx/algorithms/ConnectedComponentsSuite.scala index 81a1b7337f2a4..5e2ecfcde9b0c 100644 --- a/graph/src/test/scala/org/apache/spark/graph/algorithms/ConnectedComponentsSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/ConnectedComponentsSuite.scala @@ -1,11 +1,11 @@ -package org.apache.spark.graph.algorithms +package org.apache.spark.graphx.algorithms import org.scalatest.FunSuite import org.apache.spark.SparkContext import org.apache.spark.SparkContext._ -import org.apache.spark.graph._ -import org.apache.spark.graph.util.GraphGenerators +import org.apache.spark.graphx._ +import org.apache.spark.graphx.util.GraphGenerators import org.apache.spark.rdd._ diff --git a/graph/src/test/scala/org/apache/spark/graph/algorithms/PageRankSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/PageRankSuite.scala similarity index 97% rename from graph/src/test/scala/org/apache/spark/graph/algorithms/PageRankSuite.scala rename to graphx/src/test/scala/org/apache/spark/graphx/algorithms/PageRankSuite.scala index 81d82a5a6b719..e365b1e2305b4 100644 --- a/graph/src/test/scala/org/apache/spark/graph/algorithms/PageRankSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/PageRankSuite.scala @@ -1,13 +1,13 @@ -package org.apache.spark.graph.algorithms +package org.apache.spark.graphx.algorithms import org.scalatest.FunSuite -import org.apache.spark.graph._ +import org.apache.spark.graphx._ import org.apache.spark.SparkContext import org.apache.spark.SparkContext._ import org.apache.spark.rdd._ -import org.apache.spark.graph.util.GraphGenerators +import org.apache.spark.graphx.util.GraphGenerators object GridPageRank { diff --git a/graph/src/test/scala/org/apache/spark/graph/algorithms/SVDPlusPlusSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/SVDPlusPlusSuite.scala similarity index 87% rename from graph/src/test/scala/org/apache/spark/graph/algorithms/SVDPlusPlusSuite.scala rename to graphx/src/test/scala/org/apache/spark/graphx/algorithms/SVDPlusPlusSuite.scala index a0a6eb33e36fc..06604198d73a0 100644 --- a/graph/src/test/scala/org/apache/spark/graph/algorithms/SVDPlusPlusSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/SVDPlusPlusSuite.scala @@ -1,11 +1,11 @@ -package org.apache.spark.graph.algorithms +package org.apache.spark.graphx.algorithms import org.scalatest.FunSuite import org.apache.spark.SparkContext import org.apache.spark.SparkContext._ -import org.apache.spark.graph._ -import org.apache.spark.graph.util.GraphGenerators +import org.apache.spark.graphx._ +import org.apache.spark.graphx.util.GraphGenerators import org.apache.spark.rdd._ diff --git a/graph/src/test/scala/org/apache/spark/graph/algorithms/StronglyConnectedComponentsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/StronglyConnectedComponentsSuite.scala similarity index 92% rename from graph/src/test/scala/org/apache/spark/graph/algorithms/StronglyConnectedComponentsSuite.scala rename to graphx/src/test/scala/org/apache/spark/graphx/algorithms/StronglyConnectedComponentsSuite.scala index 4afb158a68ed4..696b80944ed02 100644 --- a/graph/src/test/scala/org/apache/spark/graph/algorithms/StronglyConnectedComponentsSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/StronglyConnectedComponentsSuite.scala @@ -1,11 +1,11 @@ -package org.apache.spark.graph.algorithms +package org.apache.spark.graphx.algorithms import org.scalatest.FunSuite import org.apache.spark.SparkContext import org.apache.spark.SparkContext._ -import org.apache.spark.graph._ -import org.apache.spark.graph.util.GraphGenerators +import org.apache.spark.graphx._ +import org.apache.spark.graphx.util.GraphGenerators import org.apache.spark.rdd._ diff --git a/graph/src/test/scala/org/apache/spark/graph/algorithms/TriangleCountSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/TriangleCountSuite.scala similarity index 94% rename from graph/src/test/scala/org/apache/spark/graph/algorithms/TriangleCountSuite.scala rename to graphx/src/test/scala/org/apache/spark/graphx/algorithms/TriangleCountSuite.scala index 274ab11f0cbd1..0e59912754c05 100644 --- a/graph/src/test/scala/org/apache/spark/graph/algorithms/TriangleCountSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/TriangleCountSuite.scala @@ -1,11 +1,11 @@ -package org.apache.spark.graph.algorithms +package org.apache.spark.graphx.algorithms import org.scalatest.FunSuite import org.apache.spark.SparkContext import org.apache.spark.SparkContext._ -import org.apache.spark.graph._ -import org.apache.spark.graph.util.GraphGenerators +import org.apache.spark.graphx._ +import org.apache.spark.graphx.util.GraphGenerators import org.apache.spark.rdd._ diff --git a/graph/src/test/scala/org/apache/spark/graph/impl/EdgePartitionSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala similarity index 97% rename from graph/src/test/scala/org/apache/spark/graph/impl/EdgePartitionSuite.scala rename to graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala index fd0beee2f6408..eb82436f0964c 100644 --- a/graph/src/test/scala/org/apache/spark/graph/impl/EdgePartitionSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala @@ -1,11 +1,11 @@ -package org.apache.spark.graph.impl +package org.apache.spark.graphx.impl import scala.reflect.ClassTag import scala.util.Random import org.scalatest.FunSuite -import org.apache.spark.graph._ +import org.apache.spark.graphx._ class EdgePartitionSuite extends FunSuite { diff --git a/graph/src/test/scala/org/apache/spark/graph/impl/VertexPartitionSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/impl/VertexPartitionSuite.scala similarity index 98% rename from graph/src/test/scala/org/apache/spark/graph/impl/VertexPartitionSuite.scala rename to graphx/src/test/scala/org/apache/spark/graphx/impl/VertexPartitionSuite.scala index 72579a48c2885..d37d64e8c849e 100644 --- a/graph/src/test/scala/org/apache/spark/graph/impl/VertexPartitionSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/impl/VertexPartitionSuite.scala @@ -1,6 +1,6 @@ -package org.apache.spark.graph.impl +package org.apache.spark.graphx.impl -import org.apache.spark.graph._ +import org.apache.spark.graphx._ import org.scalatest.FunSuite class VertexPartitionSuite extends FunSuite { diff --git a/graph/src/test/scala/org/apache/spark/graph/util/BytecodeUtilsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/util/BytecodeUtilsSuite.scala similarity index 98% rename from graph/src/test/scala/org/apache/spark/graph/util/BytecodeUtilsSuite.scala rename to graphx/src/test/scala/org/apache/spark/graphx/util/BytecodeUtilsSuite.scala index d85e877ddfaf1..11db339750920 100644 --- a/graph/src/test/scala/org/apache/spark/graph/util/BytecodeUtilsSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/util/BytecodeUtilsSuite.scala @@ -1,4 +1,4 @@ -package org.apache.spark.graph.util +package org.apache.spark.graphx.util import org.scalatest.FunSuite diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index c2cd6fb45abda..c2b1c0c35cc5b 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -48,20 +48,20 @@ object SparkBuild extends Build { lazy val core = Project("core", file("core"), settings = coreSettings) lazy val repl = Project("repl", file("repl"), settings = replSettings) - .dependsOn(core, graph, bagel, mllib) + .dependsOn(core, graphx, bagel, mllib) lazy val tools = Project("tools", file("tools"), settings = toolsSettings) dependsOn(core) dependsOn(streaming) lazy val bagel = Project("bagel", file("bagel"), settings = bagelSettings) dependsOn(core) - lazy val graph = Project("graph", file("graph"), settings = graphSettings) dependsOn(core) + lazy val graphx = Project("graphx", file("graphx"), settings = graphxSettings) dependsOn(core) lazy val streaming = Project("streaming", file("streaming"), settings = streamingSettings) dependsOn(core) lazy val mllib = Project("mllib", file("mllib"), settings = mllibSettings) dependsOn(core) lazy val assemblyProj = Project("assembly", file("assembly"), settings = assemblyProjSettings) - .dependsOn(core, graph, bagel, mllib, repl, streaming) dependsOn(maybeYarn: _*) + .dependsOn(core, graphx, bagel, mllib, repl, streaming) dependsOn(maybeYarn: _*) lazy val assembleDeps = TaskKey[Unit]("assemble-deps", "Build assembly of dependencies and packages Spark projects") @@ -111,10 +111,10 @@ object SparkBuild extends Build { lazy val allExternalRefs = Seq[ProjectReference](externalTwitter, externalKafka, externalFlume, externalZeromq, externalMqtt) lazy val examples = Project("examples", file("examples"), settings = examplesSettings) - .dependsOn(core, mllib, graph, bagel, streaming, externalTwitter) dependsOn(allExternal: _*) + .dependsOn(core, mllib, graphx, bagel, streaming, externalTwitter) dependsOn(allExternal: _*) // Everything except assembly, tools and examples belong to packageProjects - lazy val packageProjects = Seq[ProjectReference](core, repl, bagel, streaming, mllib, graph) ++ maybeYarnRef + lazy val packageProjects = Seq[ProjectReference](core, repl, bagel, streaming, mllib, graphx) ++ maybeYarnRef lazy val allProjects = packageProjects ++ allExternalRefs ++ Seq[ProjectReference](examples, tools, assemblyProj) @@ -308,7 +308,7 @@ object SparkBuild extends Build { name := "spark-tools" ) ++ assemblySettings ++ extraAssemblySettings - def graphSettings = sharedSettings ++ Seq( + def graphxSettings = sharedSettings ++ Seq( name := "spark-graphx" ) From b7c92dded33e61976dea10beef88ab52e2009b42 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Thu, 9 Jan 2014 20:44:28 -0800 Subject: [PATCH 363/531] Add implicit algorithm methods for Graph; remove standalone PageRank --- .../org/apache/spark/graphx/Analytics.scala | 2 +- .../spark/graphx/algorithms/Algorithms.scala | 56 +++++++++++++++++++ .../algorithms/ConnectedComponents.scala | 6 +- .../spark/graphx/algorithms/PageRank.scala | 55 ++---------------- .../StronglyConnectedComponents.scala | 8 ++- .../spark/graphx/algorithms/package.scala | 8 +++ .../algorithms/ConnectedComponentsSuite.scala | 8 +-- .../graphx/algorithms/PageRankSuite.scala | 27 ++++----- .../StronglyConnectedComponentsSuite.scala | 6 +- .../algorithms/TriangleCountSuite.scala | 8 +-- 10 files changed, 99 insertions(+), 85 deletions(-) create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/algorithms/Algorithms.scala create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/algorithms/package.scala diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Analytics.scala b/graphx/src/main/scala/org/apache/spark/graphx/Analytics.scala index 0cafc3fdf94ce..def6d69190960 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Analytics.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Analytics.scala @@ -83,7 +83,7 @@ object Analytics extends Logging { println("GRAPHX: Number of edges " + graph.edges.count) //val pr = Analytics.pagerank(graph, numIter) - val pr = PageRank.runStandalone(graph, tol) + val pr = graph.pageRank(tol).vertices println("GRAPHX: Total rank: " + pr.map(_._2).reduce(_+_)) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/Algorithms.scala b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/Algorithms.scala new file mode 100644 index 0000000000000..4af7af545c256 --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/Algorithms.scala @@ -0,0 +1,56 @@ +package org.apache.spark.graphx.algorithms + +import scala.reflect.ClassTag + +import org.apache.spark.graphx._ + +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.algorithms.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.algorithms.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.algorithms.ConnectedComponents]] + */ + def connectedComponents(): Graph[VertexID, ED] = { + ConnectedComponents.run(self) + } + + /** + * Compute the number of triangles passing through each vertex. + * + * @see [[org.apache.spark.graphx.algorithms.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.algorithms.StronglyConnectedComponents]] + */ + def stronglyConnectedComponents(numIter: Int): Graph[VertexID, ED] = { + StronglyConnectedComponents.run(self, numIter) + } +} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/ConnectedComponents.scala b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/ConnectedComponents.scala index a0dd36da60f32..137a81f4d5384 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/ConnectedComponents.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/ConnectedComponents.scala @@ -1,11 +1,13 @@ package org.apache.spark.graphx.algorithms +import scala.reflect.ClassTag + import org.apache.spark.graphx._ object ConnectedComponents { /** - * Compute the connected component membership of each vertex and return an RDD with the vertex + * 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. * * @tparam VD the vertex attribute type (discarded in the computation) @@ -16,7 +18,7 @@ object ConnectedComponents { * @return a graph with vertex attributes containing the smallest vertex in each * connected component */ - def run[VD: Manifest, ED: Manifest](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]) = { diff --git a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/PageRank.scala b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/PageRank.scala index 0292b7316d392..b423104eda358 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/PageRank.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/PageRank.scala @@ -1,5 +1,7 @@ package org.apache.spark.graphx.algorithms +import scala.reflect.ClassTag + import org.apache.spark.Logging import org.apache.spark.graphx._ @@ -42,7 +44,7 @@ object PageRank extends Logging { * containing the normalized weight. * */ - def run[VD: Manifest, ED: Manifest]( + def run[VD: ClassTag, ED: ClassTag]( graph: Graph[VD, ED], numIter: Int, resetProb: Double = 0.15): Graph[Double, Double] = { @@ -109,7 +111,7 @@ object PageRank extends Logging { * @return the graph containing with each vertex containing the PageRank and each edge * containing the normalized weight. */ - def runUntillConvergence[VD: Manifest, ED: Manifest]( + def runUntilConvergence[VD: ClassTag, ED: ClassTag]( graph: Graph[VD, ED], tol: Double, resetProb: Double = 0.15): Graph[Double, Double] = { // Initialize the pagerankGraph with each edge attribute @@ -153,53 +155,4 @@ object PageRank extends Logging { .mapVertices((vid, attr) => attr._1) } // end of deltaPageRank - def runStandalone[VD: Manifest, ED: Manifest]( - graph: Graph[VD, ED], tol: Double, resetProb: Double = 0.15): VertexRDD[Double] = { - - // Initialize the ranks - var ranks: VertexRDD[Double] = graph.vertices.mapValues((vid, attr) => resetProb).cache() - - // Initialize the delta graph where each vertex stores its delta and each edge knows its weight - var deltaGraph: Graph[Double, Double] = - graph.outerJoinVertices(graph.outDegrees)((vid, vdata, deg) => deg.getOrElse(0)) - .mapTriplets(e => 1.0 / e.srcAttr) - .mapVertices((vid, degree) => resetProb).cache() - var numDeltas: Long = ranks.count() - - var prevDeltas: Option[VertexRDD[Double]] = None - - var i = 0 - val weight = (1.0 - resetProb) - while (numDeltas > 0) { - // Compute new deltas. Only deltas that existed in the last round (i.e., were greater than - // `tol`) get to send messages; those that were less than `tol` would send messages less than - // `tol` as well. - val deltas = deltaGraph - .mapReduceTriplets[Double]( - et => Iterator((et.dstId, et.srcAttr * et.attr * weight)), - _ + _, - prevDeltas.map((_, EdgeDirection.Out))) - .filter { case (vid, delta) => delta > tol } - .cache() - prevDeltas = Some(deltas) - numDeltas = deltas.count() - logInfo("Standalone PageRank: iter %d has %d deltas".format(i, numDeltas)) - - // Update deltaGraph with the deltas - deltaGraph = deltaGraph.outerJoinVertices(deltas) { (vid, old, newOpt) => - newOpt.getOrElse(old) - }.cache() - - // Update ranks - ranks = ranks.leftZipJoin(deltas) { (vid, oldRank, deltaOpt) => - oldRank + deltaOpt.getOrElse(0.0) - } - ranks.foreach(x => {}) // force the iteration for ease of debugging - - i += 1 - } - - ranks - } - } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/StronglyConnectedComponents.scala b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/StronglyConnectedComponents.scala index f64fc3ef0f52a..49ec91aeddb32 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/StronglyConnectedComponents.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/StronglyConnectedComponents.scala @@ -1,12 +1,14 @@ package org.apache.spark.graphx.algorithms +import scala.reflect.ClassTag + import org.apache.spark.graphx._ object StronglyConnectedComponents { /** - * Compute the strongly connected component (SCC) of each vertex and return an RDD with the vertex - * value containing the lowest vertex id in the SCC containing that vertex. + * 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. * * @tparam VD the vertex attribute type (discarded in the computation) * @tparam ED the edge attribute type (preserved in the computation) @@ -15,7 +17,7 @@ object StronglyConnectedComponents { * * @return a graph with vertex attributes containing the smallest vertex id in each SCC */ - def run[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], numIter: Int): Graph[VertexID, ED] = { + def run[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED], numIter: Int): Graph[VertexID, ED] = { // the graph we update with final SCC ids, and the graph we return at the end var sccGraph = graph.mapVertices { case (vid, _) => vid } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/package.scala b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/package.scala new file mode 100644 index 0000000000000..fbabf1257cc99 --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/package.scala @@ -0,0 +1,8 @@ +package org.apache.spark.graphx + +import scala.reflect.ClassTag + +package object algorithms { + implicit def graphToAlgorithms[VD: ClassTag, ED: ClassTag]( + graph: Graph[VD, ED]): Algorithms[VD, ED] = new Algorithms(graph) +} diff --git a/graphx/src/test/scala/org/apache/spark/graphx/algorithms/ConnectedComponentsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/ConnectedComponentsSuite.scala index 5e2ecfcde9b0c..209191ef07867 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/algorithms/ConnectedComponentsSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/ConnectedComponentsSuite.scala @@ -14,7 +14,7 @@ class ConnectedComponentsSuite extends FunSuite with LocalSparkContext { test("Grid Connected Components") { withSpark { sc => val gridGraph = GraphGenerators.gridGraph(sc, 10, 10).cache() - val ccGraph = ConnectedComponents.run(gridGraph).cache() + val ccGraph = gridGraph.connectedComponents().cache() val maxCCid = ccGraph.vertices.map { case (vid, ccId) => ccId }.sum assert(maxCCid === 0) } @@ -24,7 +24,7 @@ class ConnectedComponentsSuite extends FunSuite with LocalSparkContext { test("Reverse Grid Connected Components") { withSpark { sc => val gridGraph = GraphGenerators.gridGraph(sc, 10, 10).reverse.cache() - val ccGraph = ConnectedComponents.run(gridGraph).cache() + val ccGraph = gridGraph.connectedComponents().cache() val maxCCid = ccGraph.vertices.map { case (vid, ccId) => ccId }.sum assert(maxCCid === 0) } @@ -37,7 +37,7 @@ class ConnectedComponentsSuite extends FunSuite with LocalSparkContext { val chain2 = (10 until 20).map(x => (x, x+1) ) val rawEdges = sc.parallelize(chain1 ++ chain2, 3).map { case (s,d) => (s.toLong, d.toLong) } val twoChains = Graph.fromEdgeTuples(rawEdges, 1.0).cache() - val ccGraph = ConnectedComponents.run(twoChains).cache() + val ccGraph = twoChains.connectedComponents().cache() val vertices = ccGraph.vertices.collect() for ( (id, cc) <- vertices ) { if(id < 10) { assert(cc === 0) } @@ -60,7 +60,7 @@ class ConnectedComponentsSuite extends FunSuite with LocalSparkContext { val chain2 = (10 until 20).map(x => (x, x+1) ) val rawEdges = sc.parallelize(chain1 ++ chain2, 3).map { case (s,d) => (s.toLong, d.toLong) } val twoChains = Graph.fromEdgeTuples(rawEdges, true).reverse.cache() - val ccGraph = ConnectedComponents.run(twoChains).cache() + val ccGraph = twoChains.connectedComponents().cache() val vertices = ccGraph.vertices.collect for ( (id, cc) <- vertices ) { if (id < 10) { diff --git a/graphx/src/test/scala/org/apache/spark/graphx/algorithms/PageRankSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/PageRankSuite.scala index e365b1e2305b4..cd857bd3a18b2 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/algorithms/PageRankSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/PageRankSuite.scala @@ -2,13 +2,12 @@ package org.apache.spark.graphx.algorithms import org.scalatest.FunSuite -import org.apache.spark.graphx._ import org.apache.spark.SparkContext import org.apache.spark.SparkContext._ -import org.apache.spark.rdd._ - +import org.apache.spark.graphx._ +import org.apache.spark.graphx.algorithms._ import org.apache.spark.graphx.util.GraphGenerators - +import org.apache.spark.rdd._ object GridPageRank { def apply(nRows: Int, nCols: Int, nIter: Int, resetProb: Double) = { @@ -58,8 +57,8 @@ class PageRankSuite extends FunSuite with LocalSparkContext { val resetProb = 0.15 val errorTol = 1.0e-5 - val staticRanks1 = PageRank.run(starGraph, numIter = 1, resetProb).vertices.cache() - val staticRanks2 = PageRank.run(starGraph, numIter = 2, resetProb).vertices.cache() + val staticRanks1 = starGraph.staticPageRank(numIter = 1, resetProb).vertices.cache() + val staticRanks2 = starGraph.staticPageRank(numIter = 2, resetProb).vertices.cache() // Static PageRank should only take 2 iterations to converge val notMatching = staticRanks1.innerZipJoin(staticRanks2) { (vid, pr1, pr2) => @@ -74,10 +73,8 @@ class PageRankSuite extends FunSuite with LocalSparkContext { } assert(staticErrors.sum === 0) - val dynamicRanks = PageRank.runUntillConvergence(starGraph, 0, resetProb).vertices.cache() - val standaloneRanks = PageRank.runStandalone(starGraph, 0, resetProb).cache() + val dynamicRanks = starGraph.pageRank(0, resetProb).vertices.cache() assert(compareRanks(staticRanks2, dynamicRanks) < errorTol) - assert(compareRanks(staticRanks2, standaloneRanks) < errorTol) } } // end of test Star PageRank @@ -93,14 +90,12 @@ class PageRankSuite extends FunSuite with LocalSparkContext { val errorTol = 1.0e-5 val gridGraph = GraphGenerators.gridGraph(sc, rows, cols).cache() - val staticRanks = PageRank.run(gridGraph, numIter, resetProb).vertices.cache() - val dynamicRanks = PageRank.runUntillConvergence(gridGraph, tol, resetProb).vertices.cache() - val standaloneRanks = PageRank.runStandalone(gridGraph, tol, resetProb).cache() + val staticRanks = gridGraph.staticPageRank(numIter, resetProb).vertices.cache() + val dynamicRanks = gridGraph.pageRank(tol, resetProb).vertices.cache() val referenceRanks = VertexRDD(sc.parallelize(GridPageRank(rows, cols, numIter, resetProb))) assert(compareRanks(staticRanks, referenceRanks) < errorTol) assert(compareRanks(dynamicRanks, referenceRanks) < errorTol) - assert(compareRanks(standaloneRanks, referenceRanks) < errorTol) } } // end of Grid PageRank @@ -115,12 +110,10 @@ class PageRankSuite extends FunSuite with LocalSparkContext { val numIter = 10 val errorTol = 1.0e-5 - val staticRanks = PageRank.run(chain, numIter, resetProb).vertices.cache() - val dynamicRanks = PageRank.runUntillConvergence(chain, tol, resetProb).vertices.cache() - val standaloneRanks = PageRank.runStandalone(chain, tol, resetProb).cache() + val staticRanks = chain.staticPageRank(numIter, resetProb).vertices.cache() + val dynamicRanks = chain.pageRank(tol, resetProb).vertices.cache() assert(compareRanks(staticRanks, dynamicRanks) < errorTol) - assert(compareRanks(dynamicRanks, standaloneRanks) < errorTol) } } } diff --git a/graphx/src/test/scala/org/apache/spark/graphx/algorithms/StronglyConnectedComponentsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/StronglyConnectedComponentsSuite.scala index 696b80944ed02..fee7d20161cf3 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/algorithms/StronglyConnectedComponentsSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/StronglyConnectedComponentsSuite.scala @@ -16,7 +16,7 @@ class StronglyConnectedComponentsSuite extends FunSuite with LocalSparkContext { val vertices = sc.parallelize((1L to 5L).map(x => (x, -1))) val edges = sc.parallelize(Seq.empty[Edge[Int]]) val graph = Graph(vertices, edges) - val sccGraph = StronglyConnectedComponents.run(graph, 5) + val sccGraph = graph.stronglyConnectedComponents(5) for ((id, scc) <- sccGraph.vertices.collect) { assert(id == scc) } @@ -27,7 +27,7 @@ class StronglyConnectedComponentsSuite extends FunSuite with LocalSparkContext { withSpark { sc => val rawEdges = sc.parallelize((0L to 6L).map(x => (x, (x + 1) % 7))) val graph = Graph.fromEdgeTuples(rawEdges, -1) - val sccGraph = StronglyConnectedComponents.run(graph, 20) + val sccGraph = graph.stronglyConnectedComponents(20) for ((id, scc) <- sccGraph.vertices.collect) { assert(0L == scc) } @@ -42,7 +42,7 @@ class StronglyConnectedComponentsSuite extends FunSuite with LocalSparkContext { Array(6L -> 0L, 5L -> 7L) val rawEdges = sc.parallelize(edges) val graph = Graph.fromEdgeTuples(rawEdges, -1) - val sccGraph = StronglyConnectedComponents.run(graph, 20) + val sccGraph = graph.stronglyConnectedComponents(20) for ((id, scc) <- sccGraph.vertices.collect) { if (id < 3) assert(0L == scc) diff --git a/graphx/src/test/scala/org/apache/spark/graphx/algorithms/TriangleCountSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/TriangleCountSuite.scala index 0e59912754c05..b85b289da638b 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/algorithms/TriangleCountSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/TriangleCountSuite.scala @@ -15,7 +15,7 @@ class TriangleCountSuite extends FunSuite with LocalSparkContext { withSpark { sc => val rawEdges = sc.parallelize(Array( 0L->1L, 1L->2L, 2L->0L ), 2) val graph = Graph.fromEdgeTuples(rawEdges, true).cache() - val triangleCount = TriangleCount.run(graph) + val triangleCount = graph.triangleCount() val verts = triangleCount.vertices verts.collect.foreach { case (vid, count) => assert(count === 1) } } @@ -27,7 +27,7 @@ class TriangleCountSuite extends FunSuite with LocalSparkContext { Array(0L -> -1L, -1L -> -2L, -2L -> 0L) val rawEdges = sc.parallelize(triangles, 2) val graph = Graph.fromEdgeTuples(rawEdges, true).cache() - val triangleCount = TriangleCount.run(graph) + val triangleCount = graph.triangleCount() val verts = triangleCount.vertices verts.collect().foreach { case (vid, count) => if (vid == 0) { @@ -47,7 +47,7 @@ class TriangleCountSuite extends FunSuite with LocalSparkContext { val revTriangles = triangles.map { case (a,b) => (b,a) } val rawEdges = sc.parallelize(triangles ++ revTriangles, 2) val graph = Graph.fromEdgeTuples(rawEdges, true).cache() - val triangleCount = TriangleCount.run(graph) + val triangleCount = graph.triangleCount() val verts = triangleCount.vertices verts.collect().foreach { case (vid, count) => if (vid == 0) { @@ -64,7 +64,7 @@ class TriangleCountSuite extends FunSuite with LocalSparkContext { val rawEdges = sc.parallelize(Array(0L -> 1L, 1L -> 2L, 2L -> 0L) ++ Array(0L -> 1L, 1L -> 2L, 2L -> 0L), 2) val graph = Graph.fromEdgeTuples(rawEdges, true, uniqueEdges = Some(RandomVertexCut)).cache() - val triangleCount = TriangleCount.run(graph) + val triangleCount = graph.triangleCount() val verts = triangleCount.vertices verts.collect.foreach { case (vid, count) => assert(count === 1) } } From 210f2dd84fb2de623745a162377b989712f7ef0f Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Thu, 9 Jan 2014 21:47:40 -0800 Subject: [PATCH 364/531] graph -> graphx in bin/compute-classpath.sh --- bin/compute-classpath.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index c10725e708661..e01cfa1eb88ae 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -40,7 +40,7 @@ if [ -f "$FWDIR"/assembly/target/scala-$SCALA_VERSION/spark-assembly*hadoop*-dep CLASSPATH="$CLASSPATH:$FWDIR/repl/target/scala-$SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$FWDIR/mllib/target/scala-$SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$FWDIR/bagel/target/scala-$SCALA_VERSION/classes" - CLASSPATH="$CLASSPATH:$FWDIR/graph/target/scala-$SCALA_VERSION/classes" + CLASSPATH="$CLASSPATH:$FWDIR/graphx/target/scala-$SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$FWDIR/streaming/target/scala-$SCALA_VERSION/classes" DEPS_ASSEMBLY_JAR=`ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/spark-assembly*hadoop*-deps.jar` @@ -61,7 +61,7 @@ if [[ $SPARK_TESTING == 1 ]]; then CLASSPATH="$CLASSPATH:$FWDIR/repl/target/scala-$SCALA_VERSION/test-classes" CLASSPATH="$CLASSPATH:$FWDIR/mllib/target/scala-$SCALA_VERSION/test-classes" CLASSPATH="$CLASSPATH:$FWDIR/bagel/target/scala-$SCALA_VERSION/test-classes" - CLASSPATH="$CLASSPATH:$FWDIR/graph/target/scala-$SCALA_VERSION/test-classes" + CLASSPATH="$CLASSPATH:$FWDIR/graphx/target/scala-$SCALA_VERSION/test-classes" CLASSPATH="$CLASSPATH:$FWDIR/streaming/target/scala-$SCALA_VERSION/test-classes" fi From 8ae108f6c48528f3bb7498d586eb51a70c043764 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Thu, 9 Jan 2014 23:25:35 -0800 Subject: [PATCH 365/531] Unpersist previous iterations in Pregel --- .../org/apache/spark/graphx/Analytics.scala | 2 +- .../org/apache/spark/graphx/EdgeRDD.scala | 5 +++++ .../org/apache/spark/graphx/Pregel.scala | 19 +++++++++++++++---- .../org/apache/spark/graphx/VertexRDD.scala | 5 +++++ .../spark/graphx/algorithms/PageRank.scala | 1 + .../graphx/impl/ReplicatedVertexView.scala | 16 ++++++++++++++-- 6 files changed, 41 insertions(+), 7 deletions(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Analytics.scala b/graphx/src/main/scala/org/apache/spark/graphx/Analytics.scala index def6d69190960..2c4c885a04c78 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Analytics.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Analytics.scala @@ -83,7 +83,7 @@ object Analytics extends Logging { println("GRAPHX: Number of edges " + graph.edges.count) //val pr = Analytics.pagerank(graph, numIter) - val pr = graph.pageRank(tol).vertices + val pr = graph.pageRank(tol).vertices.cache() println("GRAPHX: Total rank: " + pr.map(_._2).reduce(_+_)) 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 e4ef460e6f88a..7fd65806268df 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala @@ -44,6 +44,11 @@ class EdgeRDD[@specialized ED: ClassTag]( /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */ override def cache(): EdgeRDD[ED] = persist() + override def unpersist(blocking: Boolean = true): EdgeRDD[ED] = { + partitionsRDD.unpersist(blocking) + this + } + def mapEdgePartitions[ED2: ClassTag](f: (PartitionID, EdgePartition[ED]) => EdgePartition[ED2]) : EdgeRDD[ED2] = { // iter => iter.map { case (pid, ep) => (pid, f(ep)) } 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 8ddb78813557f..ed8733a806389 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala @@ -93,25 +93,36 @@ object Pregel { mergeMsg: (A, A) => A) : Graph[VD, ED] = { - var g = graph.mapVertices( (vid, vdata) => vprog(vid, vdata, initialMsg) ) + var g = graph.mapVertices( (vid, vdata) => vprog(vid, vdata, initialMsg) ).cache() // compute the messages - var messages = g.mapReduceTriplets(sendMsg, mergeMsg).cache() + var messages = g.mapReduceTriplets(sendMsg, mergeMsg) var activeMessages = messages.count() // Loop + var prevG: Graph[VD, ED] = null var i = 0 while (activeMessages > 0 && i < maxIterations) { // Receive the messages. Vertices that didn't get any messages do not appear in newVerts. val newVerts = g.vertices.innerJoin(messages)(vprog).cache() // Update the graph with the new vertices. + prevG = g g = g.outerJoinVertices(newVerts) { (vid, old, newOpt) => newOpt.getOrElse(old) } + g.vertices.cache() val oldMessages = messages // Send new messages. Vertices that didn't get any messages don't appear in newVerts, so don't - // get to send messages. + // get to send messages. We must cache messages so it can be materialized on the next line, + // allowing us to uncache the previous iteration. messages = g.mapReduceTriplets(sendMsg, mergeMsg, Some((newVerts, EdgeDirection.Out))).cache() + // Materializes messages, newVerts, and g.rvv (which materializes g.vertices). Hides + // oldMessages (depended on by newVerts), newVerts (depended on by messages), prevG.vertices + // (depended on by newVerts and g.vertices), and prevG.rvv (depended on by oldMessages and + // g.rvv). activeMessages = messages.count() - // after counting we can unpersist the old messages + // Unpersist hidden RDDs oldMessages.unpersist(blocking=false) + newVerts.unpersist(blocking=false) + prevG.vertices.unpersist(blocking=false) + prevG.asInstanceOf[org.apache.spark.graphx.impl.GraphImpl[VD, ED]].replicatedVertexView.unpersist(blocking=false) // count the iteration i += 1 } 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 cfee9b089f9e4..971e2615d4afd 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala @@ -98,6 +98,11 @@ class VertexRDD[@specialized VD: ClassTag]( /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */ override def cache(): VertexRDD[VD] = persist() + override def unpersist(blocking: Boolean = true): VertexRDD[VD] = { + partitionsRDD.unpersist(blocking) + this + } + /** Return the number of vertices in this set. */ override def count(): Long = { partitionsRDD.map(_.size).reduce(_ + _) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/PageRank.scala b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/PageRank.scala index b423104eda358..179d310554ec1 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/PageRank.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/PageRank.scala @@ -125,6 +125,7 @@ object PageRank extends Logging { .mapTriplets( e => 1.0 / e.srcAttr ) // Set the vertex attributes to (initalPR, delta = 0) .mapVertices( (id, attr) => (0.0, 0.0) ) + .cache() // Display statistics about pagerank logInfo(pagerankGraph.statistics.toString) 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 63180bc3af473..0e2f5a9dd93cd 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 @@ -14,9 +14,11 @@ import org.apache.spark.graphx._ * specified, `updatedVerts` are treated as incremental updates to the previous view. Otherwise, a * fresh view is created. * - * The view is always cached (i.e., once it is created, it remains materialized). This avoids + * The view is always cached (i.e., once it is evaluated, it remains materialized). This avoids * constructing it twice if the user calls graph.triplets followed by graph.mapReduceTriplets, for - * example. + * example. However, it means iterative algorithms must manually call `Graph.unpersist` on previous + * iterations' graphs for best GC performance. See the implementation of + * [[org.apache.spark.graphx.Pregel]] for an example. */ private[impl] class ReplicatedVertexView[VD: ClassTag]( @@ -51,6 +53,16 @@ class ReplicatedVertexView[VD: ClassTag]( private lazy val dstAttrOnly: RDD[(PartitionID, VertexPartition[VD])] = create(false, true) private lazy val noAttrs: RDD[(PartitionID, VertexPartition[VD])] = create(false, false) + def unpersist(blocking: Boolean = true): ReplicatedVertexView[VD] = { + bothAttrs.unpersist(blocking) + srcAttrOnly.unpersist(blocking) + dstAttrOnly.unpersist(blocking) + noAttrs.unpersist(blocking) + // Don't unpersist localVertexIDMap because a future ReplicatedVertexView may be using it + // without modification + this + } + def get(includeSrc: Boolean, includeDst: Boolean): RDD[(PartitionID, VertexPartition[VD])] = { (includeSrc, includeDst) match { case (true, true) => bothAttrs From 2578332f97d489bf0e238b2dbe1a3e1d0258a910 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Thu, 9 Jan 2014 23:34:35 -0800 Subject: [PATCH 366/531] Add Graph.unpersistVertices() --- .../main/scala/org/apache/spark/graphx/Graph.scala | 6 ++++++ .../scala/org/apache/spark/graphx/Pregel.scala | 14 ++++++-------- .../org/apache/spark/graphx/impl/GraphImpl.scala | 6 ++++++ 3 files changed, 18 insertions(+), 8 deletions(-) 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 2b7c0a2583f21..6f2d19d0dab8f 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala @@ -93,6 +93,12 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] { */ def cache(): Graph[VD, ED] + /** + * Uncache only the vertices of this graph, leaving the edges alone. This is useful because most + * graph operations modify the vertices but reuse the edges. + */ + def unpersistVertices(blocking: Boolean = true): Graph[VD, ED] + /** * Repartition the edges in the graph according to partitionStrategy. */ 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 ed8733a806389..0af230ed29bb4 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala @@ -106,23 +106,21 @@ object Pregel { // Update the graph with the new vertices. prevG = g g = g.outerJoinVertices(newVerts) { (vid, old, newOpt) => newOpt.getOrElse(old) } - g.vertices.cache() + g.cache() val oldMessages = messages // Send new messages. Vertices that didn't get any messages don't appear in newVerts, so don't // get to send messages. We must cache messages so it can be materialized on the next line, // allowing us to uncache the previous iteration. messages = g.mapReduceTriplets(sendMsg, mergeMsg, Some((newVerts, EdgeDirection.Out))).cache() - // Materializes messages, newVerts, and g.rvv (which materializes g.vertices). Hides - // oldMessages (depended on by newVerts), newVerts (depended on by messages), prevG.vertices - // (depended on by newVerts and g.vertices), and prevG.rvv (depended on by oldMessages and - // g.rvv). + // The call to count() materializes `messages`, `newVerts`, and the vertices of `g`. This + // hides oldMessages (depended on by newVerts), newVerts (depended on by messages), and the + // vertices of prevG (depended on by newVerts, oldMessages, and the vertices of g). activeMessages = messages.count() - // Unpersist hidden RDDs + // Unpersist the RDDs hidden by newly-materialized RDDs oldMessages.unpersist(blocking=false) newVerts.unpersist(blocking=false) - prevG.vertices.unpersist(blocking=false) - prevG.asInstanceOf[org.apache.spark.graphx.impl.GraphImpl[VD, ED]].replicatedVertexView.unpersist(blocking=false) + prevG.unpersistVertices(blocking=false) // count the iteration i += 1 } 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 be9f1881500f5..2dd1324d4fdda 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 @@ -65,6 +65,12 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( override def cache(): Graph[VD, ED] = persist(StorageLevel.MEMORY_ONLY) + override def unpersistVertices(blocking: Boolean = true): Graph[VD, ED] = { + vertices.unpersist(blocking) + replicatedVertexView.unpersist(blocking) + this + } + override def partitionBy(partitionStrategy: PartitionStrategy): Graph[VD, ED] = { val numPartitions = edges.partitions.size val edTag = classTag[ED] From 8b6b8ac87f6ffb92b3395344bf2696d5c7fb3798 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 10 Jan 2014 00:34:08 -0800 Subject: [PATCH 367/531] Unpersist previous iterations in GraphLab --- .../scala/org/apache/spark/graphx/GraphLab.scala | 16 ++++++++++------ 1 file changed, 10 insertions(+), 6 deletions(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphLab.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphLab.scala index 437288405f307..94cfa7e126e0d 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphLab.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphLab.scala @@ -102,6 +102,7 @@ object GraphLab extends Logging { // Main Loop --------------------------------------------------------------------- var i = 0 var numActive = activeGraph.numVertices + var prevActiveGraph: Graph[(Boolean, VD), ED] = null while (i < numIter && numActive > 0) { // Gather @@ -109,22 +110,25 @@ object GraphLab extends Logging { activeGraph.aggregateNeighbors(gather, mergeFunc, gatherDirection) // Apply - activeGraph = activeGraph.outerJoinVertices(gathered)(apply).cache() - - + val applied = activeGraph.outerJoinVertices(gathered)(apply) // Scatter is basically a gather in the opposite direction so we reverse the edge direction - // activeGraph: Graph[(Boolean, VD), ED] val scattered: RDD[(VertexID, Boolean)] = - activeGraph.aggregateNeighbors(scatter, _ || _, scatterDirection.reverse) + applied.aggregateNeighbors(scatter, _ || _, scatterDirection.reverse) + prevActiveGraph = activeGraph activeGraph = activeGraph.outerJoinVertices(scattered)(applyActive).cache() - // Calculate the number of active vertices + // Calculate the number of active vertices. The call to reduce() materializes the vertices of + // `activeGraph`, hiding the vertices of `prevActiveGraph`. numActive = activeGraph.vertices.map{ case (vid, data) => if (data._1) 1 else 0 }.reduce(_ + _) logInfo("Number active vertices: " + numActive) + + // Unpersist the RDDs hidden by newly-materialized RDDs + prevActiveGraph.unpersistVertices(blocking=false) + i += 1 } From ba511f890ee0d7f85746126c4be734538ede21ea Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 10 Jan 2014 00:35:02 -0800 Subject: [PATCH 368/531] Avoid recomputation by caching all multiply-used RDDs --- .../spark/graphx/algorithms/PageRank.scala | 1 + .../spark/graphx/algorithms/SVDPlusPlus.scala | 4 ++ .../StronglyConnectedComponents.scala | 9 ++-- .../apache/spark/graphx/impl/GraphImpl.scala | 47 +++++++++++-------- .../apache/spark/graphx/GraphOpsSuite.scala | 8 ++-- .../org/apache/spark/graphx/GraphSuite.scala | 8 ++-- .../org/apache/spark/graphx/PregelSuite.scala | 6 +-- .../apache/spark/graphx/VertexRDDSuite.scala | 12 ++--- .../algorithms/ConnectedComponentsSuite.scala | 16 +++---- .../graphx/algorithms/PageRankSuite.scala | 8 ++-- .../graphx/algorithms/SVDPlusPlusSuite.scala | 1 + 11 files changed, 67 insertions(+), 53 deletions(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/PageRank.scala b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/PageRank.scala index 179d310554ec1..ab447d5422903 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/PageRank.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/PageRank.scala @@ -61,6 +61,7 @@ object PageRank extends Logging { .mapTriplets( e => 1.0 / e.srcAttr ) // Set the vertex attributes to the initial pagerank values .mapVertices( (id, attr) => 1.0 ) + .cache() // Display statistics about pagerank logInfo(pagerankGraph.statistics.toString) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/SVDPlusPlus.scala b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/SVDPlusPlus.scala index 8fdfa3d9077a4..2a13553d7921c 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/SVDPlusPlus.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/SVDPlusPlus.scala @@ -42,6 +42,7 @@ object SVDPlusPlus { } // calculate global rating mean + edges.cache() val (rs, rc) = edges.map(e => (e.attr, 1L)).reduce((a, b) => (a._1 + b._1, a._2 + b._2)) val u = rs / rc @@ -72,11 +73,13 @@ object SVDPlusPlus { for (i <- 0 until conf.maxIters) { // 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 } // 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)]) => @@ -94,6 +97,7 @@ object SVDPlusPlus { val err = (et.attr - pred) * (et.attr - pred) Iterator((et.dstId, err)) } + 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 diff --git a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/StronglyConnectedComponents.scala b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/StronglyConnectedComponents.scala index 49ec91aeddb32..864f0ec57caff 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/StronglyConnectedComponents.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/StronglyConnectedComponents.scala @@ -22,7 +22,7 @@ object StronglyConnectedComponents { // the graph we update with final SCC ids, and the graph we return at the end var sccGraph = graph.mapVertices { case (vid, _) => vid } // graph we are going to work with in our iterations - var sccWorkGraph = graph.mapVertices { case (vid, _) => (vid, false) } + var sccWorkGraph = graph.mapVertices { case (vid, _) => (vid, false) }.cache() var numVertices = sccWorkGraph.numVertices var iter = 0 @@ -32,10 +32,9 @@ object StronglyConnectedComponents { numVertices = sccWorkGraph.numVertices sccWorkGraph = sccWorkGraph.outerJoinVertices(sccWorkGraph.outDegrees) { (vid, data, degreeOpt) => if (degreeOpt.isDefined) data else (vid, true) - } - sccWorkGraph = sccWorkGraph.outerJoinVertices(sccWorkGraph.inDegrees) { + }.outerJoinVertices(sccWorkGraph.inDegrees) { (vid, data, degreeOpt) => if (degreeOpt.isDefined) data else (vid, true) - } + }.cache() // get all vertices to be removed val finalVertices = sccWorkGraph.vertices @@ -47,7 +46,7 @@ object StronglyConnectedComponents { (vid, scc, opt) => opt.getOrElse(scc) } // only keep vertices that are not final - sccWorkGraph = sccWorkGraph.subgraph(vpred = (vid, data) => !data._2) + sccWorkGraph = sccWorkGraph.subgraph(vpred = (vid, data) => !data._2).cache() } while (sccWorkGraph.numVertices < numVertices) sccWorkGraph = sccWorkGraph.mapVertices{ case (vid, (color, isFinal)) => (vid, isFinal) } 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 2dd1324d4fdda..987a646c0c18a 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,19 +32,6 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( @transient val replicatedVertexView: ReplicatedVertexView[VD]) extends Graph[VD, ED] with Serializable { - def this( - vertices: VertexRDD[VD], - edges: EdgeRDD[ED], - routingTable: RoutingTable) = { - this(vertices, edges, routingTable, new ReplicatedVertexView(vertices, edges, routingTable)) - } - - def this( - vertices: VertexRDD[VD], - edges: EdgeRDD[ED]) = { - this(vertices, edges, new RoutingTable(edges, vertices)) - } - /** 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] @@ -90,7 +77,7 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( val edgePartition = builder.toEdgePartition Iterator((pid, edgePartition)) }, preservesPartitioning = true).cache()) - new GraphImpl(vertices, newEdges) + GraphImpl(vertices, newEdges) } override def statistics: Map[String, Any] = { @@ -166,7 +153,7 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( override def mapVertices[VD2: ClassTag](f: (VertexID, VD) => VD2): Graph[VD2, ED] = { if (classTag[VD] equals classTag[VD2]) { // The map preserves type, so we can use incremental replication - val newVerts = vertices.mapVertexPartitions(_.map(f)) + val newVerts = vertices.mapVertexPartitions(_.map(f)).cache() val changedVerts = vertices.asInstanceOf[VertexRDD[VD2]].diff(newVerts) val newReplicatedVertexView = new ReplicatedVertexView[VD2]( changedVerts, edges, routingTable, @@ -174,7 +161,7 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( new GraphImpl(newVerts, edges, routingTable, newReplicatedVertexView) } else { // The map does not preserve type, so we must re-replicate all vertices - new GraphImpl(vertices.mapVertexPartitions(_.map(f)), edges, routingTable) + GraphImpl(vertices.mapVertexPartitions(_.map(f)), edges, routingTable) } } @@ -336,7 +323,7 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( } else { // updateF does not preserve type, so we must re-replicate all vertices val newVerts = vertices.leftJoin(updates)(updateF) - new GraphImpl(newVerts, edges, routingTable) + GraphImpl(newVerts, edges, routingTable) } } @@ -382,7 +369,29 @@ object GraphImpl { val vertexRDD = VertexRDD(vids, vPartitioned, defaultVertexAttr) - new GraphImpl(vertexRDD, edgeRDD) + GraphImpl(vertexRDD, edgeRDD) + } + + def apply[VD: ClassTag, ED: ClassTag]( + vertices: VertexRDD[VD], + edges: EdgeRDD[ED]): GraphImpl[VD, ED] = { + // Cache RDDs that are referenced multiple times + edges.cache() + + GraphImpl(vertices, edges, new RoutingTable(edges, vertices)) + } + + def apply[VD: ClassTag, ED: ClassTag]( + vertices: VertexRDD[VD], + edges: EdgeRDD[ED], + routingTable: RoutingTable): GraphImpl[VD, ED] = { + // Cache RDDs that are referenced multiple times. `routingTable` is cached by default, so we + // don't cache it explicitly. + vertices.cache() + edges.cache() + + new GraphImpl( + vertices, edges, routingTable, new ReplicatedVertexView(vertices, edges, routingTable)) } /** @@ -413,7 +422,7 @@ object GraphImpl { val vids = collectVertexIDsFromEdges(edges, new HashPartitioner(edges.partitions.size)) // Create the VertexRDD. val vertices = VertexRDD(vids.mapValues(x => defaultVertexAttr)) - new GraphImpl(vertices, edges) + GraphImpl(vertices, edges) } /** Collects all vids mentioned in edges and partitions them by partitioner. */ 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 cc281fce997bf..cd3c0bbd302b1 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala @@ -53,8 +53,8 @@ class GraphOpsSuite extends FunSuite with LocalSparkContext { withSpark { sc => 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) - val nbrs = graph.collectNeighborIds(EdgeDirection.Both) + val graph = Graph.fromEdgeTuples(rawEdges, 1.0).cache() + val nbrs = graph.collectNeighborIds(EdgeDirection.Both).cache() assert(nbrs.count === chain.size) assert(graph.numVertices === nbrs.count) nbrs.collect.foreach { case (vid, nbrs) => assert(nbrs.size === 2) } @@ -71,14 +71,14 @@ class GraphOpsSuite extends FunSuite with LocalSparkContext { val n = 5 val vertices = sc.parallelize((0 to n).map(x => (x:VertexID, x))) val edges = sc.parallelize((1 to n).map(x => Edge(0, x, x))) - val graph: Graph[Int, Int] = Graph(vertices, edges) + val graph: Graph[Int, Int] = Graph(vertices, edges).cache() val filteredGraph = graph.filter( graph => { val degrees: VertexRDD[Int] = graph.outDegrees graph.outerJoinVertices(degrees) {(vid, data, deg) => deg.getOrElse(0)} }, vpred = (vid: VertexID, deg:Int) => deg > 0 - ) + ).cache() val v = filteredGraph.vertices.collect().toSet assert(v === Set((0,0))) 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 094fa722a0030..c32a6cbb81128 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala @@ -175,7 +175,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { val n = 5 val vertices = sc.parallelize((0 to n).map(x => (x:VertexID, x))) val edges = sc.parallelize((1 to n).map(x => Edge(0, x, x))) - val graph: Graph[Int, Int] = Graph(vertices, edges) + val graph: Graph[Int, Int] = Graph(vertices, edges).cache() val subgraph = graph.subgraph( e => e.dstId != 4L, @@ -211,7 +211,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { test("mapReduceTriplets") { withSpark { sc => val n = 5 - val star = starGraph(sc, n).mapVertices { (_, _) => 0 } + val star = starGraph(sc, n).mapVertices { (_, _) => 0 }.cache() val starDeg = star.joinVertices(star.degrees){ (vid, oldV, deg) => deg } val neighborDegreeSums = starDeg.mapReduceTriplets( edge => Iterator((edge.srcId, edge.dstAttr), (edge.dstId, edge.srcAttr)), @@ -235,7 +235,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { // outerJoinVertices followed by mapReduceTriplets(activeSetOpt) val ringEdges = sc.parallelize((0 until n).map(x => (x: VertexID, (x+1) % n: VertexID)), 3) val ring = Graph.fromEdgeTuples(ringEdges, 0) .mapVertices((vid, attr) => vid).cache() - val changed = ring.vertices.filter { case (vid, attr) => attr % 2 == 1 }.mapValues(-_) + val changed = ring.vertices.filter { case (vid, attr) => attr % 2 == 1 }.mapValues(-_).cache() val changedGraph = ring.outerJoinVertices(changed) { (vid, old, newOpt) => newOpt.getOrElse(old) } val numOddNeighbors = changedGraph.mapReduceTriplets(et => { // Map function should only run on edges with source in the active set @@ -252,7 +252,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { test("outerJoinVertices") { withSpark { sc => val n = 5 - val reverseStar = starGraph(sc, n).reverse + val reverseStar = starGraph(sc, n).reverse.cache() // outerJoinVertices changing type val reverseStarDegrees = reverseStar.outerJoinVertices(reverseStar.outDegrees) { (vid, a, bOpt) => bOpt.getOrElse(0) } 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 429622357f0bd..1ff3d75633f22 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/PregelSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/PregelSuite.scala @@ -10,8 +10,8 @@ class PregelSuite extends FunSuite with LocalSparkContext { test("1 iteration") { withSpark { sc => val n = 5 - val star = - Graph.fromEdgeTuples(sc.parallelize((1 to n).map(x => (0: VertexID, x: VertexID)), 3), "v") + val starEdges = (1 to n).map(x => (0: VertexID, x: VertexID)) + val star = Graph.fromEdgeTuples(sc.parallelize(starEdges, 3), "v").cache() val result = Pregel(star, 0)( (vid, attr, msg) => attr, et => Iterator.empty, @@ -27,7 +27,7 @@ class PregelSuite extends FunSuite with LocalSparkContext { sc.parallelize((1 until n).map(x => (x: VertexID, x + 1: VertexID)), 3), 0).cache() assert(chain.vertices.collect.toSet === (1 to n).map(x => (x: VertexID, 0)).toSet) - val chainWithSeed = chain.mapVertices { (vid, attr) => if (vid == 1) 1 else 0 } + val chainWithSeed = chain.mapVertices { (vid, attr) => if (vid == 1) 1 else 0 }.cache() assert(chainWithSeed.vertices.collect.toSet === Set((1: VertexID, 1)) ++ (2 to n).map(x => (x: VertexID, 0)).toSet) val result = Pregel(chainWithSeed, 0)( diff --git a/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala index 573b708e89dd9..d94a3aa67c925 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala @@ -33,8 +33,8 @@ class VertexRDDSuite extends FunSuite with LocalSparkContext { test("diff") { withSpark { sc => val n = 100 - val verts = vertices(sc, n) - val flipEvens = verts.mapValues(x => if (x % 2 == 0) -x else x) + val verts = vertices(sc, n).cache() + val flipEvens = verts.mapValues(x => if (x % 2 == 0) -x else x).cache() // diff should keep only the changed vertices assert(verts.diff(flipEvens).map(_._2).collect().toSet === (2 to n by 2).map(-_).toSet) // diff should keep the vertex values from `other` @@ -45,8 +45,8 @@ class VertexRDDSuite extends FunSuite with LocalSparkContext { test("leftJoin") { withSpark { sc => val n = 100 - val verts = vertices(sc, n) - val evens = verts.filter(q => ((q._2 % 2) == 0)) + val verts = vertices(sc, n).cache() + val evens = verts.filter(q => ((q._2 % 2) == 0)).cache() // leftJoin with another VertexRDD assert(verts.leftJoin(evens) { (id, a, bOpt) => a - bOpt.getOrElse(0) }.collect.toSet === (0 to n by 2).map(x => (x.toLong, 0)).toSet ++ (1 to n by 2).map(x => (x.toLong, x)).toSet) @@ -60,8 +60,8 @@ class VertexRDDSuite extends FunSuite with LocalSparkContext { test("innerJoin") { withSpark { sc => val n = 100 - val verts = vertices(sc, n) - val evens = verts.filter(q => ((q._2 % 2) == 0)) + val verts = vertices(sc, n).cache() + val evens = verts.filter(q => ((q._2 % 2) == 0)).cache() // innerJoin with another VertexRDD assert(verts.innerJoin(evens) { (id, a, b) => a - b }.collect.toSet === (0 to n by 2).map(x => (x.toLong, 0)).toSet) diff --git a/graphx/src/test/scala/org/apache/spark/graphx/algorithms/ConnectedComponentsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/ConnectedComponentsSuite.scala index 209191ef07867..16fc3fe5a21c7 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/algorithms/ConnectedComponentsSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/ConnectedComponentsSuite.scala @@ -13,8 +13,8 @@ class ConnectedComponentsSuite extends FunSuite with LocalSparkContext { test("Grid Connected Components") { withSpark { sc => - val gridGraph = GraphGenerators.gridGraph(sc, 10, 10).cache() - val ccGraph = gridGraph.connectedComponents().cache() + val gridGraph = GraphGenerators.gridGraph(sc, 10, 10) + val ccGraph = gridGraph.connectedComponents() val maxCCid = ccGraph.vertices.map { case (vid, ccId) => ccId }.sum assert(maxCCid === 0) } @@ -23,8 +23,8 @@ class ConnectedComponentsSuite extends FunSuite with LocalSparkContext { test("Reverse Grid Connected Components") { withSpark { sc => - val gridGraph = GraphGenerators.gridGraph(sc, 10, 10).reverse.cache() - val ccGraph = gridGraph.connectedComponents().cache() + val gridGraph = GraphGenerators.gridGraph(sc, 10, 10).reverse + val ccGraph = gridGraph.connectedComponents() val maxCCid = ccGraph.vertices.map { case (vid, ccId) => ccId }.sum assert(maxCCid === 0) } @@ -36,8 +36,8 @@ class ConnectedComponentsSuite extends FunSuite with LocalSparkContext { val chain1 = (0 until 9).map(x => (x, x+1) ) val chain2 = (10 until 20).map(x => (x, x+1) ) val rawEdges = sc.parallelize(chain1 ++ chain2, 3).map { case (s,d) => (s.toLong, d.toLong) } - val twoChains = Graph.fromEdgeTuples(rawEdges, 1.0).cache() - val ccGraph = twoChains.connectedComponents().cache() + val twoChains = Graph.fromEdgeTuples(rawEdges, 1.0) + val ccGraph = twoChains.connectedComponents() val vertices = ccGraph.vertices.collect() for ( (id, cc) <- vertices ) { if(id < 10) { assert(cc === 0) } @@ -59,8 +59,8 @@ class ConnectedComponentsSuite extends FunSuite with LocalSparkContext { val chain1 = (0 until 9).map(x => (x, x+1) ) val chain2 = (10 until 20).map(x => (x, x+1) ) val rawEdges = sc.parallelize(chain1 ++ chain2, 3).map { case (s,d) => (s.toLong, d.toLong) } - val twoChains = Graph.fromEdgeTuples(rawEdges, true).reverse.cache() - val ccGraph = twoChains.connectedComponents().cache() + val twoChains = Graph.fromEdgeTuples(rawEdges, true).reverse + val ccGraph = twoChains.connectedComponents() val vertices = ccGraph.vertices.collect for ( (id, cc) <- vertices ) { if (id < 10) { diff --git a/graphx/src/test/scala/org/apache/spark/graphx/algorithms/PageRankSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/PageRankSuite.scala index cd857bd3a18b2..de2c2d110773c 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/algorithms/PageRankSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/PageRankSuite.scala @@ -57,7 +57,7 @@ class PageRankSuite extends FunSuite with LocalSparkContext { val resetProb = 0.15 val errorTol = 1.0e-5 - val staticRanks1 = starGraph.staticPageRank(numIter = 1, resetProb).vertices.cache() + val staticRanks1 = starGraph.staticPageRank(numIter = 1, resetProb).vertices val staticRanks2 = starGraph.staticPageRank(numIter = 2, resetProb).vertices.cache() // Static PageRank should only take 2 iterations to converge @@ -92,7 +92,7 @@ class PageRankSuite extends FunSuite with LocalSparkContext { val staticRanks = gridGraph.staticPageRank(numIter, resetProb).vertices.cache() val dynamicRanks = gridGraph.pageRank(tol, resetProb).vertices.cache() - val referenceRanks = VertexRDD(sc.parallelize(GridPageRank(rows, cols, numIter, resetProb))) + val referenceRanks = VertexRDD(sc.parallelize(GridPageRank(rows, cols, numIter, resetProb))).cache() assert(compareRanks(staticRanks, referenceRanks) < errorTol) assert(compareRanks(dynamicRanks, referenceRanks) < errorTol) @@ -110,8 +110,8 @@ class PageRankSuite extends FunSuite with LocalSparkContext { val numIter = 10 val errorTol = 1.0e-5 - val staticRanks = chain.staticPageRank(numIter, resetProb).vertices.cache() - val dynamicRanks = chain.pageRank(tol, resetProb).vertices.cache() + val staticRanks = chain.staticPageRank(numIter, resetProb).vertices + val dynamicRanks = chain.pageRank(tol, resetProb).vertices assert(compareRanks(staticRanks, dynamicRanks) < errorTol) } diff --git a/graphx/src/test/scala/org/apache/spark/graphx/algorithms/SVDPlusPlusSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/SVDPlusPlusSuite.scala index 06604198d73a0..7bd93e0e6c000 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/algorithms/SVDPlusPlusSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/SVDPlusPlusSuite.scala @@ -20,6 +20,7 @@ class SVDPlusPlusSuite extends FunSuite with LocalSparkContext { } val conf = new SVDPlusPlusConf(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) => if (vid % 2 == 1) vd._4 else 0.0 }.reduce(_ + _) / graph.triplets.collect.size From b1eeefb4016d69aa0beadd302496c8250766d9b7 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Fri, 10 Jan 2014 00:39:08 -0800 Subject: [PATCH 369/531] WIP. Updating figures and cleaning up initial skeleton for GraphX Programming guide. --- docs/_layouts/global.html | 10 +- docs/graphx-programming-guide.md | 277 +++++++++---------- docs/img/data_parallel_vs_graph_parallel.png | Bin 199060 -> 432725 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/graphx_figures.pptx | Bin 0 -> 1118035 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 -> 79056 bytes docs/img/tables_and_graphs.png | Bin 68905 -> 166265 bytes docs/img/vertex_routing_edge_tables.png | Bin 0 -> 570007 bytes docs/index.md | 6 +- 12 files changed, 134 insertions(+), 159 deletions(-) create mode 100644 docs/img/edge_cut_vs_vertex_cut.png create mode 100644 docs/img/graph_analytics_pipeline.png create mode 100644 docs/img/graphx_figures.pptx create mode 100644 docs/img/graphx_logo.png create mode 100644 docs/img/graphx_performance_comparison.png create mode 100644 docs/img/property_graph.png create mode 100644 docs/img/vertex_routing_edge_tables.png diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html index ad7969d012283..7721854685fb2 100755 --- a/docs/_layouts/global.html +++ b/docs/_layouts/global.html @@ -21,7 +21,7 @@ - + @@ -67,10 +67,10 @@
  • MLlib (Machine Learning)
  • -
  • Bagel (Pregel on Spark)
  • +
  • GraphX (Graph-Parallel Spark)
  • - + @@ -161,7 +161,7 @@

    Heading

    - +
  • Spark Streaming