From 6c2691d0a0ed46a8b8093e05a4708706cf187168 Mon Sep 17 00:00:00 2001 From: Kan Zhang Date: Sat, 10 May 2014 14:01:08 -0700 Subject: [PATCH 01/18] [SPARK-1690] Tolerating empty elements when saving Python RDD to text files Tolerate empty strings in PythonRDD Author: Kan Zhang Closes #644 from kanzhang/SPARK-1690 and squashes the following commits: c62ad33 [Kan Zhang] Adding Python doctest 473ec4b [Kan Zhang] [SPARK-1690] Tolerating empty elements when saving Python RDD to text files --- .../scala/org/apache/spark/api/python/PythonRDD.scala | 5 +++-- python/pyspark/rdd.py | 8 ++++++++ 2 files changed, 11 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index 388b838d78bb..2971c277aa86 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -94,6 +94,7 @@ private[spark] class PythonRDD[T: ClassTag]( val obj = new Array[Byte](length) stream.readFully(obj) obj + case 0 => Array.empty[Byte] case SpecialLengths.TIMING_DATA => // Timing data from worker val bootTime = stream.readLong() @@ -123,7 +124,7 @@ private[spark] class PythonRDD[T: ClassTag]( stream.readFully(update) accumulator += Collections.singletonList(update) } - Array.empty[Byte] + null } } catch { @@ -143,7 +144,7 @@ private[spark] class PythonRDD[T: ClassTag]( var _nextObj = read() - def hasNext = _nextObj.length != 0 + def hasNext = _nextObj != null } new InterruptibleIterator(context, stdoutIterator) } diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 3a1c56af5b22..4f74824ba4cf 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -891,6 +891,14 @@ def saveAsTextFile(self, path): >>> from glob import glob >>> ''.join(sorted(input(glob(tempFile.name + "/part-0000*")))) '0\\n1\\n2\\n3\\n4\\n5\\n6\\n7\\n8\\n9\\n' + + Empty lines are tolerated when saving to text files. + + >>> tempFile2 = NamedTemporaryFile(delete=True) + >>> tempFile2.close() + >>> sc.parallelize(['', 'foo', '', 'bar', '']).saveAsTextFile(tempFile2.name) + >>> ''.join(sorted(input(glob(tempFile2.name + "/part-0000*")))) + '\\n\\n\\nbar\\nfoo\\n' """ def func(split, iterator): for x in iterator: From 905173df57b90f90ebafb22e43f55164445330e6 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Sat, 10 May 2014 14:48:07 -0700 Subject: [PATCH 02/18] Unify GraphImpl RDDs + other graph load optimizations This PR makes the following changes, primarily in e4fbd329aef85fe2c38b0167255d2a712893d683: 1. *Unify RDDs to avoid zipPartitions.* A graph used to be four RDDs: vertices, edges, routing table, and triplet view. This commit merges them down to two: vertices (with routing table), and edges (with replicated vertices). 2. *Avoid duplicate shuffle in graph building.* We used to do two shuffles when building a graph: one to extract routing information from the edges and move it to the vertices, and another to find nonexistent vertices referred to by edges. With this commit, the latter is done as a side effect of the former. 3. *Avoid no-op shuffle when joins are fully eliminated.* This is a side effect of unifying the edges and the triplet view. 4. *Join elimination for mapTriplets.* 5. *Ship only the needed vertex attributes when upgrading the triplet view.* If the triplet view already contains source attributes, and we now need both attributes, only ship destination attributes rather than re-shipping both. This is done in `ReplicatedVertexView#upgrade`. Author: Ankur Dave Closes #497 from ankurdave/unify-rdds and squashes the following commits: 332ab43 [Ankur Dave] Merge remote-tracking branch 'apache-spark/master' into unify-rdds 4933e2e [Ankur Dave] Exclude RoutingTable from binary compatibility check 5ba8789 [Ankur Dave] Add GraphX upgrade guide from Spark 0.9.1 13ac845 [Ankur Dave] Merge remote-tracking branch 'apache-spark/master' into unify-rdds a04765c [Ankur Dave] Remove unnecessary toOps call 57202e8 [Ankur Dave] Replace case with pair parameter 75af062 [Ankur Dave] Add explicit return types 04d3ae5 [Ankur Dave] Convert implicit parameter to context bound c88b269 [Ankur Dave] Revert upgradeIterator to if-in-a-loop 0d3584c [Ankur Dave] EdgePartition.size should be val 2a928b2 [Ankur Dave] Set locality wait 10b3596 [Ankur Dave] Clean up public API ae36110 [Ankur Dave] Fix style errors e4fbd32 [Ankur Dave] Unify GraphImpl RDDs + other graph load optimizations d6d60e2 [Ankur Dave] In GraphLoader, coalesce to minEdgePartitions 62c7b78 [Ankur Dave] In Analytics, take PageRank numIter d64e8d4 [Ankur Dave] Log current Pregel iteration --- docs/graphx-programming-guide.md | 22 +- .../org/apache/spark/graphx/EdgeRDD.scala | 56 +-- .../org/apache/spark/graphx/EdgeTriplet.scala | 2 + .../scala/org/apache/spark/graphx/Graph.scala | 2 +- .../spark/graphx/GraphKryoRegistrator.scala | 8 +- .../org/apache/spark/graphx/GraphLoader.scala | 10 +- .../org/apache/spark/graphx/GraphOps.scala | 17 +- .../org/apache/spark/graphx/Pregel.scala | 6 +- .../org/apache/spark/graphx/VertexRDD.scala | 166 ++++++--- .../spark/graphx/impl/EdgePartition.scala | 132 +++++-- .../graphx/impl/EdgePartitionBuilder.scala | 18 +- .../graphx/impl/EdgeTripletIterator.scala | 50 ++- .../apache/spark/graphx/impl/GraphImpl.scala | 344 +++++++----------- .../graphx/impl/MessageToPartition.scala | 21 +- .../graphx/impl/ReplicatedVertexView.scala | 238 ++++-------- .../spark/graphx/impl/RoutingTable.scala | 82 ----- .../graphx/impl/RoutingTablePartition.scala | 158 ++++++++ .../spark/graphx/impl/Serializers.scala | 29 ++ .../impl/ShippableVertexPartition.scala | 149 ++++++++ .../spark/graphx/impl/VertexPartition.scala | 269 ++------------ .../graphx/impl/VertexPartitionBase.scala | 91 +++++ .../graphx/impl/VertexPartitionBaseOps.scala | 245 +++++++++++++ .../apache/spark/graphx/lib/Analytics.scala | 8 +- .../org/apache/spark/graphx/GraphSuite.scala | 10 +- .../graphx/impl/EdgePartitionSuite.scala | 48 ++- .../impl/EdgeTripletIteratorSuite.scala | 10 +- .../graphx/impl/VertexPartitionSuite.scala | 11 - project/MimaBuild.scala | 2 + 28 files changed, 1353 insertions(+), 851 deletions(-) delete mode 100644 graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTable.scala create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md index 07be8ba58efa..42ab27bf55cc 100644 --- a/docs/graphx-programming-guide.md +++ b/docs/graphx-programming-guide.md @@ -86,6 +86,12 @@ support the [Bagel API](api/scala/index.html#org.apache.spark.bagel.package) and [Bagel programming guide](bagel-programming-guide.html). However, we encourage Bagel users to explore the new GraphX API and comment on issues that may complicate the transition from Bagel. +## Upgrade Guide from Spark 0.9.1 + +GraphX in Spark {{site.SPARK_VERSION}} contains one user-facing interface change from Spark 0.9.1. [`EdgeRDD`][EdgeRDD] may now store adjacent vertex attributes to construct the triplets, so it has gained a type parameter. The edges of a graph of type `Graph[VD, ED]` are of type `EdgeRDD[ED, VD]` rather than `EdgeRDD[ED]`. + +[EdgeRDD]: api/scala/index.html#org.apache.spark.graphx.EdgeRDD + # Getting Started To get started you first need to import Spark and GraphX into your project, as follows: @@ -145,12 +151,12 @@ the vertices and edges of the graph: {% highlight scala %} class Graph[VD, ED] { val vertices: VertexRDD[VD] - val edges: EdgeRDD[ED] + val edges: EdgeRDD[ED, VD] } {% endhighlight %} -The classes `VertexRDD[VD]` and `EdgeRDD[ED]` extend and are optimized versions of `RDD[(VertexID, -VD)]` and `RDD[Edge[ED]]` respectively. Both `VertexRDD[VD]` and `EdgeRDD[ED]` provide additional +The classes `VertexRDD[VD]` and `EdgeRDD[ED, VD]` extend and are optimized versions of `RDD[(VertexID, +VD)]` and `RDD[Edge[ED]]` respectively. Both `VertexRDD[VD]` and `EdgeRDD[ED, VD]` provide additional functionality built around graph computation and leverage internal optimizations. We discuss the `VertexRDD` and `EdgeRDD` API in greater detail in the section on [vertex and edge RDDs](#vertex_and_edge_rdds) but for now they can be thought of as simply RDDs of the form: @@ -302,7 +308,7 @@ class Graph[VD, ED] { val degrees: VertexRDD[Int] // Views of the graph as collections ============================================================= val vertices: VertexRDD[VD] - val edges: EdgeRDD[ED] + val edges: EdgeRDD[ED, VD] val triplets: RDD[EdgeTriplet[VD, ED]] // Functions for caching graphs ================================================================== def persist(newLevel: StorageLevel = StorageLevel.MEMORY_ONLY): Graph[VD, ED] @@ -908,7 +914,7 @@ val setC: VertexRDD[Double] = setA.innerJoin(setB)((id, a, b) => a + b) ## EdgeRDDs -The `EdgeRDD[ED]`, which extends `RDD[Edge[ED]]` organizes the edges in blocks partitioned using one +The `EdgeRDD[ED, VD]`, which extends `RDD[Edge[ED]]` organizes the edges in blocks partitioned using one of the various partitioning strategies defined in [`PartitionStrategy`][PartitionStrategy]. Within each partition, edge attributes and adjacency structure, are stored separately enabling maximum reuse when changing attribute values. @@ -918,11 +924,11 @@ reuse when changing attribute values. The three additional functions exposed by the `EdgeRDD` are: {% highlight scala %} // Transform the edge attributes while preserving the structure -def mapValues[ED2](f: Edge[ED] => ED2): EdgeRDD[ED2] +def mapValues[ED2](f: Edge[ED] => ED2): EdgeRDD[ED2, VD] // Revere the edges reusing both attributes and structure -def reverse: EdgeRDD[ED] +def reverse: EdgeRDD[ED, VD] // Join two `EdgeRDD`s partitioned using the same partitioning strategy. -def innerJoin[ED2, ED3](other: EdgeRDD[ED2])(f: (VertexId, VertexId, ED, ED2) => ED3): EdgeRDD[ED3] +def innerJoin[ED2, ED3](other: EdgeRDD[ED2, VD])(f: (VertexId, VertexId, ED, ED2) => ED3): EdgeRDD[ED3, VD] {% endhighlight %} In most applications we have found that operations on the `EdgeRDD` are accomplished through the 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 fa78ca99b889..a8fc09507251 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala @@ -20,16 +20,19 @@ package org.apache.spark.graphx import scala.reflect.{classTag, ClassTag} import org.apache.spark.{OneToOneDependency, Partition, Partitioner, TaskContext} -import org.apache.spark.graphx.impl.EdgePartition import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel +import org.apache.spark.graphx.impl.EdgePartition + /** - * `EdgeRDD[ED]` extends `RDD[Edge[ED]]` by storing the edges in columnar format on each partition - * for performance. + * `EdgeRDD[ED, VD]` extends `RDD[Edge[ED]]` by storing the edges in columnar format on each + * partition for performance. It may additionally store the vertex attributes associated with each + * edge to provide the triplet view. Shipping of the vertex attributes is managed by + * `impl.ReplicatedVertexView`. */ -class EdgeRDD[@specialized ED: ClassTag]( - val partitionsRDD: RDD[(PartitionID, EdgePartition[ED])]) +class EdgeRDD[@specialized ED: ClassTag, VD: ClassTag]( + val partitionsRDD: RDD[(PartitionID, EdgePartition[ED, VD])]) extends RDD[Edge[ED]](partitionsRDD.context, List(new OneToOneDependency(partitionsRDD))) { partitionsRDD.setName("EdgeRDD") @@ -45,8 +48,12 @@ class EdgeRDD[@specialized ED: ClassTag]( partitionsRDD.partitioner.orElse(Some(Partitioner.defaultPartitioner(partitionsRDD))) override def compute(part: Partition, context: TaskContext): Iterator[Edge[ED]] = { - val p = firstParent[(PartitionID, EdgePartition[ED])].iterator(part, context) - p.next._2.iterator.map(_.copy()) + val p = firstParent[(PartitionID, EdgePartition[ED, VD])].iterator(part, context) + if (p.hasNext) { + p.next._2.iterator.map(_.copy()) + } else { + Iterator.empty + } } override def collect(): Array[Edge[ED]] = this.map(_.copy()).collect() @@ -61,11 +68,15 @@ class EdgeRDD[@specialized ED: ClassTag]( this } - private[graphx] def mapEdgePartitions[ED2: ClassTag]( - f: (PartitionID, EdgePartition[ED]) => EdgePartition[ED2]): EdgeRDD[ED2] = { - new EdgeRDD[ED2](partitionsRDD.mapPartitions({ iter => - val (pid, ep) = iter.next() - Iterator(Tuple2(pid, f(pid, ep))) + private[graphx] def mapEdgePartitions[ED2: ClassTag, VD2: ClassTag]( + f: (PartitionID, EdgePartition[ED, VD]) => EdgePartition[ED2, VD2]): EdgeRDD[ED2, VD2] = { + new EdgeRDD[ED2, VD2](partitionsRDD.mapPartitions({ iter => + if (iter.hasNext) { + val (pid, ep) = iter.next() + Iterator(Tuple2(pid, f(pid, ep))) + } else { + Iterator.empty + } }, preservesPartitioning = true)) } @@ -76,7 +87,7 @@ class EdgeRDD[@specialized ED: ClassTag]( * @param f the function from an edge to a new edge value * @return a new EdgeRDD containing the new edge values */ - def mapValues[ED2: ClassTag](f: Edge[ED] => ED2): EdgeRDD[ED2] = + def mapValues[ED2: ClassTag](f: Edge[ED] => ED2): EdgeRDD[ED2, VD] = mapEdgePartitions((pid, part) => part.map(f)) /** @@ -84,7 +95,14 @@ class EdgeRDD[@specialized ED: ClassTag]( * * @return a new EdgeRDD containing all the edges reversed */ - def reverse: EdgeRDD[ED] = mapEdgePartitions((pid, part) => part.reverse) + def reverse: EdgeRDD[ED, VD] = mapEdgePartitions((pid, part) => part.reverse) + + /** Removes all edges but those matching `epred` and where both vertices match `vpred`. */ + def filter( + epred: EdgeTriplet[VD, ED] => Boolean, + vpred: (VertexId, VD) => Boolean): EdgeRDD[ED, VD] = { + mapEdgePartitions((pid, part) => part.filter(epred, vpred)) + } /** * Inner joins this EdgeRDD with another EdgeRDD, assuming both are partitioned using the same @@ -96,19 +114,15 @@ class EdgeRDD[@specialized ED: ClassTag]( * with values supplied by `f` */ def innerJoin[ED2: ClassTag, ED3: ClassTag] - (other: EdgeRDD[ED2]) - (f: (VertexId, VertexId, ED, ED2) => ED3): EdgeRDD[ED3] = { + (other: EdgeRDD[ED2, _]) + (f: (VertexId, VertexId, ED, ED2) => ED3): EdgeRDD[ED3, VD] = { val ed2Tag = classTag[ED2] val ed3Tag = classTag[ED3] - new EdgeRDD[ED3](partitionsRDD.zipPartitions(other.partitionsRDD, true) { + new EdgeRDD[ED3, VD](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))) }) } - - private[graphx] def collectVertexIds(): RDD[VertexId] = { - partitionsRDD.flatMap { case (_, p) => Array.concat(p.srcIds, p.dstIds) } - } } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/EdgeTriplet.scala b/graphx/src/main/scala/org/apache/spark/graphx/EdgeTriplet.scala index dfc6a801587d..9d473d5ebda4 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/EdgeTriplet.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeTriplet.scala @@ -63,4 +63,6 @@ class EdgeTriplet[VD, ED] extends Edge[ED] { if (srcId == vid) srcAttr else { assert(dstId == vid); dstAttr } override def toString = ((srcId, srcAttr), (dstId, dstAttr), attr).toString() + + def toTuple: ((VertexId, VD), (VertexId, VD), ED) = ((srcId, srcAttr), (dstId, dstAttr), attr) } 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 50395868902d..dc5dac4fdad5 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala @@ -59,7 +59,7 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] protected () extends Serializab * along with their vertex data. * */ - @transient val edges: EdgeRDD[ED] + @transient val edges: EdgeRDD[ED, VD] /** * An RDD containing the edge triplets, which are edges along with the vertex data associated with diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala index dd380d8c182c..d295d0127ac7 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala @@ -19,10 +19,11 @@ package org.apache.spark.graphx import com.esotericsoftware.kryo.Kryo -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 +import org.apache.spark.util.collection.BitSet + +import org.apache.spark.graphx.impl._ /** * Registers GraphX classes with Kryo for improved performance. @@ -33,8 +34,9 @@ class GraphKryoRegistrator extends KryoRegistrator { kryo.register(classOf[Edge[Object]]) kryo.register(classOf[MessageToPartition[Object]]) kryo.register(classOf[VertexBroadcastMsg[Object]]) + kryo.register(classOf[RoutingTableMessage]) kryo.register(classOf[(VertexId, Object)]) - kryo.register(classOf[EdgePartition[Object]]) + kryo.register(classOf[EdgePartition[Object, Object]]) kryo.register(classOf[BitSet]) kryo.register(classOf[VertexIdToIndexMap]) kryo.register(classOf[VertexAttributeBlock[Object]]) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala index 18858466db27..389490c13984 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala @@ -47,8 +47,7 @@ object GraphLoader extends Logging { * @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 + * @param minEdgePartitions the number of partitions for the edge RDD */ def edgeListFile( sc: SparkContext, @@ -60,8 +59,9 @@ object GraphLoader extends Logging { val startTime = System.currentTimeMillis // Parse the edge data table directly into edge partitions - val edges = sc.textFile(path, minEdgePartitions).mapPartitionsWithIndex { (pid, iter) => - val builder = new EdgePartitionBuilder[Int] + val lines = sc.textFile(path, minEdgePartitions).coalesce(minEdgePartitions) + val edges = lines.mapPartitionsWithIndex { (pid, iter) => + val builder = new EdgePartitionBuilder[Int, Int] iter.foreach { line => if (!line.isEmpty && line(0) != '#') { val lineArray = line.split("\\s+") @@ -78,7 +78,7 @@ object GraphLoader extends Logging { } } Iterator((pid, builder.toEdgePartition)) - }.cache() + }.cache().setName("GraphLoader.edgeListFile - edges (%s)".format(path)) edges.count() logInfo("It took %d ms to load the edges".format(System.currentTimeMillis - startTime)) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala index 4997fbc3cbcd..edd5b79da152 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala @@ -18,11 +18,13 @@ package org.apache.spark.graphx import scala.reflect.ClassTag -import org.apache.spark.SparkContext._ +import scala.util.Random + import org.apache.spark.SparkException -import org.apache.spark.graphx.lib._ +import org.apache.spark.SparkContext._ import org.apache.spark.rdd.RDD -import scala.util.Random + +import org.apache.spark.graphx.lib._ /** * Contains additional functionality for [[Graph]]. All operations are expressed in terms of the @@ -43,19 +45,22 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Seriali * The in-degree of each vertex in the graph. * @note Vertices with no in-edges are not returned in the resulting RDD. */ - @transient lazy val inDegrees: VertexRDD[Int] = degreesRDD(EdgeDirection.In) + @transient lazy val inDegrees: VertexRDD[Int] = + degreesRDD(EdgeDirection.In).setName("GraphOps.inDegrees") /** * The out-degree of each vertex in the graph. * @note Vertices with no out-edges are not returned in the resulting RDD. */ - @transient lazy val outDegrees: VertexRDD[Int] = degreesRDD(EdgeDirection.Out) + @transient lazy val outDegrees: VertexRDD[Int] = + degreesRDD(EdgeDirection.Out).setName("GraphOps.outDegrees") /** * The degree of each vertex in the graph. * @note Vertices with no edges are not returned in the resulting RDD. */ - @transient lazy val degrees: VertexRDD[Int] = degreesRDD(EdgeDirection.Either) + @transient lazy val degrees: VertexRDD[Int] = + degreesRDD(EdgeDirection.Either).setName("GraphOps.degrees") /** * Computes the neighboring vertex degrees. 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 ac07a594a12e..4572eab2875b 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala @@ -18,6 +18,7 @@ package org.apache.spark.graphx import scala.reflect.ClassTag +import org.apache.spark.Logging /** @@ -52,7 +53,7 @@ import scala.reflect.ClassTag * }}} * */ -object Pregel { +object Pregel extends Logging { /** * Execute a Pregel-like iterative vertex-parallel abstraction. The @@ -142,6 +143,9 @@ object Pregel { // 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() + + logInfo("Pregel finished iteration " + i) + // Unpersist the RDDs hidden by newly-materialized RDDs oldMessages.unpersist(blocking=false) newVerts.unpersist(blocking=false) 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 f0fc605c8857..8c62897037b6 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala @@ -24,8 +24,11 @@ import org.apache.spark.SparkContext._ import org.apache.spark.rdd._ import org.apache.spark.storage.StorageLevel -import org.apache.spark.graphx.impl.MsgRDDFunctions -import org.apache.spark.graphx.impl.VertexPartition +import org.apache.spark.graphx.impl.RoutingTablePartition +import org.apache.spark.graphx.impl.ShippableVertexPartition +import org.apache.spark.graphx.impl.VertexAttributeBlock +import org.apache.spark.graphx.impl.RoutingTableMessageRDDFunctions._ +import org.apache.spark.graphx.impl.VertexRDDFunctions._ /** * Extends `RDD[(VertexId, VD)]` by ensuring that there is only one entry for each vertex and by @@ -33,6 +36,9 @@ import org.apache.spark.graphx.impl.VertexPartition * joined efficiently. All operations except [[reindex]] preserve the index. To construct a * `VertexRDD`, use the [[org.apache.spark.graphx.VertexRDD$ VertexRDD object]]. * + * Additionally, stores routing information to enable joining the vertex attributes with an + * [[EdgeRDD]]. + * * @example Construct a `VertexRDD` from a plain RDD: * {{{ * // Construct an initial vertex set @@ -50,13 +56,11 @@ import org.apache.spark.graphx.impl.VertexPartition * @tparam VD the vertex attribute associated with each vertex in the set. */ class VertexRDD[@specialized VD: ClassTag]( - val partitionsRDD: RDD[VertexPartition[VD]]) + val partitionsRDD: RDD[ShippableVertexPartition[VD]]) extends RDD[(VertexId, VD)](partitionsRDD.context, List(new OneToOneDependency(partitionsRDD))) { require(partitionsRDD.partitioner.isDefined) - partitionsRDD.setName("VertexRDD") - /** * Construct a new VertexRDD that is indexed by only the visible vertices. The resulting * VertexRDD will be based on a different index and can no longer be quickly joined with this @@ -71,6 +75,16 @@ class VertexRDD[@specialized VD: ClassTag]( override protected def getPreferredLocations(s: Partition): Seq[String] = partitionsRDD.preferredLocations(s) + override def setName(_name: String): this.type = { + if (partitionsRDD.name != null) { + partitionsRDD.setName(partitionsRDD.name + ", " + _name) + } else { + partitionsRDD.setName(_name) + } + this + } + setName("VertexRDD") + override def persist(newLevel: StorageLevel): this.type = { partitionsRDD.persist(newLevel) this @@ -90,14 +104,14 @@ class VertexRDD[@specialized VD: ClassTag]( * Provides the `RDD[(VertexId, VD)]` equivalent output. */ override def compute(part: Partition, context: TaskContext): Iterator[(VertexId, VD)] = { - firstParent[VertexPartition[VD]].iterator(part, context).next.iterator + firstParent[ShippableVertexPartition[VD]].iterator(part, context).next.iterator } /** * Applies a function to each `VertexPartition` of this RDD and returns a new VertexRDD. */ private[graphx] def mapVertexPartitions[VD2: ClassTag]( - f: VertexPartition[VD] => VertexPartition[VD2]) + f: ShippableVertexPartition[VD] => ShippableVertexPartition[VD2]) : VertexRDD[VD2] = { val newPartitionsRDD = partitionsRDD.mapPartitions(_.map(f), preservesPartitioning = true) new VertexRDD(newPartitionsRDD) @@ -208,10 +222,8 @@ class VertexRDD[@specialized VD: ClassTag]( case _ => new VertexRDD[VD3]( partitionsRDD.zipPartitions( - other.partitionBy(this.partitioner.get), preservesPartitioning = true) - { (part, msgs) => - val vertexPartition: VertexPartition[VD] = part.next() - Iterator(vertexPartition.leftJoin(msgs)(f)) + other.copartitionWithVertices(this.partitioner.get), preservesPartitioning = true) { + (partIter, msgs) => partIter.map(_.leftJoin(msgs)(f)) } ) } @@ -254,10 +266,8 @@ class VertexRDD[@specialized VD: ClassTag]( case _ => new VertexRDD( partitionsRDD.zipPartitions( - other.partitionBy(this.partitioner.get), preservesPartitioning = true) - { (part, msgs) => - val vertexPartition: VertexPartition[VD] = part.next() - Iterator(vertexPartition.innerJoin(msgs)(f)) + other.copartitionWithVertices(this.partitioner.get), preservesPartitioning = true) { + (partIter, msgs) => partIter.map(_.innerJoin(msgs)(f)) } ) } @@ -276,14 +286,31 @@ class VertexRDD[@specialized VD: ClassTag]( */ def aggregateUsingIndex[VD2: ClassTag]( messages: RDD[(VertexId, VD2)], reduceFunc: (VD2, VD2) => VD2): VertexRDD[VD2] = { - val shuffled = MsgRDDFunctions.partitionForAggregation(messages, this.partitioner.get) + val shuffled = messages.copartitionWithVertices(this.partitioner.get) val parts = partitionsRDD.zipPartitions(shuffled, true) { (thisIter, msgIter) => - val vertexPartition: VertexPartition[VD] = thisIter.next() - Iterator(vertexPartition.aggregateUsingIndex(msgIter, reduceFunc)) + thisIter.map(_.aggregateUsingIndex(msgIter, reduceFunc)) } new VertexRDD[VD2](parts) } + /** + * Returns a new `VertexRDD` reflecting a reversal of all edge directions in the corresponding + * [[EdgeRDD]]. + */ + def reverseRoutingTables(): VertexRDD[VD] = + this.mapVertexPartitions(vPart => vPart.withRoutingTable(vPart.routingTable.reverse)) + + /** Generates an RDD of vertex attributes suitable for shipping to the edge partitions. */ + private[graphx] def shipVertexAttributes( + shipSrc: Boolean, shipDst: Boolean): RDD[(PartitionID, VertexAttributeBlock[VD])] = { + partitionsRDD.mapPartitions(_.flatMap(_.shipVertexAttributes(shipSrc, shipDst))) + } + + /** Generates an RDD of vertex IDs suitable for shipping to the edge partitions. */ + private[graphx] def shipVertexIds(): RDD[(PartitionID, Array[VertexId])] = { + partitionsRDD.mapPartitions(_.flatMap(_.shipVertexIds())) + } + } // end of VertexRDD @@ -293,52 +320,101 @@ class VertexRDD[@specialized VD: ClassTag]( object VertexRDD { /** - * Construct a `VertexRDD` from an RDD of vertex-attribute pairs. - * Duplicate entries are removed arbitrarily. + * Constructs a standalone `VertexRDD` (one that is not set up for efficient joins with an + * [[EdgeRDD]]) from an RDD of vertex-attribute pairs. Duplicate entries are removed arbitrarily. * * @tparam VD the vertex attribute type * - * @param rdd the collection of vertex-attribute pairs + * @param vertices the collection of vertex-attribute pairs */ - 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)) + def apply[VD: ClassTag](vertices: RDD[(VertexId, VD)]): VertexRDD[VD] = { + val vPartitioned: RDD[(VertexId, VD)] = vertices.partitioner match { + case Some(p) => vertices + case None => vertices.copartitionWithVertices(new HashPartitioner(vertices.partitions.size)) } - val vertexPartitions = partitioned.mapPartitions( - iter => Iterator(VertexPartition(iter)), + val vertexPartitions = vPartitioned.mapPartitions( + iter => Iterator(ShippableVertexPartition(iter)), preservesPartitioning = true) new VertexRDD(vertexPartitions) } /** - * Constructs a `VertexRDD` from an RDD of vertex-attribute pairs, merging duplicates using - * `mergeFunc`. + * Constructs a `VertexRDD` from an RDD of vertex-attribute pairs. Duplicate vertex entries are + * removed arbitrarily. The resulting `VertexRDD` will be joinable with `edges`, and any missing + * vertices referred to by `edges` will be created with the attribute `defaultVal`. * * @tparam VD the vertex attribute type * - * @param rdd the collection of vertex-attribute pairs - * @param mergeFunc the associative, commutative merge function. + * @param vertices the collection of vertex-attribute pairs + * @param edges the [[EdgeRDD]] that these vertices may be joined with + * @param defaultVal the vertex attribute to use when creating missing vertices */ - def apply[VD: ClassTag](rdd: RDD[(VertexId, VD)], mergeFunc: (VD, VD) => VD): VertexRDD[VD] = { - val partitioned: RDD[(VertexId, VD)] = rdd.partitioner match { - case Some(p) => rdd - case None => rdd.partitionBy(new HashPartitioner(rdd.partitions.size)) + def apply[VD: ClassTag]( + vertices: RDD[(VertexId, VD)], edges: EdgeRDD[_, _], defaultVal: VD): VertexRDD[VD] = { + VertexRDD(vertices, edges, defaultVal, (a, b) => b) + } + + /** + * Constructs a `VertexRDD` from an RDD of vertex-attribute pairs. Duplicate vertex entries are + * merged using `mergeFunc`. The resulting `VertexRDD` will be joinable with `edges`, and any + * missing vertices referred to by `edges` will be created with the attribute `defaultVal`. + * + * @tparam VD the vertex attribute type + * + * @param vertices the collection of vertex-attribute pairs + * @param edges the [[EdgeRDD]] that these vertices may be joined with + * @param defaultVal the vertex attribute to use when creating missing vertices + * @param mergeFunc the commutative, associative duplicate vertex attribute merge function + */ + def apply[VD: ClassTag]( + vertices: RDD[(VertexId, VD)], edges: EdgeRDD[_, _], defaultVal: VD, mergeFunc: (VD, VD) => VD + ): VertexRDD[VD] = { + val vPartitioned: RDD[(VertexId, VD)] = vertices.partitioner match { + case Some(p) => vertices + case None => vertices.copartitionWithVertices(new HashPartitioner(vertices.partitions.size)) + } + val routingTables = createRoutingTables(edges, vPartitioned.partitioner.get) + val vertexPartitions = vPartitioned.zipPartitions(routingTables, preservesPartitioning = true) { + (vertexIter, routingTableIter) => + val routingTable = + if (routingTableIter.hasNext) routingTableIter.next() else RoutingTablePartition.empty + Iterator(ShippableVertexPartition(vertexIter, routingTable, defaultVal)) } - val vertexPartitions = partitioned.mapPartitions( - iter => Iterator(VertexPartition(iter)), - preservesPartitioning = true) new VertexRDD(vertexPartitions) } /** - * Constructs a VertexRDD from the vertex IDs in `vids`, taking attributes from `rdd` and using - * `defaultVal` otherwise. + * Constructs a `VertexRDD` containing all vertices referred to in `edges`. The vertices will be + * created with the attribute `defaultVal`. The resulting `VertexRDD` will be joinable with + * `edges`. + * + * @tparam VD the vertex attribute type + * + * @param edges the [[EdgeRDD]] referring to the vertices to create + * @param numPartitions the desired number of partitions for the resulting `VertexRDD` + * @param defaultVal the vertex attribute to use when creating missing vertices */ - 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) => - value.getOrElse(default) - } + def fromEdges[VD: ClassTag]( + edges: EdgeRDD[_, _], numPartitions: Int, defaultVal: VD): VertexRDD[VD] = { + val routingTables = createRoutingTables(edges, new HashPartitioner(numPartitions)) + val vertexPartitions = routingTables.mapPartitions({ routingTableIter => + val routingTable = + if (routingTableIter.hasNext) routingTableIter.next() else RoutingTablePartition.empty + Iterator(ShippableVertexPartition(Iterator.empty, routingTable, defaultVal)) + }, preservesPartitioning = true) + new VertexRDD(vertexPartitions) + } + + private def createRoutingTables( + edges: EdgeRDD[_, _], vertexPartitioner: Partitioner): RDD[RoutingTablePartition] = { + // Determine which vertices each edge partition needs by creating a mapping from vid to pid. + val vid2pid = edges.partitionsRDD.mapPartitions(_.flatMap( + Function.tupled(RoutingTablePartition.edgePartitionToMsgs))) + .setName("VertexRDD.createRoutingTables - vid2pid (aggregation)") + + val numEdgePartitions = edges.partitions.size + vid2pid.copartitionWithVertices(vertexPartitioner).mapPartitions( + iter => Iterator(RoutingTablePartition.fromMsgs(numEdgePartitions, iter)), + preservesPartitioning = true) } } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala index b7c472e905a9..871e81f8d245 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala @@ -17,39 +17,86 @@ package org.apache.spark.graphx.impl -import scala.reflect.ClassTag +import scala.reflect.{classTag, ClassTag} import org.apache.spark.graphx._ import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap /** - * A collection of edges stored in 3 large columnar arrays (src, dst, attribute). The arrays are - * clustered by src. + * A collection of edges stored in columnar format, along with any vertex attributes referenced. The + * edges are stored in 3 large columnar arrays (src, dst, attribute). The arrays are clustered by + * src. There is an optional active vertex set for filtering computation on the edges. + * + * @tparam ED the edge attribute type + * @tparam VD the vertex attribute type * * @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. + * @param vertices a map from referenced vertex ids to their corresponding attributes. Must + * contain all vertex ids from `srcIds` and `dstIds`, though not necessarily valid attributes for + * those vertex ids. The mask is not used. + * @param activeSet an optional active vertex set for filtering computation on the edges */ private[graphx] -class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED: ClassTag]( +class EdgePartition[ + @specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED: ClassTag, VD: ClassTag]( @transient val srcIds: Array[VertexId], @transient val dstIds: Array[VertexId], @transient val data: Array[ED], - @transient val index: PrimitiveKeyOpenHashMap[VertexId, Int]) extends Serializable { + @transient val index: PrimitiveKeyOpenHashMap[VertexId, Int], + @transient val vertices: VertexPartition[VD], + @transient val activeSet: Option[VertexSet] = None + ) extends Serializable { + + /** Return a new `EdgePartition` with the specified edge data. */ + def withData[ED2: ClassTag](data_ : Array[ED2]): EdgePartition[ED2, VD] = { + new EdgePartition(srcIds, dstIds, data_, index, vertices, activeSet) + } + + /** Return a new `EdgePartition` with the specified vertex partition. */ + def withVertices[VD2: ClassTag]( + vertices_ : VertexPartition[VD2]): EdgePartition[ED, VD2] = { + new EdgePartition(srcIds, dstIds, data, index, vertices_, activeSet) + } + + /** Return a new `EdgePartition` with the specified active set, provided as an iterator. */ + def withActiveSet(iter: Iterator[VertexId]): EdgePartition[ED, VD] = { + val newActiveSet = new VertexSet + iter.foreach(newActiveSet.add(_)) + new EdgePartition(srcIds, dstIds, data, index, vertices, Some(newActiveSet)) + } + + /** Return a new `EdgePartition` with the specified active set. */ + def withActiveSet(activeSet_ : Option[VertexSet]): EdgePartition[ED, VD] = { + new EdgePartition(srcIds, dstIds, data, index, vertices, activeSet_) + } + + /** Return a new `EdgePartition` with updates to vertex attributes specified in `iter`. */ + def updateVertices(iter: Iterator[(VertexId, VD)]): EdgePartition[ED, VD] = { + this.withVertices(vertices.innerJoinKeepLeft(iter)) + } + + /** Look up vid in activeSet, throwing an exception if it is None. */ + def isActive(vid: VertexId): Boolean = { + activeSet.get.contains(vid) + } + + /** The number of active vertices, if any exist. */ + def numActives: Option[Int] = activeSet.map(_.size) /** * Reverse all the edges in this partition. * * @return a new edge partition with all edges reversed. */ - def reverse: EdgePartition[ED] = { - val builder = new EdgePartitionBuilder(size) + def reverse: EdgePartition[ED, VD] = { + val builder = new EdgePartitionBuilder(size)(classTag[ED], classTag[VD]) for (e <- iterator) { builder.add(e.dstId, e.srcId, e.attr) } - builder.toEdgePartition + builder.toEdgePartition.withVertices(vertices).withActiveSet(activeSet) } /** @@ -64,7 +111,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: ClassTag](f: Edge[ED] => ED2): EdgePartition[ED2] = { + def map[ED2: ClassTag](f: Edge[ED] => ED2): EdgePartition[ED2, VD] = { val newData = new Array[ED2](data.size) val edge = new Edge[ED]() val size = data.size @@ -76,7 +123,7 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) newData(i) = f(edge) i += 1 } - new EdgePartition(srcIds, dstIds, newData, index) + this.withData(newData) } /** @@ -91,7 +138,7 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) * @tparam ED2 the type of the new attribute * @return a new edge partition with the attribute values replaced */ - def map[ED2: ClassTag](iter: Iterator[ED2]): EdgePartition[ED2] = { + def map[ED2: ClassTag](iter: Iterator[ED2]): EdgePartition[ED2, VD] = { // Faster than iter.toArray, because the expected size is known. val newData = new Array[ED2](data.size) var i = 0 @@ -100,7 +147,23 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) i += 1 } assert(newData.size == i) - new EdgePartition(srcIds, dstIds, newData, index) + this.withData(newData) + } + + /** + * Construct a new edge partition containing only the edges matching `epred` and where both + * vertices match `vpred`. + */ + def filter( + epred: EdgeTriplet[VD, ED] => Boolean, + vpred: (VertexId, VD) => Boolean): EdgePartition[ED, VD] = { + val filtered = tripletIterator().filter(et => + vpred(et.srcId, et.srcAttr) && vpred(et.dstId, et.dstAttr) && epred(et)) + val builder = new EdgePartitionBuilder[ED, VD] + for (e <- filtered) { + builder.add(e.srcId, e.dstId, e.attr) + } + builder.toEdgePartition.withVertices(vertices).withActiveSet(activeSet) } /** @@ -119,8 +182,8 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) * @param merge a commutative associative merge operation * @return a new edge partition without duplicate edges */ - def groupEdges(merge: (ED, ED) => ED): EdgePartition[ED] = { - val builder = new EdgePartitionBuilder[ED] + def groupEdges(merge: (ED, ED) => ED): EdgePartition[ED, VD] = { + val builder = new EdgePartitionBuilder[ED, VD] var currSrcId: VertexId = null.asInstanceOf[VertexId] var currDstId: VertexId = null.asInstanceOf[VertexId] var currAttr: ED = null.asInstanceOf[ED] @@ -141,11 +204,11 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) if (size > 0) { builder.add(currSrcId, currDstId, currAttr) } - builder.toEdgePartition + builder.toEdgePartition.withVertices(vertices).withActiveSet(activeSet) } /** - * Apply `f` to all edges present in both `this` and `other` and return a new EdgePartition + * 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 @@ -155,9 +218,9 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) * once. */ def innerJoin[ED2: ClassTag, ED3: ClassTag] - (other: EdgePartition[ED2]) - (f: (VertexId, VertexId, ED, ED2) => ED3): EdgePartition[ED3] = { - val builder = new EdgePartitionBuilder[ED3] + (other: EdgePartition[ED2, _]) + (f: (VertexId, VertexId, ED, ED2) => ED3): EdgePartition[ED3, VD] = { + val builder = new EdgePartitionBuilder[ED3, VD] var i = 0 var j = 0 // For i = index of each edge in `this`... @@ -175,7 +238,7 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) } i += 1 } - builder.toEdgePartition + builder.toEdgePartition.withVertices(vertices).withActiveSet(activeSet) } /** @@ -183,7 +246,7 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) * * @return size of the partition */ - def size: Int = srcIds.size + val size: Int = srcIds.size /** The number of unique source vertices in the partition. */ def indexSize: Int = index.size @@ -211,10 +274,35 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) } } + /** + * Get an iterator over the edge triplets in this partition. + * + * It is safe to keep references to the objects from this iterator. + */ + def tripletIterator( + includeSrc: Boolean = true, includeDst: Boolean = true): Iterator[EdgeTriplet[VD, ED]] = { + new EdgeTripletIterator(this, includeSrc, includeDst) + } + + /** + * Upgrade the given edge iterator into a triplet iterator. + * + * Be careful not to keep references to the objects from this iterator. To improve GC performance + * the same object is re-used in `next()`. + */ + def upgradeIterator( + edgeIter: Iterator[Edge[ED]], includeSrc: Boolean = true, includeDst: Boolean = true) + : Iterator[EdgeTriplet[VD, ED]] = { + new ReusingEdgeTripletIterator(edgeIter, this, includeSrc, includeDst) + } + /** * 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. + * + * Be careful not to keep references to the objects from this iterator. To improve GC performance + * the same object is re-used in `next()`. */ def indexIterator(srcIdPred: VertexId => Boolean): Iterator[Edge[ED]] = index.iterator.filter(kv => srcIdPred(kv._1)).flatMap(Function.tupled(clusterIterator)) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala index 63ccccb056b4..ecb49bef42e4 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala @@ -20,12 +20,14 @@ package org.apache.spark.graphx.impl import scala.reflect.ClassTag import scala.util.Sorting +import org.apache.spark.util.collection.{BitSet, OpenHashSet, PrimitiveVector} + import org.apache.spark.graphx._ import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap -import org.apache.spark.util.collection.PrimitiveVector private[graphx] -class EdgePartitionBuilder[@specialized(Long, Int, Double) ED: ClassTag](size: Int = 64) { +class EdgePartitionBuilder[@specialized(Long, Int, Double) ED: ClassTag, VD: ClassTag]( + size: Int = 64) { var edges = new PrimitiveVector[Edge[ED]](size) /** Add a new edge to the partition. */ @@ -33,7 +35,7 @@ class EdgePartitionBuilder[@specialized(Long, Int, Double) ED: ClassTag](size: I edges += Edge(src, dst, d) } - def toEdgePartition: EdgePartition[ED] = { + def toEdgePartition: EdgePartition[ED, VD] = { val edgeArray = edges.trim().array Sorting.quickSort(edgeArray)(Edge.lexicographicOrdering) val srcIds = new Array[VertexId](edgeArray.size) @@ -57,6 +59,14 @@ class EdgePartitionBuilder[@specialized(Long, Int, Double) ED: ClassTag](size: I i += 1 } } - new EdgePartition(srcIds, dstIds, data, index) + + // Create and populate a VertexPartition with vids from the edges, but no attributes + val vidsIter = srcIds.iterator ++ dstIds.iterator + val vertexIds = new OpenHashSet[VertexId] + vidsIter.foreach(vid => vertexIds.add(vid)) + val vertices = new VertexPartition( + vertexIds, new Array[VD](vertexIds.capacity), vertexIds.getBitSet) + + new EdgePartition(srcIds, dstIds, data, index, vertices) } } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala index 220a89d73d71..ebb0b9418d65 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala @@ -23,32 +23,62 @@ import org.apache.spark.graphx._ import org.apache.spark.graphx.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. + * The Iterator type returned when constructing edge triplets. This could be an anonymous class in + * EdgePartition.tripletIterator, but we name it here explicitly so it is easier to debug / profile. */ private[impl] class EdgeTripletIterator[VD: ClassTag, ED: ClassTag]( - val vidToIndex: VertexIdToIndexMap, - val vertexArray: Array[VD], - val edgePartition: EdgePartition[ED]) + val edgePartition: EdgePartition[ED, VD], + val includeSrc: Boolean, + val includeDst: Boolean) extends Iterator[EdgeTriplet[VD, ED]] { // Current position in the array. private var pos = 0 - private val vmap = new PrimitiveKeyOpenHashMap[VertexId, VD](vidToIndex, vertexArray) - override def hasNext: Boolean = pos < edgePartition.size override def next() = { val triplet = new EdgeTriplet[VD, ED] triplet.srcId = edgePartition.srcIds(pos) - triplet.srcAttr = vmap(triplet.srcId) + if (includeSrc) { + triplet.srcAttr = edgePartition.vertices(triplet.srcId) + } triplet.dstId = edgePartition.dstIds(pos) - triplet.dstAttr = vmap(triplet.dstId) + if (includeDst) { + triplet.dstAttr = edgePartition.vertices(triplet.dstId) + } triplet.attr = edgePartition.data(pos) pos += 1 triplet } } + +/** + * An Iterator type for internal use that reuses EdgeTriplet objects. This could be an anonymous + * class in EdgePartition.upgradeIterator, but we name it here explicitly so it is easier to debug / + * profile. + */ +private[impl] +class ReusingEdgeTripletIterator[VD: ClassTag, ED: ClassTag]( + val edgeIter: Iterator[Edge[ED]], + val edgePartition: EdgePartition[ED, VD], + val includeSrc: Boolean, + val includeDst: Boolean) + extends Iterator[EdgeTriplet[VD, ED]] { + + private val triplet = new EdgeTriplet[VD, ED] + + override def hasNext = edgeIter.hasNext + + override def next() = { + triplet.set(edgeIter.next()) + if (includeSrc) { + triplet.srcAttr = edgePartition.vertices(triplet.srcId) + } + if (includeDst) { + triplet.dstAttr = edgePartition.vertices(triplet.dstId) + } + triplet + } +} 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 9eabccdee48d..2f2d0e03fd7b 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 @@ -19,54 +19,45 @@ 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.HashPartitioner import org.apache.spark.SparkContext._ +import org.apache.spark.rdd.{RDD, ShuffledRDD} +import org.apache.spark.storage.StorageLevel + 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 /** - * A graph that supports computation on graphs. + * An implementation of [[org.apache.spark.graphx.Graph]] to support computation on graphs. * - * Graphs are represented using two classes of data: vertex-partitioned and - * edge-partitioned. `vertices` contains vertex attributes, which are vertex-partitioned. `edges` - * 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. `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. + * Graphs are represented using two RDDs: `vertices`, which contains vertex attributes and the + * routing information for shipping vertex attributes to edge partitions, and + * `replicatedVertexView`, which contains edges and the vertex attributes mentioned by each edge. */ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( @transient val vertices: VertexRDD[VD], - @transient val edges: EdgeRDD[ED], - @transient val routingTable: RoutingTable, - @transient val replicatedVertexView: ReplicatedVertexView[VD]) + @transient val replicatedVertexView: ReplicatedVertexView[VD, ED]) extends Graph[VD, ED] with Serializable { /** Default constructor is provided to support serialization */ - protected def this() = this(null, null, null, null) + protected def this() = this(null, null) + + @transient override val edges: EdgeRDD[ED, VD] = replicatedVertexView.edges /** 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] - val edTag = classTag[ED] - 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) - } + @transient override lazy val triplets: RDD[EdgeTriplet[VD, ED]] = { + replicatedVertexView.upgrade(vertices, true, true) + replicatedVertexView.edges.partitionsRDD.mapPartitions(_.flatMap { + case (pid, part) => part.tripletIterator() + }) } override def persist(newLevel: StorageLevel): Graph[VD, ED] = { vertices.persist(newLevel) - edges.persist(newLevel) + replicatedVertexView.edges.persist(newLevel) this } @@ -74,14 +65,15 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( override def unpersistVertices(blocking: Boolean = true): Graph[VD, ED] = { vertices.unpersist(blocking) - replicatedVertexView.unpersist(blocking) + // TODO: unpersist the replicated vertices in `replicatedVertexView` but leave the edges alone this } override def partitionBy(partitionStrategy: PartitionStrategy): Graph[VD, ED] = { - val numPartitions = edges.partitions.size + val numPartitions = replicatedVertexView.edges.partitions.size val edTag = classTag[ED] - val newEdges = new EdgeRDD(edges.map { e => + val vdTag = classTag[VD] + val newEdges = new EdgeRDD(replicatedVertexView.edges.map { e => val part: PartitionID = partitionStrategy.getPartition(e.srcId, e.dstId, numPartitions) // Should we be using 3-tuple or an optimized class @@ -89,105 +81,79 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( } .partitionBy(new HashPartitioner(numPartitions)) .mapPartitionsWithIndex( { (pid, iter) => - val builder = new EdgePartitionBuilder[ED]()(edTag) + val builder = new EdgePartitionBuilder[ED, VD]()(edTag, vdTag) 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()) - GraphImpl(vertices, newEdges) + }, preservesPartitioning = true)) + GraphImpl.fromExistingRDDs(vertices, newEdges) } override def reverse: Graph[VD, ED] = { - val newETable = edges.mapEdgePartitions((pid, part) => part.reverse) - GraphImpl(vertices, newETable) + new GraphImpl(vertices.reverseRoutingTables(), replicatedVertexView.reverse()) } override def mapVertices[VD2: ClassTag](f: (VertexId, VD) => VD2): Graph[VD2, ED] = { if (classTag[VD] equals classTag[VD2]) { + vertices.cache() // The map preserves type, so we can use incremental replication val newVerts = vertices.mapVertexPartitions(_.map(f)).cache() val changedVerts = vertices.asInstanceOf[VertexRDD[VD2]].diff(newVerts) - val newReplicatedVertexView = new ReplicatedVertexView[VD2]( - changedVerts, edges, routingTable, - Some(replicatedVertexView.asInstanceOf[ReplicatedVertexView[VD2]])) - new GraphImpl(newVerts, edges, routingTable, newReplicatedVertexView) + val newReplicatedVertexView = replicatedVertexView.asInstanceOf[ReplicatedVertexView[VD2, ED]] + .updateVertices(changedVerts) + new GraphImpl(newVerts, newReplicatedVertexView) } else { // The map does not preserve type, so we must re-replicate all vertices - GraphImpl(vertices.mapVertexPartitions(_.map(f)), edges, routingTable) + GraphImpl(vertices.mapVertexPartitions(_.map(f)), replicatedVertexView.edges) } } override def mapEdges[ED2: ClassTag]( 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) + val newEdges = replicatedVertexView.edges + .mapEdgePartitions((pid, part) => part.map(f(pid, part.iterator))) + new GraphImpl(vertices, replicatedVertexView.withEdges(newEdges)) } override def mapTriplets[ED2: ClassTag]( f: (PartitionID, Iterator[EdgeTriplet[VD, ED]]) => Iterator[ED2]): Graph[VD, ED2] = { - val newEdgePartitions = - 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) - val et = new EdgeTriplet[VD, ED] - val inputIterator = edgePartition.iterator.map { e => - et.set(e) - et.srcAttr = vPart(e.srcId) - et.dstAttr = vPart(e.dstId) - et - } - // 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) + vertices.cache() + val mapUsesSrcAttr = accessesVertexAttr(f, "srcAttr") + val mapUsesDstAttr = accessesVertexAttr(f, "dstAttr") + replicatedVertexView.upgrade(vertices, mapUsesSrcAttr, mapUsesDstAttr) + val newEdges = replicatedVertexView.edges.mapEdgePartitions { (pid, part) => + part.map(f(pid, part.tripletIterator(mapUsesSrcAttr, mapUsesDstAttr))) + } + new GraphImpl(vertices, replicatedVertexView.withEdges(newEdges)) } override def subgraph( epred: EdgeTriplet[VD, ED] => Boolean = x => true, vpred: (VertexId, VD) => Boolean = (a, b) => true): Graph[VD, ED] = { + vertices.cache() // Filter the vertices, reusing the partitioner and the index from this graph val newVerts = vertices.mapVertexPartitions(_.filter(vpred)) - - // Filter the edges - 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]()(edTag) - iter.foreach { et => builder.add(et.srcId, et.dstId, et.attr) } - val edgePartition = builder.toEdgePartition - Iterator((pid, edgePartition)) - }, preservesPartitioning = true)).cache() - - // 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, new RoutingTable(newEdges, newVerts), replicatedVertexView) - } // end of subgraph + // Filter the triplets. We must always upgrade the triplet view fully because vpred always runs + // on both src and dst vertices + replicatedVertexView.upgrade(vertices, true, true) + val newEdges = replicatedVertexView.edges.filter(epred, vpred) + new GraphImpl(newVerts, replicatedVertexView.withEdges(newEdges)) + } 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 } - // 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, routingTable, replicatedVertexView) + val newEdges = replicatedVertexView.edges.innerJoin(other.edges) { (src, dst, v, w) => v } + new GraphImpl(newVerts, replicatedVertexView.withEdges(newEdges)) } override def groupEdges(merge: (ED, ED) => ED): Graph[VD, ED] = { - ClosureCleaner.clean(merge) - val newETable = edges.mapEdgePartitions((pid, part) => part.groupEdges(merge)) - new GraphImpl(vertices, newETable, routingTable, replicatedVertexView) + val newEdges = replicatedVertexView.edges.mapEdgePartitions( + (pid, part) => part.groupEdges(merge)) + new GraphImpl(vertices, replicatedVertexView.withEdges(newEdges)) } // /////////////////////////////////////////////////////////////////////////////////////////////// @@ -199,68 +165,58 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( reduceFunc: (A, A) => A, activeSetOpt: Option[(VertexRDD[_], EdgeDirection)] = None): VertexRDD[A] = { - ClosureCleaner.clean(mapFunc) - ClosureCleaner.clean(reduceFunc) + vertices.cache() // For each vertex, replicate its attribute only to partitions where it is // in the relevant position in an edge. val mapUsesSrcAttr = accessesVertexAttr(mapFunc, "srcAttr") val mapUsesDstAttr = accessesVertexAttr(mapFunc, "dstAttr") - val vs = activeSetOpt match { + replicatedVertexView.upgrade(vertices, mapUsesSrcAttr, mapUsesDstAttr) + val view = activeSetOpt match { case Some((activeSet, _)) => - replicatedVertexView.get(mapUsesSrcAttr, mapUsesDstAttr, activeSet) + replicatedVertexView.withActiveSet(activeSet) case None => - replicatedVertexView.get(mapUsesSrcAttr, mapUsesDstAttr) + replicatedVertexView } val activeDirectionOpt = activeSetOpt.map(_._2) // Map and combine. - 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) - // 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(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.Either) => - // TODO: Because we only have a clustered index on the source vertex ID, we can't filter - // the index here. Instead we have to scan all edges and then do the filter. - edgePartition.iterator.filter(e => vPart.isActive(e.srcId) || vPart.isActive(e.dstId)) - case Some(EdgeDirection.Out) => - if (activeFraction < 0.8) { - edgePartition.indexIterator(srcVertexId => vPart.isActive(srcVertexId)) - } 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 - } - - // Scan edges and run the map function - val et = new EdgeTriplet[VD, ED] - val mapOutputs = edgeIter.flatMap { e => - et.set(e) - if (mapUsesSrcAttr) { - et.srcAttr = vPart(e.srcId) - } - if (mapUsesDstAttr) { - et.dstAttr = vPart(e.dstId) + val preAgg = view.edges.partitionsRDD.mapPartitions(_.flatMap { + case (pid, edgePartition) => + // Choose scan method + val activeFraction = edgePartition.numActives.getOrElse(0) / edgePartition.indexSize.toFloat + val edgeIter = activeDirectionOpt match { + case Some(EdgeDirection.Both) => + if (activeFraction < 0.8) { + edgePartition.indexIterator(srcVertexId => edgePartition.isActive(srcVertexId)) + .filter(e => edgePartition.isActive(e.dstId)) + } else { + edgePartition.iterator.filter(e => + edgePartition.isActive(e.srcId) && edgePartition.isActive(e.dstId)) + } + case Some(EdgeDirection.Either) => + // TODO: Because we only have a clustered index on the source vertex ID, we can't filter + // the index here. Instead we have to scan all edges and then do the filter. + edgePartition.iterator.filter(e => + edgePartition.isActive(e.srcId) || edgePartition.isActive(e.dstId)) + case Some(EdgeDirection.Out) => + if (activeFraction < 0.8) { + edgePartition.indexIterator(srcVertexId => edgePartition.isActive(srcVertexId)) + } else { + edgePartition.iterator.filter(e => edgePartition.isActive(e.srcId)) + } + case Some(EdgeDirection.In) => + edgePartition.iterator.filter(e => edgePartition.isActive(e.dstId)) + case _ => // None + edgePartition.iterator } - mapFunc(et) - } - // Note: This doesn't allow users to send messages to arbitrary vertices. - vPart.aggregateUsingIndex(mapOutputs, reduceFunc).iterator - } + + // Scan edges and run the map function + val mapOutputs = edgePartition.upgradeIterator(edgeIter, mapUsesSrcAttr, mapUsesDstAttr) + .flatMap(mapFunc(_)) + // Note: This doesn't allow users to send messages to arbitrary vertices. + edgePartition.vertices.aggregateUsingIndex(mapOutputs, reduceFunc).iterator + }).setName("GraphImpl.mapReduceTriplets - preAgg") // do the final reduction reusing the index map vertices.aggregateUsingIndex(preAgg, reduceFunc) @@ -268,20 +224,19 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( override def outerJoinVertices[U: ClassTag, VD2: ClassTag] (other: RDD[(VertexId, U)]) - (updateF: (VertexId, VD, Option[U]) => VD2): Graph[VD2, ED] = - { + (updateF: (VertexId, VD, Option[U]) => VD2): Graph[VD2, ED] = { if (classTag[VD] equals classTag[VD2]) { + vertices.cache() // updateF preserves type, so we can use incremental replication - val newVerts = vertices.leftJoin(other)(updateF) + val newVerts = vertices.leftJoin(other)(updateF).cache() val changedVerts = vertices.asInstanceOf[VertexRDD[VD2]].diff(newVerts) - val newReplicatedVertexView = new ReplicatedVertexView[VD2]( - changedVerts, edges, routingTable, - Some(replicatedVertexView.asInstanceOf[ReplicatedVertexView[VD2]])) - new GraphImpl(newVerts, edges, routingTable, newReplicatedVertexView) + val newReplicatedVertexView = replicatedVertexView.asInstanceOf[ReplicatedVertexView[VD2, ED]] + .updateVertices(changedVerts) + new GraphImpl(newVerts, newReplicatedVertexView) } else { // updateF does not preserve type, so we must re-replicate all vertices val newVerts = vertices.leftJoin(other)(updateF) - GraphImpl(newVerts, edges, routingTable) + GraphImpl(newVerts, replicatedVertexView.edges) } } @@ -298,73 +253,68 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( object GraphImpl { + /** Create a graph from edges, setting referenced vertices to `defaultVertexAttr`. */ def apply[VD: ClassTag, ED: ClassTag]( edges: RDD[Edge[ED]], - defaultVertexAttr: VD): GraphImpl[VD, ED] = - { + defaultVertexAttr: VD): GraphImpl[VD, ED] = { fromEdgeRDD(createEdgeRDD(edges), defaultVertexAttr) } + /** Create a graph from EdgePartitions, setting referenced vertices to `defaultVertexAttr`. */ def fromEdgePartitions[VD: ClassTag, ED: ClassTag]( - edgePartitions: RDD[(PartitionID, EdgePartition[ED])], + edgePartitions: RDD[(PartitionID, EdgePartition[ED, VD])], defaultVertexAttr: VD): GraphImpl[VD, ED] = { fromEdgeRDD(new EdgeRDD(edgePartitions), defaultVertexAttr) } + /** Create a graph from vertices and edges, setting missing vertices to `defaultVertexAttr`. */ def apply[VD: ClassTag, ED: ClassTag]( vertices: RDD[(VertexId, VD)], edges: RDD[Edge[ED]], - defaultVertexAttr: VD): GraphImpl[VD, ED] = - { - val edgeRDD = createEdgeRDD(edges).cache() - - // Get the set of all vids - val partitioner = Partitioner.defaultPartitioner(vertices) - val vPartitioned = vertices.partitionBy(partitioner) - val vidsFromEdges = collectVertexIdsFromEdges(edgeRDD, partitioner) - val vids = vPartitioned.zipPartitions(vidsFromEdges) { (vertexIter, vidsFromEdgesIter) => - vertexIter.map(_._1) ++ vidsFromEdgesIter.map(_._1) - } - - val vertexRDD = VertexRDD(vids, vPartitioned, defaultVertexAttr) - + defaultVertexAttr: VD): GraphImpl[VD, ED] = { + val edgeRDD = createEdgeRDD(edges)(classTag[ED], classTag[VD]).cache() + val vertexRDD = VertexRDD(vertices, edgeRDD, defaultVertexAttr) GraphImpl(vertexRDD, edgeRDD) } + /** Create a graph from a VertexRDD and an EdgeRDD with arbitrary replicated vertices. */ 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)) + edges: EdgeRDD[ED, _]): GraphImpl[VD, ED] = { + // Convert the vertex partitions in edges to the correct type + val newEdges = edges.mapEdgePartitions( + (pid, part) => part.withVertices(part.vertices.map( + (vid, attr) => null.asInstanceOf[VD]))) + GraphImpl.fromExistingRDDs(vertices, newEdges) } - def apply[VD: ClassTag, ED: ClassTag]( + /** + * Create a graph from a VertexRDD and an EdgeRDD with the same replicated vertex type as the + * vertices. + */ + def fromExistingRDDs[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)) + edges: EdgeRDD[ED, VD]): GraphImpl[VD, ED] = { + new GraphImpl(vertices, new ReplicatedVertexView(edges)) } /** - * Create the edge RDD, which is much more efficient for Java heap storage than the normal edges - * 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 - * edges in a partition. + * Create a graph from an EdgeRDD with the correct vertex type, setting missing vertices to + * `defaultVertexAttr`. The vertices will have the same number of partitions as the EdgeRDD. */ - private def createEdgeRDD[ED: ClassTag]( - edges: RDD[Edge[ED]]): EdgeRDD[ED] = { + private def fromEdgeRDD[VD: ClassTag, ED: ClassTag]( + edges: EdgeRDD[ED, VD], + defaultVertexAttr: VD): GraphImpl[VD, ED] = { + edges.cache() + val vertices = VertexRDD.fromEdges(edges, edges.partitions.size, defaultVertexAttr) + fromExistingRDDs(vertices, edges) + } + + /** Create an EdgeRDD from a set of edges. */ + private def createEdgeRDD[ED: ClassTag, VD: ClassTag]( + edges: RDD[Edge[ED]]): EdgeRDD[ED, VD] = { val edgePartitions = edges.mapPartitionsWithIndex { (pid, iter) => - val builder = new EdgePartitionBuilder[ED] + val builder = new EdgePartitionBuilder[ED, VD] iter.foreach { e => builder.add(e.srcId, e.dstId, e.attr) } @@ -373,24 +323,4 @@ object GraphImpl { new EdgeRDD(edgePartitions) } - private def fromEdgeRDD[VD: ClassTag, ED: ClassTag]( - edges: EdgeRDD[ED], - defaultVertexAttr: VD): GraphImpl[VD, ED] = { - edges.cache() - // Get the set of all vids - val vids = collectVertexIdsFromEdges(edges, new HashPartitioner(edges.partitions.size)) - // Create the VertexRDD. - val vertices = VertexRDD(vids.mapValues(x => defaultVertexAttr)) - GraphImpl(vertices, edges) - } - - /** Collects all vids mentioned in edges and partitions them by partitioner. */ - private def collectVertexIdsFromEdges( - edges: EdgeRDD[_], - partitioner: Partitioner): RDD[(VertexId, Int)] = { - // TODO: Consider doing map side distinct before shuffle. - new ShuffledRDD[VertexId, Int, (VertexId, Int)]( - edges.collectVertexIds.map(vid => (vid, 0)), partitioner) - .setSerializer(new VertexIdMsgSerializer) - } } // end of object GraphImpl diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala index c45ba3d2f8c2..1c6d7e59e9a2 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala @@ -89,7 +89,6 @@ class MsgRDDFunctions[T: ClassTag](self: RDD[MessageToPartition[T]]) { } - private[graphx] object MsgRDDFunctions { implicit def rdd2PartitionRDDFunctions[T: ClassTag](rdd: RDD[MessageToPartition[T]]) = { @@ -99,18 +98,28 @@ object MsgRDDFunctions { implicit def rdd2vertexMessageRDDFunctions[T: ClassTag](rdd: RDD[VertexBroadcastMsg[T]]) = { new VertexBroadcastMsgRDDFunctions(rdd) } +} - def partitionForAggregation[T: ClassTag](msgs: RDD[(VertexId, T)], partitioner: Partitioner) = { - val rdd = new ShuffledRDD[VertexId, T, (VertexId, T)](msgs, partitioner) +private[graphx] +class VertexRDDFunctions[VD: ClassTag](self: RDD[(VertexId, VD)]) { + def copartitionWithVertices(partitioner: Partitioner): RDD[(VertexId, VD)] = { + val rdd = new ShuffledRDD[VertexId, VD, (VertexId, VD)](self, partitioner) // Set a custom serializer if the data is of int or double type. - if (classTag[T] == ClassTag.Int) { + if (classTag[VD] == ClassTag.Int) { rdd.setSerializer(new IntAggMsgSerializer) - } else if (classTag[T] == ClassTag.Long) { + } else if (classTag[VD] == ClassTag.Long) { rdd.setSerializer(new LongAggMsgSerializer) - } else if (classTag[T] == ClassTag.Double) { + } else if (classTag[VD] == ClassTag.Double) { rdd.setSerializer(new DoubleAggMsgSerializer) } rdd } } + +private[graphx] +object VertexRDDFunctions { + implicit def rdd2VertexRDDFunctions[VD: ClassTag](rdd: RDD[(VertexId, VD)]) = { + new VertexRDDFunctions(rdd) + } +} 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 a8154b63ce5f..3a0bba1b93b4 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 @@ -21,192 +21,102 @@ import scala.reflect.{classTag, ClassTag} import org.apache.spark.SparkContext._ import org.apache.spark.rdd.RDD -import org.apache.spark.util.collection.{PrimitiveVector, OpenHashSet} import org.apache.spark.graphx._ /** - * 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 - * 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 evaluated, it remains materialized). This avoids - * constructing it twice if the user calls graph.triplets followed by graph.mapReduceTriplets, for - * 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. + * Manages shipping vertex attributes to the edge partitions of an + * [[org.apache.spark.graphx.EdgeRDD]]. Vertex attributes may be partially shipped to construct a + * triplet view with vertex attributes on only one side, and they may be updated. An active vertex + * set may additionally be shipped to the edge partitions. Be careful not to store a reference to + * `edges`, since it may be modified when the attribute shipping level is upgraded. */ private[impl] -class ReplicatedVertexView[VD: ClassTag]( - updatedVerts: VertexRDD[VD], - edges: EdgeRDD[_], - routingTable: RoutingTable, - prevViewOpt: Option[ReplicatedVertexView[VD]] = None) { +class ReplicatedVertexView[VD: ClassTag, ED: ClassTag]( + var edges: EdgeRDD[ED, VD], + var hasSrcId: Boolean = false, + var hasDstId: Boolean = false) { /** - * 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. + * Return a new `ReplicatedVertexView` with the specified `EdgeRDD`, which must have the same + * shipping level. */ - private val localVertexIdMap: RDD[(Int, VertexIdToIndexMap)] = prevViewOpt match { - case Some(prevView) => - prevView.localVertexIdMap - 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().setName("ReplicatedVertexView localVertexIdMap") - } - - 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 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 withEdges[VD2: ClassTag, ED2: ClassTag]( + edges_ : EdgeRDD[ED2, VD2]): ReplicatedVertexView[VD2, ED2] = { + new ReplicatedVertexView(edges_, hasSrcId, hasDstId) } - def get(includeSrc: Boolean, includeDst: Boolean): RDD[(PartitionID, VertexPartition[VD])] = { - (includeSrc, includeDst) match { - case (true, true) => bothAttrs - case (true, false) => srcAttrOnly - case (false, true) => dstAttrOnly - case (false, false) => noAttrs - } + /** + * Return a new `ReplicatedVertexView` where edges are reversed and shipping levels are swapped to + * match. + */ + def reverse() = { + val newEdges = edges.mapEdgePartitions((pid, part) => part.reverse) + new ReplicatedVertexView(newEdges, hasDstId, hasSrcId) } - def get( - includeSrc: Boolean, - includeDst: Boolean, - 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 - // also shipped there. - 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 - // VertexPartitions - get(includeSrc, includeDst).zipPartitions(shippedActives) { (viewIter, shippedActivesIter) => - val (pid, vPart) = viewIter.next() - val newPart = vPart.replaceActives(shippedActivesIter.flatMap(_._2.iterator)) - Iterator((pid, newPart)) + /** + * Upgrade the shipping level in-place to the specified levels by shipping vertex attributes from + * `vertices`. This operation modifies the `ReplicatedVertexView`, and callers can access `edges` + * afterwards to obtain the upgraded view. + */ + def upgrade(vertices: VertexRDD[VD], includeSrc: Boolean, includeDst: Boolean) { + val shipSrc = includeSrc && !hasSrcId + val shipDst = includeDst && !hasDstId + if (shipSrc || shipDst) { + val shippedVerts: RDD[(Int, VertexAttributeBlock[VD])] = + vertices.shipVertexAttributes(shipSrc, shipDst) + .setName("ReplicatedVertexView.upgrade(%s, %s) - shippedVerts %s %s (broadcast)".format( + includeSrc, includeDst, shipSrc, shipDst)) + .partitionBy(edges.partitioner.get) + val newEdges = new EdgeRDD(edges.partitionsRDD.zipPartitions(shippedVerts) { + (ePartIter, shippedVertsIter) => ePartIter.map { + case (pid, edgePartition) => + (pid, edgePartition.updateVertices(shippedVertsIter.flatMap(_._2.iterator))) + } + }) + edges = newEdges + hasSrcId = includeSrc + hasDstId = includeDst } } - private def create(includeSrc: Boolean, includeDst: Boolean) - : RDD[(PartitionID, VertexPartition[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(_, _)(vdTag)) + /** + * Return a new `ReplicatedVertexView` where the `activeSet` in each edge partition contains only + * vertex ids present in `actives`. This ships a vertex id to all edge partitions where it is + * referenced, ignoring the attribute shipping level. + */ + def withActiveSet(actives: VertexRDD[_]): ReplicatedVertexView[VD, ED] = { + val shippedActives = actives.shipVertexIds() + .setName("ReplicatedVertexView.withActiveSet - shippedActives (broadcast)") .partitionBy(edges.partitioner.get) - // TODO: Consider using a specialized shuffler. - - prevViewOpt match { - case Some(prevView) => - // Update prevView with shippedVerts, setting staleness flags in the resulting - // VertexPartitions - 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 - // 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 - val vertexArray = vdTag.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) - vertexArray(ind) = attr - } - } - val newVPart = new VertexPartition( - vidToIndex, vertexArray, vidToIndex.getBitSet)(vdTag) - Iterator((pid, newVPart)) - }.cache().setName("ReplicatedVertexView %s %s".format(includeSrc, includeDst)) - } - } -} - -private object ReplicatedVertexView { - protected def buildBuffer[VD: ClassTag]( - pid2vidIter: Iterator[Array[Array[VertexId]]], - vertexPartIter: Iterator[VertexPartition[VD]]) = { - 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[VertexId](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 + val newEdges = new EdgeRDD(edges.partitionsRDD.zipPartitions(shippedActives) { + (ePartIter, shippedActivesIter) => ePartIter.map { + case (pid, edgePartition) => + (pid, edgePartition.withActiveSet(shippedActivesIter.flatMap(_._2.iterator))) } - (pid, new VertexAttributeBlock(vids.trim().array, attrs.trim().array)) - } + }) + new ReplicatedVertexView(newEdges, hasSrcId, hasDstId) } - protected def buildActiveBuffer( - pid2vidIter: Iterator[Array[Array[VertexId]]], - activePartIter: Iterator[VertexPartition[_]]) - : Iterator[(Int, Array[VertexId])] = { - val pid2vid: Array[Array[VertexId]] = pid2vidIter.next() - val activePart: VertexPartition[_] = activePartIter.next() + /** + * Return a new `ReplicatedVertexView` where vertex attributes in edge partition are updated using + * `updates`. This ships a vertex attribute only to the edge partitions where it is in the + * position(s) specified by the attribute shipping level. + */ + def updateVertices(updates: VertexRDD[VD]): ReplicatedVertexView[VD, ED] = { + val shippedVerts = updates.shipVertexAttributes(hasSrcId, hasDstId) + .setName("ReplicatedVertexView.updateVertices - shippedVerts %s %s (broadcast)".format( + hasSrcId, hasDstId)) + .partitionBy(edges.partitioner.get) - Iterator.tabulate(pid2vid.size) { pid => - val vidsCandidate = pid2vid(pid) - val size = vidsCandidate.length - val actives = new PrimitiveVector[VertexId](vidsCandidate.size) - var i = 0 - while (i < size) { - val vid = vidsCandidate(i) - if (activePart.isDefined(vid)) { - actives += vid - } - i += 1 + val newEdges = new EdgeRDD(edges.partitionsRDD.zipPartitions(shippedVerts) { + (ePartIter, shippedVertsIter) => ePartIter.map { + case (pid, edgePartition) => + (pid, edgePartition.updateVertices(shippedVertsIter.flatMap(_._2.iterator))) } - (pid, actives.trim().array) - } + }) + new ReplicatedVertexView(newEdges, hasSrcId, hasDstId) } } - -private[graphx] -class VertexAttributeBlock[VD: ClassTag](val vids: Array[VertexId], val attrs: Array[VD]) - extends Serializable { - def iterator: Iterator[(VertexId, VD)] = - (0 until vids.size).iterator.map { i => (vids(i), attrs(i)) } -} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTable.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTable.scala deleted file mode 100644 index 022d5668e294..000000000000 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTable.scala +++ /dev/null @@ -1,82 +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.graphx.impl - -import org.apache.spark.SparkContext._ -import org.apache.spark.graphx._ -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. - */ -private[impl] -class RoutingTable(edges: EdgeRDD[_], vertices: VertexRDD[_]) { - - 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[VertexId]]] = - (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[VertexId]]] = { - // Determine which vertices each edge partition needs by creating a mapping from vid to pid. - 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. - 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 numEdgePartitions = edges.partitions.size - vid2pid.partitionBy(vertices.partitioner.get).mapPartitions { iter => - val pid2vid = Array.fill(numEdgePartitions)(new PrimitiveVector[VertexId]) - for ((vid, pid) <- iter) { - pid2vid(pid) += vid - } - - Iterator(pid2vid.map(_.trim().array)) - }.cache().setName("RoutingTable %s %s".format(includeSrcAttr, includeDstAttr)) - } -} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala new file mode 100644 index 000000000000..927e32ad0f44 --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala @@ -0,0 +1,158 @@ +/* + * 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.graphx.impl + +import scala.reflect.ClassTag + +import org.apache.spark.Partitioner +import org.apache.spark.rdd.RDD +import org.apache.spark.rdd.ShuffledRDD +import org.apache.spark.util.collection.{BitSet, PrimitiveVector} + +import org.apache.spark.graphx._ +import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap + +/** + * A message from the edge partition `pid` to the vertex partition containing `vid` specifying that + * the edge partition references `vid` in the specified `position` (src, dst, or both). +*/ +private[graphx] +class RoutingTableMessage( + var vid: VertexId, + var pid: PartitionID, + var position: Byte) + extends Product2[VertexId, (PartitionID, Byte)] with Serializable { + override def _1 = vid + override def _2 = (pid, position) + override def canEqual(that: Any): Boolean = that.isInstanceOf[RoutingTableMessage] +} + +private[graphx] +class RoutingTableMessageRDDFunctions(self: RDD[RoutingTableMessage]) { + /** Copartition an `RDD[RoutingTableMessage]` with the vertex RDD with the given `partitioner`. */ + def copartitionWithVertices(partitioner: Partitioner): RDD[RoutingTableMessage] = { + new ShuffledRDD[VertexId, (PartitionID, Byte), RoutingTableMessage](self, partitioner) + .setSerializer(new RoutingTableMessageSerializer) + } +} + +private[graphx] +object RoutingTableMessageRDDFunctions { + import scala.language.implicitConversions + + implicit def rdd2RoutingTableMessageRDDFunctions(rdd: RDD[RoutingTableMessage]) = { + new RoutingTableMessageRDDFunctions(rdd) + } +} + +private[graphx] +object RoutingTablePartition { + val empty: RoutingTablePartition = new RoutingTablePartition(Array.empty) + + /** Generate a `RoutingTableMessage` for each vertex referenced in `edgePartition`. */ + def edgePartitionToMsgs(pid: PartitionID, edgePartition: EdgePartition[_, _]) + : Iterator[RoutingTableMessage] = { + // Determine which positions each vertex id appears in using a map where the low 2 bits + // represent src and dst + val map = new PrimitiveKeyOpenHashMap[VertexId, Byte] + edgePartition.srcIds.iterator.foreach { srcId => + map.changeValue(srcId, 0x1, (b: Byte) => (b | 0x1).toByte) + } + edgePartition.dstIds.iterator.foreach { dstId => + map.changeValue(dstId, 0x2, (b: Byte) => (b | 0x2).toByte) + } + map.iterator.map { vidAndPosition => + new RoutingTableMessage(vidAndPosition._1, pid, vidAndPosition._2) + } + } + + /** Build a `RoutingTablePartition` from `RoutingTableMessage`s. */ + def fromMsgs(numEdgePartitions: Int, iter: Iterator[RoutingTableMessage]) + : RoutingTablePartition = { + val pid2vid = Array.fill(numEdgePartitions)(new PrimitiveVector[VertexId]) + val srcFlags = Array.fill(numEdgePartitions)(new PrimitiveVector[Boolean]) + val dstFlags = Array.fill(numEdgePartitions)(new PrimitiveVector[Boolean]) + for (msg <- iter) { + pid2vid(msg.pid) += msg.vid + srcFlags(msg.pid) += (msg.position & 0x1) != 0 + dstFlags(msg.pid) += (msg.position & 0x2) != 0 + } + + new RoutingTablePartition(pid2vid.zipWithIndex.map { + case (vids, pid) => (vids.trim().array, toBitSet(srcFlags(pid)), toBitSet(dstFlags(pid))) + }) + } + + /** Compact the given vector of Booleans into a BitSet. */ + private def toBitSet(flags: PrimitiveVector[Boolean]): BitSet = { + val bitset = new BitSet(flags.size) + var i = 0 + while (i < flags.size) { + if (flags(i)) { + bitset.set(i) + } + i += 1 + } + bitset + } +} + +/** + * Stores the locations of edge-partition join sites for each vertex attribute in a particular + * vertex partition. This provides routing information for shipping vertex attributes to edge + * partitions. + */ +private[graphx] +class RoutingTablePartition( + private val routingTable: Array[(Array[VertexId], BitSet, BitSet)]) { + /** The maximum number of edge partitions this `RoutingTablePartition` is built to join with. */ + val numEdgePartitions: Int = routingTable.size + + /** Returns the number of vertices that will be sent to the specified edge partition. */ + def partitionSize(pid: PartitionID): Int = routingTable(pid)._1.size + + /** Returns an iterator over all vertex ids stored in this `RoutingTablePartition`. */ + def iterator: Iterator[VertexId] = routingTable.iterator.flatMap(_._1.iterator) + + /** Returns a new RoutingTablePartition reflecting a reversal of all edge directions. */ + def reverse: RoutingTablePartition = { + new RoutingTablePartition(routingTable.map { + case (vids, srcVids, dstVids) => (vids, dstVids, srcVids) + }) + } + + /** + * Runs `f` on each vertex id to be sent to the specified edge partition. Vertex ids can be + * filtered by the position they have in the edge partition. + */ + def foreachWithinEdgePartition + (pid: PartitionID, includeSrc: Boolean, includeDst: Boolean) + (f: VertexId => Unit) { + val (vidsCandidate, srcVids, dstVids) = routingTable(pid) + val size = vidsCandidate.length + if (includeSrc && includeDst) { + // Avoid checks for performance + vidsCandidate.iterator.foreach(f) + } else if (!includeSrc && !includeDst) { + // Do nothing + } else { + val relevantVids = if (includeSrc) srcVids else dstVids + relevantVids.iterator.foreach { i => f(vidsCandidate(i)) } + } + } +} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala index 1de42eeca1f0..033237f59721 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala @@ -27,6 +27,35 @@ import scala.reflect.ClassTag import org.apache.spark.graphx._ import org.apache.spark.serializer._ +private[graphx] +class RoutingTableMessageSerializer extends Serializer with Serializable { + override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { + + override def serializeStream(s: OutputStream): SerializationStream = + new ShuffleSerializationStream(s) { + def writeObject[T: ClassTag](t: T): SerializationStream = { + val msg = t.asInstanceOf[RoutingTableMessage] + writeVarLong(msg.vid, optimizePositive = false) + writeUnsignedVarInt(msg.pid) + // TODO: Write only the bottom two bits of msg.position + s.write(msg.position) + this + } + } + + override def deserializeStream(s: InputStream): DeserializationStream = + new ShuffleDeserializationStream(s) { + override def readObject[T: ClassTag](): T = { + val a = readVarLong(optimizePositive = false) + val b = readUnsignedVarInt() + val c = s.read() + if (c == -1) throw new EOFException + new RoutingTableMessage(a, b, c.toByte).asInstanceOf[T] + } + } + } +} + private[graphx] class VertexIdMsgSerializer extends Serializer with Serializable { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala new file mode 100644 index 000000000000..f4e221d4e05a --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala @@ -0,0 +1,149 @@ +/* + * 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.graphx.impl + +import scala.reflect.ClassTag + +import org.apache.spark.util.collection.{BitSet, PrimitiveVector} + +import org.apache.spark.graphx._ +import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap + +/** Stores vertex attributes to ship to an edge partition. */ +private[graphx] +class VertexAttributeBlock[VD: ClassTag](val vids: Array[VertexId], val attrs: Array[VD]) + extends Serializable { + def iterator: Iterator[(VertexId, VD)] = + (0 until vids.size).iterator.map { i => (vids(i), attrs(i)) } +} + +private[graphx] +object ShippableVertexPartition { + /** Construct a `ShippableVertexPartition` from the given vertices without any routing table. */ + def apply[VD: ClassTag](iter: Iterator[(VertexId, VD)]): ShippableVertexPartition[VD] = + apply(iter, RoutingTablePartition.empty, null.asInstanceOf[VD]) + + /** + * Construct a `ShippableVertexPartition` from the given vertices with the specified routing + * table, filling in missing vertices mentioned in the routing table using `defaultVal`. + */ + def apply[VD: ClassTag]( + iter: Iterator[(VertexId, VD)], routingTable: RoutingTablePartition, defaultVal: VD) + : ShippableVertexPartition[VD] = { + val fullIter = iter ++ routingTable.iterator.map(vid => (vid, defaultVal)) + val (index, values, mask) = VertexPartitionBase.initFrom(fullIter, (a: VD, b: VD) => a) + new ShippableVertexPartition(index, values, mask, routingTable) + } + + import scala.language.implicitConversions + + /** + * Implicit conversion to allow invoking `VertexPartitionBase` operations directly on a + * `ShippableVertexPartition`. + */ + implicit def shippablePartitionToOps[VD: ClassTag](partition: ShippableVertexPartition[VD]) = + new ShippableVertexPartitionOps(partition) + + /** + * Implicit evidence that `ShippableVertexPartition` is a member of the + * `VertexPartitionBaseOpsConstructor` typeclass. This enables invoking `VertexPartitionBase` + * operations on a `ShippableVertexPartition` via an evidence parameter, as in + * [[VertexPartitionBaseOps]]. + */ + implicit object ShippableVertexPartitionOpsConstructor + extends VertexPartitionBaseOpsConstructor[ShippableVertexPartition] { + def toOps[VD: ClassTag](partition: ShippableVertexPartition[VD]) + : VertexPartitionBaseOps[VD, ShippableVertexPartition] = shippablePartitionToOps(partition) + } +} + +/** + * A map from vertex id to vertex attribute that additionally stores edge partition join sites for + * each vertex attribute, enabling joining with an [[org.apache.spark.graphx.EdgeRDD]]. + */ +private[graphx] +class ShippableVertexPartition[VD: ClassTag]( + val index: VertexIdToIndexMap, + val values: Array[VD], + val mask: BitSet, + val routingTable: RoutingTablePartition) + extends VertexPartitionBase[VD] { + + /** Return a new ShippableVertexPartition with the specified routing table. */ + def withRoutingTable(routingTable_ : RoutingTablePartition): ShippableVertexPartition[VD] = { + new ShippableVertexPartition(index, values, mask, routingTable_) + } + + /** + * Generate a `VertexAttributeBlock` for each edge partition keyed on the edge partition ID. The + * `VertexAttributeBlock` contains the vertex attributes from the current partition that are + * referenced in the specified positions in the edge partition. + */ + def shipVertexAttributes( + shipSrc: Boolean, shipDst: Boolean): Iterator[(PartitionID, VertexAttributeBlock[VD])] = { + Iterator.tabulate(routingTable.numEdgePartitions) { pid => + val initialSize = if (shipSrc && shipDst) routingTable.partitionSize(pid) else 64 + val vids = new PrimitiveVector[VertexId](initialSize) + val attrs = new PrimitiveVector[VD](initialSize) + var i = 0 + routingTable.foreachWithinEdgePartition(pid, shipSrc, shipDst) { vid => + if (isDefined(vid)) { + vids += vid + attrs += this(vid) + } + i += 1 + } + (pid, new VertexAttributeBlock(vids.trim().array, attrs.trim().array)) + } + } + + /** + * Generate a `VertexId` array for each edge partition keyed on the edge partition ID. The array + * contains the visible vertex ids from the current partition that are referenced in the edge + * partition. + */ + def shipVertexIds(): Iterator[(PartitionID, Array[VertexId])] = { + Iterator.tabulate(routingTable.numEdgePartitions) { pid => + val vids = new PrimitiveVector[VertexId](routingTable.partitionSize(pid)) + var i = 0 + routingTable.foreachWithinEdgePartition(pid, true, true) { vid => + if (isDefined(vid)) { + vids += vid + } + i += 1 + } + (pid, vids.trim().array) + } + } +} + +private[graphx] class ShippableVertexPartitionOps[VD: ClassTag](self: ShippableVertexPartition[VD]) + extends VertexPartitionBaseOps[VD, ShippableVertexPartition](self) { + + def withIndex(index: VertexIdToIndexMap): ShippableVertexPartition[VD] = { + new ShippableVertexPartition(index, self.values, self.mask, self.routingTable) + } + + def withValues[VD2: ClassTag](values: Array[VD2]): ShippableVertexPartition[VD2] = { + new ShippableVertexPartition(self.index, values, self.mask, self.routingTable) + } + + def withMask(mask: BitSet): ShippableVertexPartition[VD] = { + new ShippableVertexPartition(self.index, self.values, mask, self.routingTable) + } +} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala index 7a54b413dc8c..f1d174720a1b 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala @@ -19,260 +19,59 @@ package org.apache.spark.graphx.impl import scala.reflect.ClassTag -import org.apache.spark.Logging +import org.apache.spark.util.collection.BitSet + import org.apache.spark.graphx._ import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap -import org.apache.spark.util.collection.BitSet private[graphx] object VertexPartition { - - 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[(VertexId, VD)], mergeFunc: (VD, VD) => VD) - : VertexPartition[VD] = - { - val map = new PrimitiveKeyOpenHashMap[VertexId, VD] - iter.foreach { case (k, v) => - map.setMerge(k, v, mergeFunc) - } - new VertexPartition(map.keySet, map._values, map.keySet.getBitSet) - } -} - - -private[graphx] -class VertexPartition[@specialized(Long, Int, Double) VD: ClassTag]( - val index: VertexIdToIndexMap, - val values: Array[VD], - 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 - - def size: Int = mask.cardinality() - - /** Return the vertex attribute for the given vertex ID. */ - def apply(vid: VertexId): VD = values(index.getPos(vid)) - - 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: VertexId): Boolean = { - activeSet.get.contains(vid) + /** Construct a `VertexPartition` from the given vertices. */ + def apply[VD: ClassTag](iter: Iterator[(VertexId, VD)]) + : VertexPartition[VD] = { + val (index, values, mask) = VertexPartitionBase.initFrom(iter) + new VertexPartition(index, values, mask) } - /** The number of active vertices, if any exist. */ - def numActives: Option[Int] = activeSet.map(_.size) + import scala.language.implicitConversions /** - * 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 VertexRDD. The resulting - * VertexPartition retains the same index. + * Implicit conversion to allow invoking `VertexPartitionBase` operations directly on a + * `VertexPartition`. */ - 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) - 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. - * - * @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: (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 - var i = mask.nextSetBit(0) - while (i >= 0) { - if (pred(index.getValue(i), values(i))) { - newMask.set(i) - } - i = mask.nextSetBit(i + 1) - } - new VertexPartition(index, values, newMask) - } + implicit def partitionToOps[VD: ClassTag](partition: VertexPartition[VD]) = + new VertexPartitionOps(partition) /** - * 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. + * Implicit evidence that `VertexPartition` is a member of the `VertexPartitionBaseOpsConstructor` + * typeclass. This enables invoking `VertexPartitionBase` operations on a `VertexPartition` via an + * evidence parameter, as in [[VertexPartitionBaseOps]]. */ - def diff(other: VertexPartition[VD]): VertexPartition[VD] = { - 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) - } - new VertexPartition(index, other.values, newMask) - } - } - - /** Left outer join another VertexPartition. */ - def leftJoin[VD2: ClassTag, VD3: ClassTag] - (other: VertexPartition[VD2]) - (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) - } 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) - } - } - - /** Left outer join another iterator of messages. */ - def leftJoin[VD2: ClassTag, VD3: ClassTag] - (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: (VertexId, VD, U) => VD2): VertexPartition[VD2] = { - 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) - } - } - - /** - * Inner join an iterator of messages. - */ - def innerJoin[U: ClassTag, VD2: ClassTag] - (iter: Iterator[Product2[VertexId, U]]) - (f: (VertexId, VD, U) => VD2): VertexPartition[VD2] = { - innerJoin(createUsingIndex(iter))(f) + implicit object VertexPartitionOpsConstructor + extends VertexPartitionBaseOpsConstructor[VertexPartition] { + def toOps[VD: ClassTag](partition: VertexPartition[VD]) + : VertexPartitionBaseOps[VD, VertexPartition] = partitionToOps(partition) } +} - /** - * Similar effect as aggregateUsingIndex((a, b) => a) - */ - def createUsingIndex[VD2: ClassTag](iter: Iterator[Product2[VertexId, 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 (pos >= 0) { - newMask.set(pos) - newValues(pos) = vdata - } - } - new VertexPartition[VD2](index, newValues, newMask) - } +/** A map from vertex id to vertex attribute. */ +private[graphx] class VertexPartition[VD: ClassTag]( + val index: VertexIdToIndexMap, + val values: Array[VD], + val mask: BitSet) + extends VertexPartitionBase[VD] - /** - * 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[VertexId, VD]]): VertexPartition[VD] = { - val newMask = new BitSet(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) - if (pos >= 0) { - newMask.set(pos) - newValues(pos) = vdata - } - } - new VertexPartition(index, newValues, newMask) - } +private[graphx] class VertexPartitionOps[VD: ClassTag](self: VertexPartition[VD]) + extends VertexPartitionBaseOps[VD, VertexPartition](self) { - def aggregateUsingIndex[VD2: ClassTag]( - 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 => - val vid = product._1 - val vdata = product._2 - val pos = index.getPos(vid) - 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) + def withIndex(index: VertexIdToIndexMap): VertexPartition[VD] = { + new VertexPartition(index, self.values, self.mask) } - def replaceActives(iter: Iterator[VertexId]): VertexPartition[VD] = { - val newActiveSet = new VertexSet - iter.foreach(newActiveSet.add(_)) - new VertexPartition(index, values, mask, Some(newActiveSet)) + def withValues[VD2: ClassTag](values: Array[VD2]): VertexPartition[VD2] = { + new VertexPartition(self.index, values, self.mask) } - /** - * Construct a new VertexPartition whose index contains only the vertices in the mask. - */ - def reindex(): VertexPartition[VD] = { - val hashMap = new PrimitiveKeyOpenHashMap[VertexId, VD] - val arbitraryMerge = (a: VD, b: VD) => a - for ((k, v) <- this.iterator) { - hashMap.setMerge(k, v, arbitraryMerge) - } - new VertexPartition(hashMap.keySet, hashMap._values, hashMap.keySet.getBitSet) + def withMask(mask: BitSet): VertexPartition[VD] = { + new VertexPartition(self.index, self.values, mask) } - - def iterator: Iterator[(VertexId, VD)] = - mask.iterator.map(ind => (index.getValue(ind), values(ind))) - - def vidIterator: Iterator[VertexId] = mask.iterator.map(ind => index.getValue(ind)) } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala new file mode 100644 index 000000000000..8d9e0204d27f --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala @@ -0,0 +1,91 @@ +/* + * 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.graphx.impl + +import scala.language.higherKinds +import scala.reflect.ClassTag + +import org.apache.spark.util.collection.BitSet + +import org.apache.spark.graphx._ +import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap + +private[graphx] object VertexPartitionBase { + /** + * Construct the constituents of a VertexPartitionBase from the given vertices, merging duplicate + * entries arbitrarily. + */ + def initFrom[VD: ClassTag](iter: Iterator[(VertexId, VD)]) + : (VertexIdToIndexMap, Array[VD], BitSet) = { + val map = new PrimitiveKeyOpenHashMap[VertexId, VD] + iter.foreach { pair => + map(pair._1) = pair._2 + } + (map.keySet, map._values, map.keySet.getBitSet) + } + + /** + * Construct the constituents of a VertexPartitionBase from the given vertices, merging duplicate + * entries using `mergeFunc`. + */ + def initFrom[VD: ClassTag](iter: Iterator[(VertexId, VD)], mergeFunc: (VD, VD) => VD) + : (VertexIdToIndexMap, Array[VD], BitSet) = { + val map = new PrimitiveKeyOpenHashMap[VertexId, VD] + iter.foreach { pair => + map.setMerge(pair._1, pair._2, mergeFunc) + } + (map.keySet, map._values, map.keySet.getBitSet) + } +} + +/** + * An abstract map from vertex id to vertex attribute. [[VertexPartition]] is the corresponding + * concrete implementation. [[VertexPartitionBaseOps]] provides a variety of operations for + * VertexPartitionBase and subclasses that provide implicit evidence of membership in the + * `VertexPartitionBaseOpsConstructor` typeclass (for example, + * [[VertexPartition.VertexPartitionOpsConstructor]]). + */ +private[graphx] abstract class VertexPartitionBase[@specialized(Long, Int, Double) VD: ClassTag] { + + def index: VertexIdToIndexMap + def values: Array[VD] + def mask: BitSet + + val capacity: Int = index.capacity + + def size: Int = mask.cardinality() + + /** Return the vertex attribute for the given vertex ID. */ + def apply(vid: VertexId): VD = values(index.getPos(vid)) + + def isDefined(vid: VertexId): Boolean = { + val pos = index.getPos(vid) + pos >= 0 && mask.get(pos) + } + + def iterator: Iterator[(VertexId, VD)] = + mask.iterator.map(ind => (index.getValue(ind), values(ind))) +} + +/** + * A typeclass for subclasses of `VertexPartitionBase` representing the ability to wrap them in a + * `VertexPartitionBaseOps`. + */ +private[graphx] trait VertexPartitionBaseOpsConstructor[T[X] <: VertexPartitionBase[X]] { + def toOps[VD: ClassTag](partition: T[VD]): VertexPartitionBaseOps[VD, T] +} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala new file mode 100644 index 000000000000..21ff615feca6 --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala @@ -0,0 +1,245 @@ +/* + * 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.graphx.impl + +import scala.language.higherKinds +import scala.language.implicitConversions +import scala.reflect.ClassTag + +import org.apache.spark.Logging +import org.apache.spark.util.collection.BitSet + +import org.apache.spark.graphx._ +import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap + +/** + * An class containing additional operations for subclasses of VertexPartitionBase that provide + * implicit evidence of membership in the `VertexPartitionBaseOpsConstructor` typeclass (for + * example, [[VertexPartition.VertexPartitionOpsConstructor]]). + */ +private[graphx] abstract class VertexPartitionBaseOps + [VD: ClassTag, Self[X] <: VertexPartitionBase[X] : VertexPartitionBaseOpsConstructor] + (self: Self[VD]) + extends Logging { + + def withIndex(index: VertexIdToIndexMap): Self[VD] + def withValues[VD2: ClassTag](values: Array[VD2]): Self[VD2] + def withMask(mask: BitSet): Self[VD] + + /** + * 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 VertexRDD. The resulting + * VertexPartition retains the same index. + */ + def map[VD2: ClassTag](f: (VertexId, VD) => VD2): Self[VD2] = { + // Construct a view of the map transformation + val newValues = new Array[VD2](self.capacity) + var i = self.mask.nextSetBit(0) + while (i >= 0) { + newValues(i) = f(self.index.getValue(i), self.values(i)) + i = self.mask.nextSetBit(i + 1) + } + this.withValues(newValues) + } + + /** + * 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: (VertexId, VD) => Boolean): Self[VD] = { + // Allocate the array to store the results into + val newMask = new BitSet(self.capacity) + // Iterate over the active bits in the old mask and evaluate the predicate + var i = self.mask.nextSetBit(0) + while (i >= 0) { + if (pred(self.index.getValue(i), self.values(i))) { + newMask.set(i) + } + i = self.mask.nextSetBit(i + 1) + } + this.withMask(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: Self[VD]): Self[VD] = { + if (self.index != other.index) { + logWarning("Diffing two VertexPartitions with different indexes is slow.") + diff(createUsingIndex(other.iterator)) + } else { + val newMask = self.mask & other.mask + var i = newMask.nextSetBit(0) + while (i >= 0) { + if (self.values(i) == other.values(i)) { + newMask.unset(i) + } + i = newMask.nextSetBit(i + 1) + } + this.withValues(other.values).withMask(newMask) + } + } + + /** Left outer join another VertexPartition. */ + def leftJoin[VD2: ClassTag, VD3: ClassTag] + (other: Self[VD2]) + (f: (VertexId, VD, Option[VD2]) => VD3): Self[VD3] = { + if (self.index != other.index) { + logWarning("Joining two VertexPartitions with different indexes is slow.") + leftJoin(createUsingIndex(other.iterator))(f) + } else { + val newValues = new Array[VD3](self.capacity) + + var i = self.mask.nextSetBit(0) + while (i >= 0) { + val otherV: Option[VD2] = if (other.mask.get(i)) Some(other.values(i)) else None + newValues(i) = f(self.index.getValue(i), self.values(i), otherV) + i = self.mask.nextSetBit(i + 1) + } + this.withValues(newValues) + } + } + + /** Left outer join another iterator of messages. */ + def leftJoin[VD2: ClassTag, VD3: ClassTag] + (other: Iterator[(VertexId, VD2)]) + (f: (VertexId, VD, Option[VD2]) => VD3): Self[VD3] = { + leftJoin(createUsingIndex(other))(f) + } + + /** Inner join another VertexPartition. */ + def innerJoin[U: ClassTag, VD2: ClassTag] + (other: Self[U]) + (f: (VertexId, VD, U) => VD2): Self[VD2] = { + if (self.index != other.index) { + logWarning("Joining two VertexPartitions with different indexes is slow.") + innerJoin(createUsingIndex(other.iterator))(f) + } else { + val newMask = self.mask & other.mask + val newValues = new Array[VD2](self.capacity) + var i = newMask.nextSetBit(0) + while (i >= 0) { + newValues(i) = f(self.index.getValue(i), self.values(i), other.values(i)) + i = newMask.nextSetBit(i + 1) + } + this.withValues(newValues).withMask(newMask) + } + } + + /** + * Inner join an iterator of messages. + */ + def innerJoin[U: ClassTag, VD2: ClassTag] + (iter: Iterator[Product2[VertexId, U]]) + (f: (VertexId, VD, U) => VD2): Self[VD2] = { + innerJoin(createUsingIndex(iter))(f) + } + + /** + * Similar effect as aggregateUsingIndex((a, b) => a) + */ + def createUsingIndex[VD2: ClassTag](iter: Iterator[Product2[VertexId, VD2]]) + : Self[VD2] = { + val newMask = new BitSet(self.capacity) + val newValues = new Array[VD2](self.capacity) + iter.foreach { pair => + val pos = self.index.getPos(pair._1) + if (pos >= 0) { + newMask.set(pos) + newValues(pos) = pair._2 + } + } + this.withValues(newValues).withMask(newMask) + } + + /** + * 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[VertexId, VD]]): Self[VD] = { + val newMask = new BitSet(self.capacity) + val newValues = new Array[VD](self.capacity) + System.arraycopy(self.values, 0, newValues, 0, newValues.length) + iter.foreach { pair => + val pos = self.index.getPos(pair._1) + if (pos >= 0) { + newMask.set(pos) + newValues(pos) = pair._2 + } + } + this.withValues(newValues).withMask(newMask) + } + + def aggregateUsingIndex[VD2: ClassTag]( + iter: Iterator[Product2[VertexId, VD2]], + reduceFunc: (VD2, VD2) => VD2): Self[VD2] = { + val newMask = new BitSet(self.capacity) + val newValues = new Array[VD2](self.capacity) + iter.foreach { product => + val vid = product._1 + val vdata = product._2 + val pos = self.index.getPos(vid) + 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 + } + } + } + this.withValues(newValues).withMask(newMask) + } + + /** + * Construct a new VertexPartition whose index contains only the vertices in the mask. + */ + def reindex(): Self[VD] = { + val hashMap = new PrimitiveKeyOpenHashMap[VertexId, VD] + val arbitraryMerge = (a: VD, b: VD) => a + for ((k, v) <- self.iterator) { + hashMap.setMerge(k, v, arbitraryMerge) + } + this.withIndex(hashMap.keySet).withValues(hashMap._values).withMask(hashMap.keySet.getBitSet) + } + + /** + * Converts a vertex partition (in particular, one of type `Self`) into a + * `VertexPartitionBaseOps`. Within this class, this allows chaining the methods defined above, + * because these methods return a `Self` and this implicit conversion re-wraps that in a + * `VertexPartitionBaseOps`. This relies on the context bound on `Self`. + */ + private implicit def toOps[VD2: ClassTag]( + partition: Self[VD2]): VertexPartitionBaseOps[VD2, Self] = { + implicitly[VertexPartitionBaseOpsConstructor[Self]].toOps(partition) + } +} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/Analytics.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/Analytics.scala index d901d4fe225f..069e042ed94a 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/Analytics.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/Analytics.scala @@ -55,6 +55,7 @@ object Analytics extends Logging { val conf = new SparkConf() .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") .set("spark.kryo.registrator", "org.apache.spark.graphx.GraphKryoRegistrator") + .set("spark.locality.wait", "100000") taskType match { case "pagerank" => @@ -62,12 +63,14 @@ object Analytics extends Logging { var outFname = "" var numEPart = 4 var partitionStrategy: Option[PartitionStrategy] = None + var numIterOpt: Option[Int] = None options.foreach{ case ("tol", v) => tol = v.toFloat case ("output", v) => outFname = v case ("numEPart", v) => numEPart = v.toInt case ("partStrategy", v) => partitionStrategy = Some(pickPartitioner(v)) + case ("numIter", v) => numIterOpt = Some(v.toInt) case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) } @@ -84,7 +87,10 @@ object Analytics extends Logging { println("GRAPHX: Number of vertices " + graph.vertices.count) println("GRAPHX: Number of edges " + graph.edges.count) - val pr = graph.pageRank(tol).vertices.cache() + val pr = (numIterOpt match { + case Some(numIter) => PageRank.run(graph, numIter) + case None => PageRank.runUntilConvergence(graph, tol) + }).vertices.cache() println("GRAPHX: Total rank: " + pr.map(_._2).reduce(_ + _)) 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 32b5fe481359..7b9bac5d9c8e 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala @@ -110,7 +110,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: VertexId, y: VertexId))), p), 0) + verts.withFilter(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) @@ -120,7 +120,13 @@ class GraphSuite extends FunSuite with LocalSparkContext { val part = iter.next()._2 Iterator((part.srcIds ++ part.dstIds).toSet) }.collect - assert(verts.forall(id => partitionSets.count(_.contains(id)) <= bound)) + if (!verts.forall(id => partitionSets.count(_.contains(id)) <= bound)) { + val numFailures = verts.count(id => partitionSets.count(_.contains(id)) > bound) + val failure = verts.maxBy(id => partitionSets.count(_.contains(id))) + fail(("Replication bound test failed for %d/%d vertices. " + + "Example: vertex %d replicated to %d (> %f) partitions.").format( + numFailures, n, failure, partitionSets.count(_.contains(failure)), bound)) + } // This should not be true for the default hash partitioning val partitionSetsUnpartitioned = graph.edges.partitionsRDD.mapPartitions { iter => val part = iter.next()._2 diff --git a/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala index e135d1d7ad6a..d2e0c01bc35e 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala @@ -26,10 +26,16 @@ import org.apache.spark.graphx._ class EdgePartitionSuite extends FunSuite { + def makeEdgePartition[A: ClassTag](xs: Iterable[(Int, Int, A)]): EdgePartition[A, Int] = { + val builder = new EdgePartitionBuilder[A, Int] + for ((src, dst, attr) <- xs) { builder.add(src: VertexId, dst: VertexId, attr) } + builder.toEdgePartition + } + 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] + val builder = new EdgePartitionBuilder[Int, Nothing] for (e <- edges) { builder.add(e.srcId, e.dstId, e.attr) } @@ -40,7 +46,7 @@ class EdgePartitionSuite extends FunSuite { test("map") { val edges = List(Edge(0, 1, 0), Edge(1, 2, 0), Edge(2, 0, 0)) - val builder = new EdgePartitionBuilder[Int] + val builder = new EdgePartitionBuilder[Int, Nothing] for (e <- edges) { builder.add(e.srcId, e.dstId, e.attr) } @@ -49,11 +55,22 @@ class EdgePartitionSuite extends FunSuite { edges.map(e => e.copy(attr = e.srcId + e.dstId))) } + test("filter") { + val edges = List(Edge(0, 1, 0), Edge(0, 2, 0), Edge(2, 0, 0)) + val builder = new EdgePartitionBuilder[Int, Int] + for (e <- edges) { + builder.add(e.srcId, e.dstId, e.attr) + } + val edgePartition = builder.toEdgePartition + val filtered = edgePartition.filter(et => et.srcId == 0, (vid, attr) => vid == 0 || vid == 1) + assert(filtered.tripletIterator().toList.map(et => (et.srcId, et.dstId)) === List((0L, 1L))) + } + 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] + val builder = new EdgePartitionBuilder[Int, Nothing] for (e <- edges) { builder.add(e.srcId, e.dstId, e.attr) } @@ -61,11 +78,19 @@ class EdgePartitionSuite extends FunSuite { assert(edgePartition.groupEdges(_ + _).iterator.map(_.copy()).toList === groupedEdges) } + test("upgradeIterator") { + val edges = List((0, 1, 0), (1, 0, 0)) + val verts = List((0L, 1), (1L, 2)) + val part = makeEdgePartition(edges).updateVertices(verts.iterator) + assert(part.upgradeIterator(part.iterator).map(_.toTuple).toList === + part.tripletIterator().toList.map(_.toTuple)) + } + test("indexIterator") { 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] + val builder = new EdgePartitionBuilder[Int, Nothing] for (e <- Random.shuffle(sortedEdges)) { builder.add(e.srcId, e.dstId, e.attr) } @@ -77,11 +102,6 @@ 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: VertexId, dst: VertexId, 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) @@ -90,4 +110,14 @@ class EdgePartitionSuite extends FunSuite { 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))) } + + test("isActive, numActives, replaceActives") { + val ep = new EdgePartitionBuilder[Nothing, Nothing].toEdgePartition + .withActiveSet(Iterator(0L, 2L, 0L)) + assert(ep.isActive(0)) + assert(!ep.isActive(1)) + assert(ep.isActive(2)) + assert(!ep.isActive(-1)) + assert(ep.numActives == Some(2)) + } } diff --git a/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgeTripletIteratorSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgeTripletIteratorSuite.scala index 9cbb2d2acdc2..49b2704390fe 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgeTripletIteratorSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgeTripletIteratorSuite.scala @@ -26,17 +26,11 @@ import org.apache.spark.graphx._ class EdgeTripletIteratorSuite extends FunSuite { test("iterator.toList") { - val builder = new EdgePartitionBuilder[Int] + val builder = new EdgePartitionBuilder[Int, Int] builder.add(1, 2, 0) builder.add(1, 3, 0) builder.add(1, 4, 0) - val vidmap = new VertexIdToIndexMap - vidmap.add(1) - vidmap.add(2) - vidmap.add(3) - vidmap.add(4) - val vs = Array.fill(vidmap.capacity)(0) - val iter = new EdgeTripletIterator[Int, Int](vidmap, vs, builder.toEdgePartition) + val iter = new EdgeTripletIterator[Int, Int](builder.toEdgePartition, true, true) val result = iter.toList.map(et => (et.srcId, et.dstId)) assert(result === Seq((1, 2), (1, 3), (1, 4))) } diff --git a/graphx/src/test/scala/org/apache/spark/graphx/impl/VertexPartitionSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/impl/VertexPartitionSuite.scala index a048d13fd12b..8bf1384d514c 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/impl/VertexPartitionSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/impl/VertexPartitionSuite.scala @@ -30,17 +30,6 @@ class VertexPartitionSuite extends FunSuite { 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) diff --git a/project/MimaBuild.scala b/project/MimaBuild.scala index efdb38e907d1..fafc9b36a77d 100644 --- a/project/MimaBuild.scala +++ b/project/MimaBuild.scala @@ -76,6 +76,8 @@ object MimaBuild { excludeSparkClass("util.XORShiftRandom") ++ excludeSparkClass("graphx.EdgeRDD") ++ excludeSparkClass("graphx.VertexRDD") ++ + excludeSparkClass("graphx.impl.GraphImpl") ++ + excludeSparkClass("graphx.impl.RoutingTable") ++ excludeSparkClass("mllib.recommendation.MFDataGenerator") ++ excludeSparkClass("mllib.optimization.SquaredGradient") ++ excludeSparkClass("mllib.regression.RidgeRegressionWithSGD") ++ From 2b7bd29eb6ee5baf739eec143044ecfc296b9b1f Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sat, 10 May 2014 20:50:40 -0700 Subject: [PATCH 03/18] SPARK-1789. Multiple versions of Netty dependencies cause FlumeStreamSuite failure TL;DR is there is a bit of JAR hell trouble with Netty, that can be mostly resolved and will resolve a test failure. I hit the error described at http://apache-spark-user-list.1001560.n3.nabble.com/SparkContext-startup-time-out-td1753.html while running FlumeStreamingSuite, and have for a short while (is it just me?) velvia notes: "I have found a workaround. If you add akka 2.2.4 to your dependencies, then everything works, probably because akka 2.2.4 brings in newer version of Jetty." There are at least 3 versions of Netty in play in the build: - the new Flume 1.4.0 dependency brings in io.netty:netty:3.4.0.Final, and that is the immediate problem - the custom version of akka 2.2.3 depends on io.netty:netty:3.6.6. - but, Spark Core directly uses io.netty:netty-all:4.0.17.Final The POMs try to exclude other versions of netty, but are excluding org.jboss.netty:netty, when in fact older versions of io.netty:netty (not netty-all) are also an issue. The org.jboss.netty:netty excludes are largely unnecessary. I replaced many of them with io.netty:netty exclusions until everything agreed on io.netty:netty-all:4.0.17.Final. But this didn't work, since Akka 2.2.3 doesn't work with Netty 4.x. Down-grading to 3.6.6.Final across the board made some Spark code not compile. If the build *keeps* io.netty:netty:3.6.6.Final as well, everything seems to work. Part of the reason seems to be that Netty 3.x used the old `org.jboss.netty` packages. This is less than ideal, but is no worse than the current situation. So this PR resolves the issue and improves the JAR hell, even if it leaves the existing theoretical Netty 3-vs-4 conflict: - Remove org.jboss.netty excludes where possible, for clarity; they're not needed except with Hadoop artifacts - Add io.netty:netty excludes where needed -- except, let akka keep its io.netty:netty - Change a bit of test code that actually depended on Netty 3.x, to use 4.x equivalent - Update SBT build accordingly A better change would be to update Akka far enough such that it agrees on Netty 4.x, but I don't know if that's feasible. Author: Sean Owen Closes #723 from srowen/SPARK-1789 and squashes the following commits: 43661b7 [Sean Owen] Update and add Netty excludes to prevent some JAR conflicts that cause test issues --- .../org/apache/spark/LocalSparkContext.scala | 3 +- examples/pom.xml | 4 +++ external/flume/pom.xml | 2 +- external/mqtt/pom.xml | 6 ---- external/twitter/pom.xml | 6 ---- external/zeromq/pom.xml | 6 ---- pom.xml | 32 ----------------- project/SparkBuild.scala | 35 ++++++++++--------- 8 files changed, 24 insertions(+), 70 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/LocalSparkContext.scala b/core/src/test/scala/org/apache/spark/LocalSparkContext.scala index 4b972f88a954..53e367a61715 100644 --- a/core/src/test/scala/org/apache/spark/LocalSparkContext.scala +++ b/core/src/test/scala/org/apache/spark/LocalSparkContext.scala @@ -17,8 +17,7 @@ package org.apache.spark -import org.jboss.netty.logging.InternalLoggerFactory -import org.jboss.netty.logging.Slf4JLoggerFactory +import _root_.io.netty.util.internal.logging.{Slf4JLoggerFactory, InternalLoggerFactory} import org.scalatest.BeforeAndAfterAll import org.scalatest.BeforeAndAfterEach import org.scalatest.Suite diff --git a/examples/pom.xml b/examples/pom.xml index e1fc149d87f1..874bcd7916f3 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -106,6 +106,10 @@ org.jboss.netty netty + + io.netty + netty + commons-logging commons-logging diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 03d3b2394f51..6aec215687fe 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -50,7 +50,7 @@ 1.4.0 - org.jboss.netty + io.netty netty diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml index 9aa1c1a9f5b8..7b2dc5ba1d7f 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -53,12 +53,6 @@ ${akka.group} akka-zeromq_${scala.binary.version} ${akka.version} - - - org.jboss.netty - netty - - org.scalatest diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml index a44345959471..5766d3a0d44e 100644 --- a/external/twitter/pom.xml +++ b/external/twitter/pom.xml @@ -48,12 +48,6 @@ org.twitter4j twitter4j-stream 3.0.3 - - - org.jboss.netty - netty - - org.scalatest diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml index a40e55876e64..4ed4196bd866 100644 --- a/external/zeromq/pom.xml +++ b/external/zeromq/pom.xml @@ -48,12 +48,6 @@ ${akka.group} akka-zeromq_${scala.binary.version} ${akka.version} - - - org.jboss.netty - netty - - org.scalatest diff --git a/pom.xml b/pom.xml index e0bff60a54cd..c4e1c6be52a1 100644 --- a/pom.xml +++ b/pom.xml @@ -324,45 +324,21 @@ ${akka.group} akka-actor_${scala.binary.version} ${akka.version} - - - org.jboss.netty - netty - - ${akka.group} akka-remote_${scala.binary.version} ${akka.version} - - - org.jboss.netty - netty - - ${akka.group} akka-slf4j_${scala.binary.version} ${akka.version} - - - org.jboss.netty - netty - - ${akka.group} akka-testkit_${scala.binary.version} ${akka.version} - - - org.jboss.netty - netty - - colt @@ -513,10 +489,6 @@ avro ${avro.version} - - org.jboss.netty - netty - io.netty netty @@ -551,10 +523,6 @@ avro-mapred ${avro.version} - - org.jboss.netty - netty - io.netty netty diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index af882b3ea7be..a12c61853e41 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -303,7 +303,8 @@ object SparkBuild extends Build { val parquetVersion = "1.4.3" val slf4jVersion = "1.7.5" - val excludeNetty = ExclusionRule(organization = "org.jboss.netty") + val excludeJBossNetty = ExclusionRule(organization = "org.jboss.netty") + val excludeIONetty = ExclusionRule(organization = "io.netty") val excludeEclipseJetty = ExclusionRule(organization = "org.eclipse.jetty") val excludeAsm = ExclusionRule(organization = "org.ow2.asm") val excludeOldAsm = ExclusionRule(organization = "asm") @@ -337,8 +338,8 @@ object SparkBuild extends Build { "commons-daemon" % "commons-daemon" % "1.0.10", // workaround for bug HADOOP-9407 "com.ning" % "compress-lzf" % "1.0.0", "org.xerial.snappy" % "snappy-java" % "1.0.5", - "org.spark-project.akka" %% "akka-remote" % akkaVersion excludeAll(excludeNetty), - "org.spark-project.akka" %% "akka-slf4j" % akkaVersion excludeAll(excludeNetty), + "org.spark-project.akka" %% "akka-remote" % akkaVersion, + "org.spark-project.akka" %% "akka-slf4j" % akkaVersion, "org.spark-project.akka" %% "akka-testkit" % akkaVersion % "test", "org.json4s" %% "json4s-jackson" % "3.2.6" excludeAll(excludeScalap), "colt" % "colt" % "1.2.0", @@ -346,8 +347,8 @@ object SparkBuild extends Build { "commons-net" % "commons-net" % "2.2", "net.java.dev.jets3t" % "jets3t" % jets3tVersion excludeAll(excludeCommonsLogging), "org.apache.derby" % "derby" % "10.4.2.0" % "test", - "org.apache.hadoop" % hadoopClient % hadoopVersion excludeAll(excludeNetty, excludeAsm, excludeCommonsLogging, excludeSLF4J, excludeOldAsm), - "org.apache.curator" % "curator-recipes" % "2.4.0" excludeAll(excludeNetty), + "org.apache.hadoop" % hadoopClient % hadoopVersion excludeAll(excludeJBossNetty, excludeAsm, excludeCommonsLogging, excludeSLF4J, excludeOldAsm), + "org.apache.curator" % "curator-recipes" % "2.4.0" excludeAll(excludeJBossNetty), "com.codahale.metrics" % "metrics-core" % codahaleMetricsVersion, "com.codahale.metrics" % "metrics-jvm" % codahaleMetricsVersion, "com.codahale.metrics" % "metrics-json" % codahaleMetricsVersion, @@ -421,7 +422,7 @@ object SparkBuild extends Build { v => "spark-examples-" + v + "-hadoop" + hadoopVersion + ".jar" }, libraryDependencies ++= Seq( "com.twitter" %% "algebird-core" % "0.1.11", - "org.apache.hbase" % "hbase" % HBASE_VERSION excludeAll(excludeNetty, excludeAsm, excludeOldAsm, excludeCommonsLogging, excludeJruby), + "org.apache.hbase" % "hbase" % HBASE_VERSION excludeAll(excludeIONetty, excludeJBossNetty, excludeAsm, excludeOldAsm, excludeCommonsLogging, excludeJruby), "org.apache.cassandra" % "cassandra-all" % "1.2.6" exclude("com.google.guava", "guava") exclude("com.googlecode.concurrentlinkedhashmap", "concurrentlinkedhashmap-lru") @@ -429,7 +430,7 @@ object SparkBuild extends Build { exclude("io.netty", "netty") exclude("jline","jline") exclude("org.apache.cassandra.deps", "avro") - excludeAll(excludeSLF4J), + excludeAll(excludeSLF4J, excludeIONetty), "com.github.scopt" %% "scopt" % "3.2.0" ) ) ++ assemblySettings ++ extraAssemblySettings @@ -561,11 +562,11 @@ object SparkBuild extends Build { def yarnEnabledSettings = Seq( libraryDependencies ++= Seq( // Exclude rule required for all ? - "org.apache.hadoop" % hadoopClient % hadoopVersion excludeAll(excludeNetty, excludeAsm, excludeOldAsm), - "org.apache.hadoop" % "hadoop-yarn-api" % hadoopVersion excludeAll(excludeNetty, excludeAsm, excludeOldAsm), - "org.apache.hadoop" % "hadoop-yarn-common" % hadoopVersion excludeAll(excludeNetty, excludeAsm, excludeOldAsm), - "org.apache.hadoop" % "hadoop-yarn-client" % hadoopVersion excludeAll(excludeNetty, excludeAsm, excludeOldAsm), - "org.apache.hadoop" % "hadoop-yarn-server-web-proxy" % hadoopVersion excludeAll(excludeNetty, excludeAsm, excludeOldAsm) + "org.apache.hadoop" % hadoopClient % hadoopVersion excludeAll(excludeJBossNetty, excludeAsm, excludeOldAsm), + "org.apache.hadoop" % "hadoop-yarn-api" % hadoopVersion excludeAll(excludeJBossNetty, excludeAsm, excludeOldAsm), + "org.apache.hadoop" % "hadoop-yarn-common" % hadoopVersion excludeAll(excludeJBossNetty, excludeAsm, excludeOldAsm), + "org.apache.hadoop" % "hadoop-yarn-client" % hadoopVersion excludeAll(excludeJBossNetty, excludeAsm, excludeOldAsm), + "org.apache.hadoop" % "hadoop-yarn-server-web-proxy" % hadoopVersion excludeAll(excludeJBossNetty, excludeAsm, excludeOldAsm) ) ) @@ -593,7 +594,7 @@ object SparkBuild extends Build { name := "spark-streaming-twitter", previousArtifact := sparkPreviousArtifact("spark-streaming-twitter"), libraryDependencies ++= Seq( - "org.twitter4j" % "twitter4j-stream" % "3.0.3" excludeAll(excludeNetty) + "org.twitter4j" % "twitter4j-stream" % "3.0.3" ) ) @@ -601,12 +602,12 @@ object SparkBuild extends Build { name := "spark-streaming-kafka", previousArtifact := sparkPreviousArtifact("spark-streaming-kafka"), libraryDependencies ++= Seq( - "com.github.sgroschupf" % "zkclient" % "0.1" excludeAll(excludeNetty), + "com.github.sgroschupf" % "zkclient" % "0.1", "org.apache.kafka" %% "kafka" % "0.8.0" exclude("com.sun.jdmk", "jmxtools") exclude("com.sun.jmx", "jmxri") exclude("net.sf.jopt-simple", "jopt-simple") - excludeAll(excludeNetty, excludeSLF4J) + excludeAll(excludeSLF4J) ) ) @@ -614,7 +615,7 @@ object SparkBuild extends Build { name := "spark-streaming-flume", previousArtifact := sparkPreviousArtifact("spark-streaming-flume"), libraryDependencies ++= Seq( - "org.apache.flume" % "flume-ng-sdk" % "1.4.0" % "compile" excludeAll(excludeNetty, excludeThrift) + "org.apache.flume" % "flume-ng-sdk" % "1.4.0" % "compile" excludeAll(excludeIONetty, excludeThrift) ) ) @@ -622,7 +623,7 @@ object SparkBuild extends Build { name := "spark-streaming-zeromq", previousArtifact := sparkPreviousArtifact("spark-streaming-zeromq"), libraryDependencies ++= Seq( - "org.spark-project.akka" %% "akka-zeromq" % akkaVersion excludeAll(excludeNetty) + "org.spark-project.akka" %% "akka-zeromq" % akkaVersion ) ) From 83e0424d87022e7a967088365931a08aa06ffd9f Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Sat, 10 May 2014 20:58:02 -0700 Subject: [PATCH 04/18] [SPARK-1774] Respect SparkSubmit --jars on YARN (client) SparkSubmit ignores `--jars` for YARN client. This is a bug. This PR also automatically adds the application jar to `spark.jar`. Previously, when running as yarn-client, you must specify the jar additionally through `--files` (because `--jars` didn't work). Now you don't have to explicitly specify it through either. Tested on a YARN cluster. Author: Andrew Or Closes #710 from andrewor14/yarn-jars and squashes the following commits: 35d1928 [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-jars c27bf6c [Andrew Or] For yarn-cluster and python, do not add primaryResource to spark.jar c92c5bf [Andrew Or] Minor cleanups 269f9f3 [Andrew Or] Fix format 013d840 [Andrew Or] Fix tests 1407474 [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-jars 3bb75e8 [Andrew Or] Allow SparkSubmit --jars to take effect in yarn-client mode --- .../scala/org/apache/spark/SparkContext.scala | 2 +- .../org/apache/spark/deploy/SparkSubmit.scala | 39 ++++--- .../spark/deploy/SparkSubmitSuite.scala | 110 ++++++++++++------ .../spark/deploy/yarn/ClientArguments.scala | 4 +- 4 files changed, 102 insertions(+), 53 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index c639b3e15ded..71bab295442f 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -917,7 +917,7 @@ class SparkContext(config: SparkConf) extends Logging { if (SparkHadoopUtil.get.isYarnMode() && (master == "yarn-standalone" || master == "yarn-cluster")) { // In order for this to work in yarn-cluster mode the user must specify the - // --addjars option to the client to upload the file into the distributed cache + // --addJars option to the client to upload the file into the distributed cache // of the AM to make it show up in the current working directory. val fileName = new Path(uri.getPath).getName() try { diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 16de6f7cdb10..c6d3cbd2e728 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -67,8 +67,7 @@ object SparkSubmit { private[spark] def printWarning(str: String) = printStream.println("Warning: " + str) /** - * @return - * a tuple containing the arguments for the child, a list of classpath + * @return a tuple containing the arguments for the child, a list of classpath * entries for the child, a list of system propertes, a list of env vars * and the main class for the child */ @@ -115,13 +114,16 @@ object SparkSubmit { val sysProps = new HashMap[String, String]() var childMainClass = "" + val isPython = args.isPython + val isYarnCluster = clusterManager == YARN && deployOnCluster + if (clusterManager == MESOS && deployOnCluster) { printErrorAndExit("Cannot currently run driver on the cluster in Mesos") } // If we're running a Python app, set the Java class to run to be our PythonRunner, add // Python files to deployment list, and pass the main file and Python path to PythonRunner - if (args.isPython) { + if (isPython) { if (deployOnCluster) { printErrorAndExit("Cannot currently run Python driver programs on cluster") } @@ -161,6 +163,7 @@ object SparkSubmit { val options = List[OptionAssigner]( OptionAssigner(args.master, ALL_CLUSTER_MGRS, false, sysProp = "spark.master"), OptionAssigner(args.name, ALL_CLUSTER_MGRS, false, sysProp = "spark.app.name"), + OptionAssigner(args.name, YARN, true, clOption = "--name", sysProp = "spark.app.name"), OptionAssigner(args.driverExtraClassPath, STANDALONE | YARN, true, sysProp = "spark.driver.extraClassPath"), OptionAssigner(args.driverExtraJavaOptions, STANDALONE | YARN, true, @@ -168,7 +171,8 @@ object SparkSubmit { OptionAssigner(args.driverExtraLibraryPath, STANDALONE | YARN, true, sysProp = "spark.driver.extraLibraryPath"), OptionAssigner(args.driverMemory, YARN, true, clOption = "--driver-memory"), - OptionAssigner(args.name, YARN, true, clOption = "--name", sysProp = "spark.app.name"), + OptionAssigner(args.driverMemory, STANDALONE, true, clOption = "--memory"), + OptionAssigner(args.driverCores, STANDALONE, true, clOption = "--cores"), OptionAssigner(args.queue, YARN, true, clOption = "--queue"), OptionAssigner(args.queue, YARN, false, sysProp = "spark.yarn.queue"), OptionAssigner(args.numExecutors, YARN, true, clOption = "--num-executors"), @@ -176,20 +180,18 @@ object SparkSubmit { OptionAssigner(args.executorMemory, YARN, true, clOption = "--executor-memory"), OptionAssigner(args.executorMemory, STANDALONE | MESOS | YARN, false, sysProp = "spark.executor.memory"), - OptionAssigner(args.driverMemory, STANDALONE, true, clOption = "--memory"), - OptionAssigner(args.driverCores, STANDALONE, true, clOption = "--cores"), OptionAssigner(args.executorCores, YARN, true, clOption = "--executor-cores"), OptionAssigner(args.executorCores, YARN, false, sysProp = "spark.executor.cores"), OptionAssigner(args.totalExecutorCores, STANDALONE | MESOS, false, sysProp = "spark.cores.max"), OptionAssigner(args.files, YARN, false, sysProp = "spark.yarn.dist.files"), OptionAssigner(args.files, YARN, true, clOption = "--files"), + OptionAssigner(args.files, LOCAL | STANDALONE | MESOS, false, sysProp = "spark.files"), + OptionAssigner(args.files, LOCAL | STANDALONE | MESOS, true, sysProp = "spark.files"), OptionAssigner(args.archives, YARN, false, sysProp = "spark.yarn.dist.archives"), OptionAssigner(args.archives, YARN, true, clOption = "--archives"), OptionAssigner(args.jars, YARN, true, clOption = "--addJars"), - OptionAssigner(args.files, LOCAL | STANDALONE | MESOS, false, sysProp = "spark.files"), - OptionAssigner(args.files, LOCAL | STANDALONE | MESOS, true, sysProp = "spark.files"), - OptionAssigner(args.jars, LOCAL | STANDALONE | MESOS, false, sysProp = "spark.jars") + OptionAssigner(args.jars, ALL_CLUSTER_MGRS, false, sysProp = "spark.jars") ) // For client mode make any added jars immediately visible on the classpath @@ -212,9 +214,10 @@ object SparkSubmit { } } - // For standalone mode, add the application jar automatically so the user doesn't have to - // call sc.addJar. TODO: Standalone mode in the cluster - if (clusterManager == STANDALONE) { + // Add the application jar automatically so the user doesn't have to call sc.addJar + // For YARN cluster mode, the jar is already distributed on each node as "app.jar" + // For python files, the primary resource is already distributed as a regular file + if (!isYarnCluster && !isPython) { var jars = sysProps.get("spark.jars").map(x => x.split(",").toSeq).getOrElse(Seq()) if (args.primaryResource != RESERVED_JAR_NAME) { jars = jars ++ Seq(args.primaryResource) @@ -222,11 +225,11 @@ object SparkSubmit { sysProps.put("spark.jars", jars.mkString(",")) } + // Standalone cluster specific configurations if (deployOnCluster && clusterManager == STANDALONE) { if (args.supervise) { childArgs += "--supervise" } - childMainClass = "org.apache.spark.deploy.Client" childArgs += "launch" childArgs += (args.master, args.primaryResource, args.mainClass) @@ -243,6 +246,7 @@ object SparkSubmit { } } + // Read from default spark properties, if any for ((k, v) <- args.getDefaultSparkProperties) { if (!sysProps.contains(k)) sysProps(k) = v } @@ -250,9 +254,12 @@ object SparkSubmit { (childArgs, childClasspath, sysProps, childMainClass) } - private def launch(childArgs: ArrayBuffer[String], childClasspath: ArrayBuffer[String], - sysProps: Map[String, String], childMainClass: String, verbose: Boolean = false) - { + private def launch( + childArgs: ArrayBuffer[String], + childClasspath: ArrayBuffer[String], + sysProps: Map[String, String], + childMainClass: String, + verbose: Boolean = false) { if (verbose) { printStream.println(s"Main class:\n$childMainClass") printStream.println(s"Arguments:\n${childArgs.mkString("\n")}") diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index c9edb03cdeb0..6c0deede5378 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -87,25 +87,41 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { } test("handles arguments with --key=val") { - val clArgs = Seq("--jars=one.jar,two.jar,three.jar", "--name=myApp") + val clArgs = Seq( + "--jars=one.jar,two.jar,three.jar", + "--name=myApp") val appArgs = new SparkSubmitArguments(clArgs) appArgs.jars should be ("one.jar,two.jar,three.jar") appArgs.name should be ("myApp") } test("handles arguments to user program") { - val clArgs = Seq("--name", "myApp", "--class", "Foo", "userjar.jar", "some", "--weird", "args") + val clArgs = Seq( + "--name", "myApp", + "--class", "Foo", + "userjar.jar", + "some", + "--weird", "args") val appArgs = new SparkSubmitArguments(clArgs) appArgs.childArgs should be (Seq("some", "--weird", "args")) } test("handles YARN cluster mode") { - val clArgs = Seq("--deploy-mode", "cluster", - "--master", "yarn", "--executor-memory", "5g", "--executor-cores", "5", - "--class", "org.SomeClass", "--jars", "one.jar,two.jar,three.jar", - "--driver-memory", "4g", "--queue", "thequeue", "--files", "file1.txt,file2.txt", - "--archives", "archive1.txt,archive2.txt", "--num-executors", "6", "--name", "beauty", - "thejar.jar", "arg1", "arg2") + val clArgs = Seq( + "--deploy-mode", "cluster", + "--master", "yarn", + "--executor-memory", "5g", + "--executor-cores", "5", + "--class", "org.SomeClass", + "--jars", "one.jar,two.jar,three.jar", + "--driver-memory", "4g", + "--queue", "thequeue", + "--files", "file1.txt,file2.txt", + "--archives", "archive1.txt,archive2.txt", + "--num-executors", "6", + "--name", "beauty", + "thejar.jar", + "arg1", "arg2") val appArgs = new SparkSubmitArguments(clArgs) val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) val childArgsStr = childArgs.mkString(" ") @@ -127,12 +143,21 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { } test("handles YARN client mode") { - val clArgs = Seq("--deploy-mode", "client", - "--master", "yarn", "--executor-memory", "5g", "--executor-cores", "5", - "--class", "org.SomeClass", "--jars", "one.jar,two.jar,three.jar", - "--driver-memory", "4g", "--queue", "thequeue", "--files", "file1.txt,file2.txt", - "--archives", "archive1.txt,archive2.txt", "--num-executors", "6", "--name", "trill", - "thejar.jar", "arg1", "arg2") + val clArgs = Seq( + "--deploy-mode", "client", + "--master", "yarn", + "--executor-memory", "5g", + "--executor-cores", "5", + "--class", "org.SomeClass", + "--jars", "one.jar,two.jar,three.jar", + "--driver-memory", "4g", + "--queue", "thequeue", + "--files", "file1.txt,file2.txt", + "--archives", "archive1.txt,archive2.txt", + "--num-executors", "6", + "--name", "trill", + "thejar.jar", + "arg1", "arg2") val appArgs = new SparkSubmitArguments(clArgs) val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) childArgs.mkString(" ") should be ("arg1 arg2") @@ -142,6 +167,7 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { classpath should contain ("two.jar") classpath should contain ("three.jar") sysProps("spark.app.name") should be ("trill") + sysProps("spark.jars") should be ("one.jar,two.jar,three.jar,thejar.jar") sysProps("spark.executor.memory") should be ("5g") sysProps("spark.executor.cores") should be ("5") sysProps("spark.yarn.queue") should be ("thequeue") @@ -152,9 +178,15 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { } test("handles standalone cluster mode") { - val clArgs = Seq("--deploy-mode", "cluster", - "--master", "spark://h:p", "--class", "org.SomeClass", - "--supervise", "--driver-memory", "4g", "--driver-cores", "5", "thejar.jar", "arg1", "arg2") + val clArgs = Seq( + "--deploy-mode", "cluster", + "--master", "spark://h:p", + "--class", "org.SomeClass", + "--supervise", + "--driver-memory", "4g", + "--driver-cores", "5", + "thejar.jar", + "arg1", "arg2") val appArgs = new SparkSubmitArguments(clArgs) val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) val childArgsStr = childArgs.mkString(" ") @@ -166,9 +198,15 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { } test("handles standalone client mode") { - val clArgs = Seq("--deploy-mode", "client", - "--master", "spark://h:p", "--executor-memory", "5g", "--total-executor-cores", "5", - "--class", "org.SomeClass", "--driver-memory", "4g", "thejar.jar", "arg1", "arg2") + val clArgs = Seq( + "--deploy-mode", "client", + "--master", "spark://h:p", + "--executor-memory", "5g", + "--total-executor-cores", "5", + "--class", "org.SomeClass", + "--driver-memory", "4g", + "thejar.jar", + "arg1", "arg2") val appArgs = new SparkSubmitArguments(clArgs) val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) childArgs.mkString(" ") should be ("arg1 arg2") @@ -179,9 +217,15 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { } test("handles mesos client mode") { - val clArgs = Seq("--deploy-mode", "client", - "--master", "mesos://h:p", "--executor-memory", "5g", "--total-executor-cores", "5", - "--class", "org.SomeClass", "--driver-memory", "4g", "thejar.jar", "arg1", "arg2") + val clArgs = Seq( + "--deploy-mode", "client", + "--master", "mesos://h:p", + "--executor-memory", "5g", + "--total-executor-cores", "5", + "--class", "org.SomeClass", + "--driver-memory", "4g", + "thejar.jar", + "arg1", "arg2") val appArgs = new SparkSubmitArguments(clArgs) val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) childArgs.mkString(" ") should be ("arg1 arg2") @@ -192,15 +236,17 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { } test("launch simple application with spark-submit") { - runSparkSubmit( - Seq( - "--class", SimpleApplicationTest.getClass.getName.stripSuffix("$"), - "--name", "testApp", - "--master", "local", - "unUsed.jar")) + val unusedJar = TestUtils.createJarWithClasses(Seq.empty) + val args = Seq( + "--class", SimpleApplicationTest.getClass.getName.stripSuffix("$"), + "--name", "testApp", + "--master", "local", + unusedJar.toString) + runSparkSubmit(args) } test("spark submit includes jars passed in through --jar") { + val unusedJar = TestUtils.createJarWithClasses(Seq.empty) val jar1 = TestUtils.createJarWithClasses(Seq("SparkSubmitClassA")) val jar2 = TestUtils.createJarWithClasses(Seq("SparkSubmitClassB")) val jarsString = Seq(jar1, jar2).map(j => j.toString).mkString(",") @@ -209,7 +255,7 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { "--name", "testApp", "--master", "local-cluster[2,1,512]", "--jars", jarsString, - "unused.jar") + unusedJar.toString) runSparkSubmit(args) } @@ -227,7 +273,7 @@ object JarCreationTest { def main(args: Array[String]) { val conf = new SparkConf() val sc = new SparkContext(conf) - val result = sc.makeRDD(1 to 100, 10).mapPartitions{ x => + val result = sc.makeRDD(1 to 100, 10).mapPartitions { x => var foundClasses = false try { Class.forName("SparkSubmitClassA", true, Thread.currentThread().getContextClassLoader) @@ -248,7 +294,6 @@ object SimpleApplicationTest { def main(args: Array[String]) { val conf = new SparkConf() val sc = new SparkContext(conf) - val configs = Seq("spark.master", "spark.app.name") for (config <- configs) { val masterValue = conf.get(config) @@ -266,6 +311,5 @@ object SimpleApplicationTest { s"Master had $config=$masterValue but executor had $config=$executorValue") } } - } } diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala index 3e4c739e34fe..b2c413b6d267 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala @@ -20,7 +20,7 @@ package org.apache.spark.deploy.yarn import scala.collection.mutable.{ArrayBuffer, HashMap} import org.apache.spark.SparkConf -import org.apache.spark.scheduler.{InputFormatInfo, SplitInfo} +import org.apache.spark.scheduler.InputFormatInfo import org.apache.spark.util.IntParam import org.apache.spark.util.MemoryParam @@ -40,9 +40,7 @@ class ClientArguments(val args: Array[String], val sparkConf: SparkConf) { var amMemory: Int = 512 // MB var amClass: String = "org.apache.spark.deploy.yarn.ApplicationMaster" var appName: String = "Spark" - // TODO var inputFormatInfo: List[InputFormatInfo] = null - // TODO(harvey) var priority = 0 parseArgs(args.toList) From 70bcdef48a051028598d380d41dfce1c9bfb2b9b Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Sat, 10 May 2014 21:08:04 -0700 Subject: [PATCH 05/18] Enabled incremental build that comes with sbt 0.13.2 More info at. https://github.com/sbt/sbt/issues/1010 Author: Prashant Sharma Closes #525 from ScrapCodes/sbt-inc-opt and squashes the following commits: ba8fa42 [Prashant Sharma] Enabled incremental build that comes with sbt 0.13.2 --- project/SparkBuild.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index a12c61853e41..12791e490ae2 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -176,7 +176,7 @@ object SparkBuild extends Build { retrievePattern := "[type]s/[artifact](-[revision])(-[classifier]).[ext]", transitiveClassifiers in Scope.GlobalScope := Seq("sources"), testListeners <<= target.map(t => Seq(new eu.henkelmann.sbt.JUnitXmlTestsListener(t.getAbsolutePath))), - + incOptions := incOptions.value.withNameHashing(true), // Fork new JVMs for tests and set Java options for those fork := true, javaOptions in Test += "-Dspark.home=" + sparkHome, From 6bee01dd04ef73c6b829110ebcdd622d521ea8ff Mon Sep 17 00:00:00 2001 From: witgo Date: Sun, 11 May 2014 14:34:27 -0700 Subject: [PATCH 06/18] remove outdated runtime Information scala home Author: witgo Closes #728 from witgo/scala_home and squashes the following commits: cdfd8be [witgo] Merge branch 'master' of https://github.com/apache/spark into scala_home fac094a [witgo] remove outdated runtime Information scala home --- core/src/main/scala/org/apache/spark/SparkEnv.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 806e77d98fc5..19d507c0cf86 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -281,8 +281,7 @@ object SparkEnv extends Logging { val jvmInformation = Seq( ("Java Version", "%s (%s)".format(Properties.javaVersion, Properties.javaVendor)), ("Java Home", Properties.javaHome), - ("Scala Version", Properties.versionString), - ("Scala Home", Properties.scalaHome) + ("Scala Version", Properties.versionString) ).sorted // Spark properties From 7d9cc9214bd06495f6838e355331dd2b5f1f7407 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 11 May 2014 17:11:55 -0700 Subject: [PATCH 07/18] SPARK-1770: Load balance elements when repartitioning. This patch adds better balancing when performing a repartition of an RDD. Previously the elements in the RDD were hash partitioned, meaning if the RDD was skewed certain partitions would end up being very large. This commit adds load balancing of elements across the repartitioned RDD splits. The load balancing is not perfect: a given output partition can have up to N more elements than the average if there are N input partitions. However, some randomization is used to minimize the probabiliy that this happens. Author: Patrick Wendell Closes #727 from pwendell/load-balance and squashes the following commits: f9da752 [Patrick Wendell] Response to Matei's feedback acfa46a [Patrick Wendell] SPARK-1770: Load balance elements when repartitioning. --- .../main/scala/org/apache/spark/rdd/RDD.scala | 15 +++++++-- .../scala/org/apache/spark/rdd/RDDSuite.scala | 33 +++++++++++++++++++ 2 files changed, 46 insertions(+), 2 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 a1ca612cc9a0..aa03e9276fb3 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -328,11 +328,22 @@ abstract class RDD[T: ClassTag]( def coalesce(numPartitions: Int, shuffle: Boolean = false)(implicit ord: Ordering[T] = null) : RDD[T] = { if (shuffle) { + /** Distributes elements evenly across output partitions, starting from a random partition. */ + def distributePartition(index: Int, items: Iterator[T]): Iterator[(Int, T)] = { + var position = (new Random(index)).nextInt(numPartitions) + items.map { t => + // Note that the hash code of the key will just be the key itself. The HashPartitioner + // will mod it with the number of total partitions. + position = position + 1 + (position, t) + } + } + // include a shuffle step so that our upstream tasks are still distributed new CoalescedRDD( - new ShuffledRDD[T, Null, (T, Null)](map(x => (x, null)), + new ShuffledRDD[Int, T, (Int, T)](mapPartitionsWithIndex(distributePartition), new HashPartitioner(numPartitions)), - numPartitions).keys + numPartitions).values } else { new CoalescedRDD(this, numPartitions) } diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index 8da9a0da700e..e686068f7a99 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -202,6 +202,39 @@ class RDDSuite extends FunSuite with SharedSparkContext { assert(repartitioned2.collect().toSet === (1 to 1000).toSet) } + test("repartitioned RDDs perform load balancing") { + // Coalesce partitions + val input = Array.fill(1000)(1) + val initialPartitions = 10 + val data = sc.parallelize(input, initialPartitions) + + val repartitioned1 = data.repartition(2) + assert(repartitioned1.partitions.size == 2) + val partitions1 = repartitioned1.glom().collect() + // some noise in balancing is allowed due to randomization + assert(math.abs(partitions1(0).length - 500) < initialPartitions) + assert(math.abs(partitions1(1).length - 500) < initialPartitions) + assert(repartitioned1.collect() === input) + + def testSplitPartitions(input: Seq[Int], initialPartitions: Int, finalPartitions: Int) { + val data = sc.parallelize(input, initialPartitions) + val repartitioned = data.repartition(finalPartitions) + assert(repartitioned.partitions.size === finalPartitions) + val partitions = repartitioned.glom().collect() + // assert all elements are present + assert(repartitioned.collect().sortWith(_ > _).toSeq === input.toSeq.sortWith(_ > _).toSeq) + // assert no bucket is overloaded + for (partition <- partitions) { + val avg = input.size / finalPartitions + val maxPossible = avg + initialPartitions + assert(partition.length <= maxPossible) + } + } + + testSplitPartitions(Array.fill(100)(1), 10, 20) + testSplitPartitions(Array.fill(10000)(1) ++ Array.fill(10000)(2), 20, 100) + } + test("coalesced RDDs") { val data = sc.parallelize(1 to 10, 10) From 05c9aa9eb1b7f13cd40bbca23e6bc7e1d20e91cd Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 11 May 2014 18:17:34 -0700 Subject: [PATCH 08/18] SPARK-1652: Set driver memory correctly in spark-submit. The previous check didn't account for the fact that the default deploy mode is "client" unless otherwise specified. Also, this sets the more narrowly defined SPARK_DRIVER_MEMORY instead of setting SPARK_MEM. Author: Patrick Wendell Closes #730 from pwendell/spark-submit and squashes the following commits: 430b98f [Patrick Wendell] Feedback from Aaron e788edf [Patrick Wendell] Changes based on Aaron's feedback f508146 [Patrick Wendell] SPARK-1652: Set driver memory correctly in spark-submit. --- bin/spark-submit | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/bin/spark-submit b/bin/spark-submit index 49bc26252cad..63903b17a290 100755 --- a/bin/spark-submit +++ b/bin/spark-submit @@ -35,8 +35,10 @@ while (($#)); do shift done -if [ ! -z $DRIVER_MEMORY ] && [ ! -z $DEPLOY_MODE ] && [ $DEPLOY_MODE = "client" ]; then - export SPARK_MEM=$DRIVER_MEMORY +DEPLOY_MODE=${DEPLOY_MODE:-"client"} + +if [ -n "$DRIVER_MEMORY" ] && [ $DEPLOY_MODE == "client" ]; then + export SPARK_DRIVER_MEMORY=$DRIVER_MEMORY fi $SPARK_HOME/bin/spark-class org.apache.spark.deploy.SparkSubmit "${ORIG_ARGS[@]}" From f938a155b2a9c126b292d5403aca31de83d5105a Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Sun, 11 May 2014 18:33:46 -0700 Subject: [PATCH 09/18] Fix error in 2d Graph Partitioner Their was a minor bug in which negative partition ids could be generated when constructing a 2D partitioning of a graph. This could lead to an inefficient 2D partition for large vertex id values. Author: Joseph E. Gonzalez Closes #709 from jegonzal/fix_2d_partitioning and squashes the following commits: 937c562 [Joseph E. Gonzalez] fixing bug in 2d partitioning algorithm where negative partition ids could be generated. --- .../scala/org/apache/spark/graphx/PartitionStrategy.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala b/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala index 0470d74cf9ef..1526ccef06fd 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala @@ -78,8 +78,8 @@ object PartitionStrategy { override def getPartition(src: VertexId, dst: VertexId, numParts: PartitionID): PartitionID = { val ceilSqrtNumParts: PartitionID = math.ceil(math.sqrt(numParts)).toInt val mixingPrime: VertexId = 1125899906842597L - val col: PartitionID = ((math.abs(src) * mixingPrime) % ceilSqrtNumParts).toInt - val row: PartitionID = ((math.abs(dst) * mixingPrime) % ceilSqrtNumParts).toInt + val col: PartitionID = (math.abs(src * mixingPrime) % ceilSqrtNumParts).toInt + val row: PartitionID = (math.abs(dst * mixingPrime) % ceilSqrtNumParts).toInt (col * ceilSqrtNumParts + row) % numParts } } From a6b02fb7486356493474c7f42bb714c9cce215ca Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Sun, 11 May 2014 19:20:42 -0700 Subject: [PATCH 10/18] SPARK-1786: Edge Partition Serialization This appears to address the issue with edge partition serialization. The solution appears to be just registering the `PrimitiveKeyOpenHashMap`. However I noticed that we appear to have forked that code in GraphX but retained the same name (which is confusing). I also renamed our local copy to `GraphXPrimitiveKeyOpenHashMap`. We should consider dropping that and using the one in Spark if possible. Author: Ankur Dave Author: Joseph E. Gonzalez Closes #724 from jegonzal/edge_partition_serialization and squashes the following commits: b0a525a [Ankur Dave] Disable reference tracking to fix serialization test bb7f548 [Ankur Dave] Add failing test for EdgePartition Kryo serialization 67dac22 [Joseph E. Gonzalez] Making EdgePartition serializable. --- .../spark/graphx/GraphKryoRegistrator.scala | 9 ++++++--- .../spark/graphx/impl/EdgePartition.scala | 14 +++++++------- .../graphx/impl/EdgePartitionBuilder.scala | 4 ++-- .../graphx/impl/EdgeTripletIterator.scala | 2 +- .../graphx/impl/RoutingTablePartition.scala | 4 ++-- .../graphx/impl/ShippableVertexPartition.scala | 2 +- .../spark/graphx/impl/VertexPartition.scala | 2 +- .../graphx/impl/VertexPartitionBase.scala | 6 +++--- .../graphx/impl/VertexPartitionBaseOps.scala | 4 ++-- ...ala => GraphXPrimitiveKeyOpenHashMap.scala} | 2 +- .../spark/graphx/impl/EdgePartitionSuite.scala | 18 ++++++++++++++++++ 11 files changed, 44 insertions(+), 23 deletions(-) rename graphx/src/main/scala/org/apache/spark/graphx/util/collection/{PrimitiveKeyOpenHashMap.scala => GraphXPrimitiveKeyOpenHashMap.scala} (98%) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala index d295d0127ac7..f97f329c0e83 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala @@ -24,6 +24,9 @@ import org.apache.spark.util.BoundedPriorityQueue import org.apache.spark.util.collection.BitSet import org.apache.spark.graphx.impl._ +import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap +import org.apache.spark.util.collection.OpenHashSet + /** * Registers GraphX classes with Kryo for improved performance. @@ -43,8 +46,8 @@ class GraphKryoRegistrator extends KryoRegistrator { kryo.register(classOf[PartitionStrategy]) kryo.register(classOf[BoundedPriorityQueue[Object]]) kryo.register(classOf[EdgeDirection]) - - // This avoids a large number of hash table lookups. - kryo.setReferences(false) + kryo.register(classOf[GraphXPrimitiveKeyOpenHashMap[VertexId, Int]]) + kryo.register(classOf[OpenHashSet[Int]]) + kryo.register(classOf[OpenHashSet[Long]]) } } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala index 871e81f8d245..a5c9cd1f8b4e 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala @@ -20,7 +20,7 @@ package org.apache.spark.graphx.impl import scala.reflect.{classTag, ClassTag} import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap /** * A collection of edges stored in columnar format, along with any vertex attributes referenced. The @@ -42,12 +42,12 @@ import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap private[graphx] class EdgePartition[ @specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED: ClassTag, VD: ClassTag]( - @transient val srcIds: Array[VertexId], - @transient val dstIds: Array[VertexId], - @transient val data: Array[ED], - @transient val index: PrimitiveKeyOpenHashMap[VertexId, Int], - @transient val vertices: VertexPartition[VD], - @transient val activeSet: Option[VertexSet] = None + val srcIds: Array[VertexId] = null, + val dstIds: Array[VertexId] = null, + val data: Array[ED] = null, + val index: GraphXPrimitiveKeyOpenHashMap[VertexId, Int] = null, + val vertices: VertexPartition[VD] = null, + val activeSet: Option[VertexSet] = None ) extends Serializable { /** Return a new `EdgePartition` with the specified edge data. */ diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala index ecb49bef42e4..4520beb99151 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala @@ -23,7 +23,7 @@ import scala.util.Sorting import org.apache.spark.util.collection.{BitSet, OpenHashSet, PrimitiveVector} import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap private[graphx] class EdgePartitionBuilder[@specialized(Long, Int, Double) ED: ClassTag, VD: ClassTag]( @@ -41,7 +41,7 @@ class EdgePartitionBuilder[@specialized(Long, Int, Double) ED: ClassTag, VD: Cla 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[VertexId, Int] + val index = new GraphXPrimitiveKeyOpenHashMap[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) { diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala index ebb0b9418d65..56f79a7097fc 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala @@ -20,7 +20,7 @@ package org.apache.spark.graphx.impl import scala.reflect.ClassTag import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap /** * The Iterator type returned when constructing edge triplets. This could be an anonymous class in diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala index 927e32ad0f44..d02e9238adba 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala @@ -25,7 +25,7 @@ import org.apache.spark.rdd.ShuffledRDD import org.apache.spark.util.collection.{BitSet, PrimitiveVector} import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap /** * A message from the edge partition `pid` to the vertex partition containing `vid` specifying that @@ -69,7 +69,7 @@ object RoutingTablePartition { : Iterator[RoutingTableMessage] = { // Determine which positions each vertex id appears in using a map where the low 2 bits // represent src and dst - val map = new PrimitiveKeyOpenHashMap[VertexId, Byte] + val map = new GraphXPrimitiveKeyOpenHashMap[VertexId, Byte] edgePartition.srcIds.iterator.foreach { srcId => map.changeValue(srcId, 0x1, (b: Byte) => (b | 0x1).toByte) } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala index f4e221d4e05a..dca54b8a7da8 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala @@ -22,7 +22,7 @@ import scala.reflect.ClassTag import org.apache.spark.util.collection.{BitSet, PrimitiveVector} import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap /** Stores vertex attributes to ship to an edge partition. */ private[graphx] diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala index f1d174720a1b..55c7a19d1bda 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala @@ -22,7 +22,7 @@ import scala.reflect.ClassTag import org.apache.spark.util.collection.BitSet import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap private[graphx] object VertexPartition { /** Construct a `VertexPartition` from the given vertices. */ diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala index 8d9e0204d27f..34939b24440a 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala @@ -23,7 +23,7 @@ import scala.reflect.ClassTag import org.apache.spark.util.collection.BitSet import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap private[graphx] object VertexPartitionBase { /** @@ -32,7 +32,7 @@ private[graphx] object VertexPartitionBase { */ def initFrom[VD: ClassTag](iter: Iterator[(VertexId, VD)]) : (VertexIdToIndexMap, Array[VD], BitSet) = { - val map = new PrimitiveKeyOpenHashMap[VertexId, VD] + val map = new GraphXPrimitiveKeyOpenHashMap[VertexId, VD] iter.foreach { pair => map(pair._1) = pair._2 } @@ -45,7 +45,7 @@ private[graphx] object VertexPartitionBase { */ def initFrom[VD: ClassTag](iter: Iterator[(VertexId, VD)], mergeFunc: (VD, VD) => VD) : (VertexIdToIndexMap, Array[VD], BitSet) = { - val map = new PrimitiveKeyOpenHashMap[VertexId, VD] + val map = new GraphXPrimitiveKeyOpenHashMap[VertexId, VD] iter.foreach { pair => map.setMerge(pair._1, pair._2, mergeFunc) } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala index 21ff615feca6..a4f769b29401 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala @@ -25,7 +25,7 @@ import org.apache.spark.Logging import org.apache.spark.util.collection.BitSet import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap /** * An class containing additional operations for subclasses of VertexPartitionBase that provide @@ -224,7 +224,7 @@ private[graphx] abstract class VertexPartitionBaseOps * Construct a new VertexPartition whose index contains only the vertices in the mask. */ def reindex(): Self[VD] = { - val hashMap = new PrimitiveKeyOpenHashMap[VertexId, VD] + val hashMap = new GraphXPrimitiveKeyOpenHashMap[VertexId, VD] val arbitraryMerge = (a: VD, b: VD) => a for ((k, v) <- self.iterator) { hashMap.setMerge(k, v, arbitraryMerge) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/util/collection/PrimitiveKeyOpenHashMap.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/collection/GraphXPrimitiveKeyOpenHashMap.scala similarity index 98% rename from graphx/src/main/scala/org/apache/spark/graphx/util/collection/PrimitiveKeyOpenHashMap.scala rename to graphx/src/main/scala/org/apache/spark/graphx/util/collection/GraphXPrimitiveKeyOpenHashMap.scala index 7b02e2ed1a9c..57b01b6f2e1f 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/util/collection/PrimitiveKeyOpenHashMap.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/util/collection/GraphXPrimitiveKeyOpenHashMap.scala @@ -29,7 +29,7 @@ import scala.reflect._ * Under the hood, it uses our OpenHashSet implementation. */ private[graphx] -class PrimitiveKeyOpenHashMap[@specialized(Long, Int) K: ClassTag, +class GraphXPrimitiveKeyOpenHashMap[@specialized(Long, Int) K: ClassTag, @specialized(Long, Int, Double) V: ClassTag]( val keySet: OpenHashSet[K], var _values: Array[V]) extends Iterable[(K, V)] diff --git a/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala index d2e0c01bc35e..28fd112f2b12 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala @@ -22,6 +22,9 @@ import scala.util.Random import org.scalatest.FunSuite +import org.apache.spark.SparkConf +import org.apache.spark.serializer.KryoSerializer + import org.apache.spark.graphx._ class EdgePartitionSuite extends FunSuite { @@ -120,4 +123,19 @@ class EdgePartitionSuite extends FunSuite { assert(!ep.isActive(-1)) assert(ep.numActives == Some(2)) } + + test("Kryo serialization") { + val aList = List((0, 1, 0), (1, 0, 0), (1, 2, 0), (5, 4, 0), (5, 5, 0)) + val a: EdgePartition[Int, Int] = makeEdgePartition(aList) + val conf = new SparkConf() + .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + .set("spark.kryo.registrator", "org.apache.spark.graphx.GraphKryoRegistrator") + val s = new KryoSerializer(conf).newInstance() + val aSer: EdgePartition[Int, Int] = s.deserialize(s.serialize(a)) + assert(aSer.srcIds.toList === a.srcIds.toList) + assert(aSer.dstIds.toList === a.dstIds.toList) + assert(aSer.data.toList === a.data.toList) + assert(aSer.index != null) + assert(aSer.vertices.iterator.toSet === a.vertices.iterator.toSet) + } } From af15c82bfe2c3f73142b8f310784a0e85841539d Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 12 May 2014 10:49:03 -0700 Subject: [PATCH 11/18] Revert "SPARK-1786: Edge Partition Serialization" This reverts commit a6b02fb7486356493474c7f42bb714c9cce215ca. --- .../spark/graphx/GraphKryoRegistrator.scala | 9 +++------ .../spark/graphx/impl/EdgePartition.scala | 14 +++++++------- .../graphx/impl/EdgePartitionBuilder.scala | 4 ++-- .../graphx/impl/EdgeTripletIterator.scala | 2 +- .../graphx/impl/RoutingTablePartition.scala | 4 ++-- .../graphx/impl/ShippableVertexPartition.scala | 2 +- .../spark/graphx/impl/VertexPartition.scala | 2 +- .../graphx/impl/VertexPartitionBase.scala | 6 +++--- .../graphx/impl/VertexPartitionBaseOps.scala | 4 ++-- ...Map.scala => PrimitiveKeyOpenHashMap.scala} | 2 +- .../spark/graphx/impl/EdgePartitionSuite.scala | 18 ------------------ 11 files changed, 23 insertions(+), 44 deletions(-) rename graphx/src/main/scala/org/apache/spark/graphx/util/collection/{GraphXPrimitiveKeyOpenHashMap.scala => PrimitiveKeyOpenHashMap.scala} (98%) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala index f97f329c0e83..d295d0127ac7 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala @@ -24,9 +24,6 @@ import org.apache.spark.util.BoundedPriorityQueue import org.apache.spark.util.collection.BitSet import org.apache.spark.graphx.impl._ -import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap -import org.apache.spark.util.collection.OpenHashSet - /** * Registers GraphX classes with Kryo for improved performance. @@ -46,8 +43,8 @@ class GraphKryoRegistrator extends KryoRegistrator { kryo.register(classOf[PartitionStrategy]) kryo.register(classOf[BoundedPriorityQueue[Object]]) kryo.register(classOf[EdgeDirection]) - kryo.register(classOf[GraphXPrimitiveKeyOpenHashMap[VertexId, Int]]) - kryo.register(classOf[OpenHashSet[Int]]) - kryo.register(classOf[OpenHashSet[Long]]) + + // This avoids a large number of hash table lookups. + kryo.setReferences(false) } } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala index a5c9cd1f8b4e..871e81f8d245 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala @@ -20,7 +20,7 @@ package org.apache.spark.graphx.impl import scala.reflect.{classTag, ClassTag} import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap /** * A collection of edges stored in columnar format, along with any vertex attributes referenced. The @@ -42,12 +42,12 @@ import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap private[graphx] class EdgePartition[ @specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED: ClassTag, VD: ClassTag]( - val srcIds: Array[VertexId] = null, - val dstIds: Array[VertexId] = null, - val data: Array[ED] = null, - val index: GraphXPrimitiveKeyOpenHashMap[VertexId, Int] = null, - val vertices: VertexPartition[VD] = null, - val activeSet: Option[VertexSet] = None + @transient val srcIds: Array[VertexId], + @transient val dstIds: Array[VertexId], + @transient val data: Array[ED], + @transient val index: PrimitiveKeyOpenHashMap[VertexId, Int], + @transient val vertices: VertexPartition[VD], + @transient val activeSet: Option[VertexSet] = None ) extends Serializable { /** Return a new `EdgePartition` with the specified edge data. */ diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala index 4520beb99151..ecb49bef42e4 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala @@ -23,7 +23,7 @@ import scala.util.Sorting import org.apache.spark.util.collection.{BitSet, OpenHashSet, PrimitiveVector} import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap private[graphx] class EdgePartitionBuilder[@specialized(Long, Int, Double) ED: ClassTag, VD: ClassTag]( @@ -41,7 +41,7 @@ class EdgePartitionBuilder[@specialized(Long, Int, Double) ED: ClassTag, VD: Cla val srcIds = new Array[VertexId](edgeArray.size) val dstIds = new Array[VertexId](edgeArray.size) val data = new Array[ED](edgeArray.size) - val index = new GraphXPrimitiveKeyOpenHashMap[VertexId, 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) { diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala index 56f79a7097fc..ebb0b9418d65 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala @@ -20,7 +20,7 @@ package org.apache.spark.graphx.impl import scala.reflect.ClassTag import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap /** * The Iterator type returned when constructing edge triplets. This could be an anonymous class in diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala index d02e9238adba..927e32ad0f44 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala @@ -25,7 +25,7 @@ import org.apache.spark.rdd.ShuffledRDD import org.apache.spark.util.collection.{BitSet, PrimitiveVector} import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap /** * A message from the edge partition `pid` to the vertex partition containing `vid` specifying that @@ -69,7 +69,7 @@ object RoutingTablePartition { : Iterator[RoutingTableMessage] = { // Determine which positions each vertex id appears in using a map where the low 2 bits // represent src and dst - val map = new GraphXPrimitiveKeyOpenHashMap[VertexId, Byte] + val map = new PrimitiveKeyOpenHashMap[VertexId, Byte] edgePartition.srcIds.iterator.foreach { srcId => map.changeValue(srcId, 0x1, (b: Byte) => (b | 0x1).toByte) } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala index dca54b8a7da8..f4e221d4e05a 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala @@ -22,7 +22,7 @@ import scala.reflect.ClassTag import org.apache.spark.util.collection.{BitSet, PrimitiveVector} import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap /** Stores vertex attributes to ship to an edge partition. */ private[graphx] diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala index 55c7a19d1bda..f1d174720a1b 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala @@ -22,7 +22,7 @@ import scala.reflect.ClassTag import org.apache.spark.util.collection.BitSet import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap private[graphx] object VertexPartition { /** Construct a `VertexPartition` from the given vertices. */ diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala index 34939b24440a..8d9e0204d27f 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala @@ -23,7 +23,7 @@ import scala.reflect.ClassTag import org.apache.spark.util.collection.BitSet import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap private[graphx] object VertexPartitionBase { /** @@ -32,7 +32,7 @@ private[graphx] object VertexPartitionBase { */ def initFrom[VD: ClassTag](iter: Iterator[(VertexId, VD)]) : (VertexIdToIndexMap, Array[VD], BitSet) = { - val map = new GraphXPrimitiveKeyOpenHashMap[VertexId, VD] + val map = new PrimitiveKeyOpenHashMap[VertexId, VD] iter.foreach { pair => map(pair._1) = pair._2 } @@ -45,7 +45,7 @@ private[graphx] object VertexPartitionBase { */ def initFrom[VD: ClassTag](iter: Iterator[(VertexId, VD)], mergeFunc: (VD, VD) => VD) : (VertexIdToIndexMap, Array[VD], BitSet) = { - val map = new GraphXPrimitiveKeyOpenHashMap[VertexId, VD] + val map = new PrimitiveKeyOpenHashMap[VertexId, VD] iter.foreach { pair => map.setMerge(pair._1, pair._2, mergeFunc) } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala index a4f769b29401..21ff615feca6 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala @@ -25,7 +25,7 @@ import org.apache.spark.Logging import org.apache.spark.util.collection.BitSet import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap /** * An class containing additional operations for subclasses of VertexPartitionBase that provide @@ -224,7 +224,7 @@ private[graphx] abstract class VertexPartitionBaseOps * Construct a new VertexPartition whose index contains only the vertices in the mask. */ def reindex(): Self[VD] = { - val hashMap = new GraphXPrimitiveKeyOpenHashMap[VertexId, VD] + val hashMap = new PrimitiveKeyOpenHashMap[VertexId, VD] val arbitraryMerge = (a: VD, b: VD) => a for ((k, v) <- self.iterator) { hashMap.setMerge(k, v, arbitraryMerge) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/util/collection/GraphXPrimitiveKeyOpenHashMap.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/collection/PrimitiveKeyOpenHashMap.scala similarity index 98% rename from graphx/src/main/scala/org/apache/spark/graphx/util/collection/GraphXPrimitiveKeyOpenHashMap.scala rename to graphx/src/main/scala/org/apache/spark/graphx/util/collection/PrimitiveKeyOpenHashMap.scala index 57b01b6f2e1f..7b02e2ed1a9c 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/util/collection/GraphXPrimitiveKeyOpenHashMap.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/util/collection/PrimitiveKeyOpenHashMap.scala @@ -29,7 +29,7 @@ import scala.reflect._ * Under the hood, it uses our OpenHashSet implementation. */ private[graphx] -class GraphXPrimitiveKeyOpenHashMap[@specialized(Long, Int) K: ClassTag, +class PrimitiveKeyOpenHashMap[@specialized(Long, Int) K: ClassTag, @specialized(Long, Int, Double) V: ClassTag]( val keySet: OpenHashSet[K], var _values: Array[V]) extends Iterable[(K, V)] diff --git a/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala index 28fd112f2b12..d2e0c01bc35e 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala @@ -22,9 +22,6 @@ import scala.util.Random import org.scalatest.FunSuite -import org.apache.spark.SparkConf -import org.apache.spark.serializer.KryoSerializer - import org.apache.spark.graphx._ class EdgePartitionSuite extends FunSuite { @@ -123,19 +120,4 @@ class EdgePartitionSuite extends FunSuite { assert(!ep.isActive(-1)) assert(ep.numActives == Some(2)) } - - test("Kryo serialization") { - val aList = List((0, 1, 0), (1, 0, 0), (1, 2, 0), (5, 4, 0), (5, 5, 0)) - val a: EdgePartition[Int, Int] = makeEdgePartition(aList) - val conf = new SparkConf() - .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - .set("spark.kryo.registrator", "org.apache.spark.graphx.GraphKryoRegistrator") - val s = new KryoSerializer(conf).newInstance() - val aSer: EdgePartition[Int, Int] = s.deserialize(s.serialize(a)) - assert(aSer.srcIds.toList === a.srcIds.toList) - assert(aSer.dstIds.toList === a.dstIds.toList) - assert(aSer.data.toList === a.data.toList) - assert(aSer.index != null) - assert(aSer.vertices.iterator.toSet === a.vertices.iterator.toSet) - } } From 3af1f386439cdddd42e545ad63d089f4dfdf9f8a Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Mon, 12 May 2014 11:08:52 -0700 Subject: [PATCH 12/18] SPARK-1772 Stop catching Throwable, let Executors die The main issue this patch fixes is [SPARK-1772](https://issues.apache.org/jira/browse/SPARK-1772), in which Executors may not die when fatal exceptions (e.g., OOM) are thrown. This patch causes Executors to delegate to the ExecutorUncaughtExceptionHandler when a fatal exception is thrown. This patch also continues the fight in the neverending war against `case t: Throwable =>`, by only catching Exceptions in many places, and adding a wrapper for Threads and Runnables to make sure any uncaught exceptions are at least printed to the logs. It also turns out that it is unlikely that the IndestructibleActorSystem actually works, given testing ([here](https://gist.github.com/aarondav/ca1f0cdcd50727f89c0d)). The uncaughtExceptionHandler is not called from the places that we expected it would be. [SPARK-1620](https://issues.apache.org/jira/browse/SPARK-1620) deals with part of this issue, but refactoring our Actor Systems to ensure that exceptions are dealt with properly is a much bigger change, outside the scope of this PR. Author: Aaron Davidson Closes #715 from aarondav/throwable and squashes the following commits: f9b9bfe [Aaron Davidson] Remove other redundant 'throw e' e937a0a [Aaron Davidson] Address Prashant and Matei's comments 1867867 [Aaron Davidson] [RFC] SPARK-1772 Stop catching Throwable, let Executors die --- .../org/apache/spark/ContextCleaner.scala | 11 +-- .../scala/org/apache/spark/SparkContext.scala | 12 ++-- .../apache/spark/api/python/PythonRDD.scala | 3 +- .../api/python/PythonWorkerFactory.scala | 1 - .../org/apache/spark/deploy/Client.scala | 2 +- .../apache/spark/deploy/SparkHadoopUtil.scala | 2 +- .../spark/deploy/history/HistoryServer.scala | 8 +-- .../apache/spark/deploy/master/Master.scala | 4 +- .../spark/deploy/worker/DriverWrapper.scala | 2 +- .../CoarseGrainedExecutorBackend.scala | 2 +- .../org/apache/spark/executor/Executor.scala | 37 +++------- .../ExecutorUncaughtExceptionHandler.scala | 53 +++++++++++++++ .../scheduler/EventLoggingListener.scala | 4 +- .../spark/scheduler/TaskResultGetter.scala | 8 +-- .../spark/storage/DiskBlockManager.scala | 6 +- .../spark/storage/TachyonBlockManager.scala | 7 +- .../org/apache/spark/util/AkkaUtils.scala | 11 +-- .../util/IndestructibleActorSystem.scala | 68 ------------------- .../scala/org/apache/spark/util/Utils.scala | 26 ++++++- 19 files changed, 127 insertions(+), 140 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/executor/ExecutorUncaughtExceptionHandler.scala delete mode 100644 core/src/main/scala/org/apache/spark/util/IndestructibleActorSystem.scala diff --git a/core/src/main/scala/org/apache/spark/ContextCleaner.scala b/core/src/main/scala/org/apache/spark/ContextCleaner.scala index 54e08d7866f7..e2d2250982da 100644 --- a/core/src/main/scala/org/apache/spark/ContextCleaner.scala +++ b/core/src/main/scala/org/apache/spark/ContextCleaner.scala @@ -23,6 +23,7 @@ import scala.collection.mutable.{ArrayBuffer, SynchronizedBuffer} import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD +import org.apache.spark.util.Utils /** * Classes that represent cleaning tasks. @@ -110,7 +111,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { } /** Keep cleaning RDD, shuffle, and broadcast state. */ - private def keepCleaning() { + private def keepCleaning(): Unit = Utils.logUncaughtExceptions { while (!stopped) { try { val reference = Option(referenceQueue.remove(ContextCleaner.REF_QUEUE_POLL_TIMEOUT)) @@ -128,7 +129,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { } } } catch { - case t: Throwable => logError("Error in cleaning thread", t) + case e: Exception => logError("Error in cleaning thread", e) } } } @@ -141,7 +142,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { listeners.foreach(_.rddCleaned(rddId)) logInfo("Cleaned RDD " + rddId) } catch { - case t: Throwable => logError("Error cleaning RDD " + rddId, t) + case e: Exception => logError("Error cleaning RDD " + rddId, e) } } @@ -154,7 +155,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { listeners.foreach(_.shuffleCleaned(shuffleId)) logInfo("Cleaned shuffle " + shuffleId) } catch { - case t: Throwable => logError("Error cleaning shuffle " + shuffleId, t) + case e: Exception => logError("Error cleaning shuffle " + shuffleId, e) } } @@ -166,7 +167,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { listeners.foreach(_.broadcastCleaned(broadcastId)) logInfo("Cleaned broadcast " + broadcastId) } catch { - case t: Throwable => logError("Error cleaning broadcast " + broadcastId, t) + case e: Exception => logError("Error cleaning broadcast " + broadcastId, e) } } diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 71bab295442f..e6121a705497 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1494,8 +1494,8 @@ object SparkContext extends Logging { } catch { // TODO: Enumerate the exact reasons why it can fail // But irrespective of it, it means we cannot proceed ! - case th: Throwable => { - throw new SparkException("YARN mode not available ?", th) + case e: Exception => { + throw new SparkException("YARN mode not available ?", e) } } val backend = new CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem) @@ -1510,8 +1510,8 @@ object SparkContext extends Logging { cons.newInstance(sc).asInstanceOf[TaskSchedulerImpl] } catch { - case th: Throwable => { - throw new SparkException("YARN mode not available ?", th) + case e: Exception => { + throw new SparkException("YARN mode not available ?", e) } } @@ -1521,8 +1521,8 @@ object SparkContext extends Logging { val cons = clazz.getConstructor(classOf[TaskSchedulerImpl], classOf[SparkContext]) cons.newInstance(scheduler, sc).asInstanceOf[CoarseGrainedSchedulerBackend] } catch { - case th: Throwable => { - throw new SparkException("YARN mode not available ?", th) + case e: Exception => { + throw new SparkException("YARN mode not available ?", e) } } diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index 2971c277aa86..57b28b997236 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -171,7 +171,7 @@ private[spark] class PythonRDD[T: ClassTag]( this.interrupt() } - override def run() { + override def run(): Unit = Utils.logUncaughtExceptions { try { SparkEnv.set(env) val stream = new BufferedOutputStream(worker.getOutputStream, bufferSize) @@ -282,7 +282,6 @@ private[spark] object PythonRDD { } } catch { case eof: EOFException => {} - case e: Throwable => throw e } JavaRDD.fromRDD(sc.sc.parallelize(objs, parallelism)) } diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala index 002f2acd94de..759cbe2c46c5 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala @@ -71,7 +71,6 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String stopDaemon() startDaemon() new Socket(daemonHost, daemonPort) - case e: Throwable => throw e } } } diff --git a/core/src/main/scala/org/apache/spark/deploy/Client.scala b/core/src/main/scala/org/apache/spark/deploy/Client.scala index 7ead1171525d..aeb159adc31d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/Client.scala +++ b/core/src/main/scala/org/apache/spark/deploy/Client.scala @@ -157,7 +157,7 @@ object Client { // TODO: See if we can initialize akka so return messages are sent back using the same TCP // flow. Else, this (sadly) requires the DriverClient be routable from the Master. val (actorSystem, _) = AkkaUtils.createActorSystem( - "driverClient", Utils.localHostName(), 0, false, conf, new SecurityManager(conf)) + "driverClient", Utils.localHostName(), 0, conf, new SecurityManager(conf)) actorSystem.actorOf(Props(classOf[ClientActor], driverArgs, conf)) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index e2df1b895412..148115d3ed35 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -103,7 +103,7 @@ object SparkHadoopUtil { .newInstance() .asInstanceOf[SparkHadoopUtil] } catch { - case th: Throwable => throw new SparkException("Unable to load YARN support", th) + case e: Exception => throw new SparkException("Unable to load YARN support", e) } } else { new SparkHadoopUtil diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index 1238bbf9da2f..a9c11dca5678 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -70,7 +70,7 @@ class HistoryServer( * TODO: Add a mechanism to update manually. */ private val logCheckingThread = new Thread { - override def run() { + override def run(): Unit = Utils.logUncaughtExceptions { while (!stopped) { val now = System.currentTimeMillis if (now - lastLogCheckTime > UPDATE_INTERVAL_MS) { @@ -154,7 +154,7 @@ class HistoryServer( numCompletedApplications = logInfos.size } catch { - case t: Throwable => logError("Exception in checking for event log updates", t) + case e: Exception => logError("Exception in checking for event log updates", e) } } else { logWarning("Attempted to check for event log updates before binding the server.") @@ -231,8 +231,8 @@ class HistoryServer( dir.getModificationTime } } catch { - case t: Throwable => - logError("Exception in accessing modification time of %s".format(dir.getPath), t) + case e: Exception => + logError("Exception in accessing modification time of %s".format(dir.getPath), e) -1L } } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index f254f5585ba2..c6dec305bffc 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -684,8 +684,8 @@ private[spark] class Master( webUi.attachSparkUI(ui) return true } catch { - case t: Throwable => - logError("Exception in replaying log for application %s (%s)".format(appName, app.id), t) + case e: Exception => + logError("Exception in replaying log for application %s (%s)".format(appName, app.id), e) } } else { logWarning("Application %s (%s) has no valid logs: %s".format(appName, app.id, eventLogDir)) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala index be15138f6240..05e242e6df70 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala @@ -31,7 +31,7 @@ object DriverWrapper { case workerUrl :: mainClass :: extraArgs => val conf = new SparkConf() val (actorSystem, _) = AkkaUtils.createActorSystem("Driver", - Utils.localHostName(), 0, false, conf, new SecurityManager(conf)) + Utils.localHostName(), 0, conf, new SecurityManager(conf)) actorSystem.actorOf(Props(classOf[WorkerWatcher], workerUrl), name = "workerWatcher") // Delegate to supplied main class diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index e912ae8a5d3c..84aec65b7765 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -105,7 +105,7 @@ private[spark] object CoarseGrainedExecutorBackend { // Create a new ActorSystem to run the backend, because we can't create a // SparkEnv / Executor before getting started with all our system properties, etc val (actorSystem, boundPort) = AkkaUtils.createActorSystem("sparkExecutor", hostname, 0, - indestructible = true, conf = conf, new SecurityManager(conf)) + conf, new SecurityManager(conf)) // set it val sparkHostPort = hostname + ":" + boundPort actorSystem.actorOf( diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 98e7e0be813b..baee7a216a7c 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -74,28 +74,7 @@ private[spark] class Executor( // Setup an uncaught exception handler for non-local mode. // Make any thread terminations due to uncaught exceptions kill the entire // executor process to avoid surprising stalls. - Thread.setDefaultUncaughtExceptionHandler( - new Thread.UncaughtExceptionHandler { - override def uncaughtException(thread: Thread, exception: Throwable) { - try { - logError("Uncaught exception in thread " + thread, exception) - - // We may have been called from a shutdown hook. If so, we must not call System.exit(). - // (If we do, we will deadlock.) - if (!Utils.inShutdown()) { - if (exception.isInstanceOf[OutOfMemoryError]) { - System.exit(ExecutorExitCode.OOM) - } else { - System.exit(ExecutorExitCode.UNCAUGHT_EXCEPTION) - } - } - } catch { - case oom: OutOfMemoryError => Runtime.getRuntime.halt(ExecutorExitCode.OOM) - case t: Throwable => Runtime.getRuntime.halt(ExecutorExitCode.UNCAUGHT_EXCEPTION_TWICE) - } - } - } - ) + Thread.setDefaultUncaughtExceptionHandler(ExecutorUncaughtExceptionHandler) } val executorSource = new ExecutorSource(this, executorId) @@ -259,6 +238,11 @@ private[spark] class Executor( } case t: Throwable => { + // Attempt to exit cleanly by informing the driver of our failure. + // If anything goes wrong (or this was a fatal exception), we will delegate to + // the default uncaught exception handler, which will terminate the Executor. + logError("Exception in task ID " + taskId, t) + val serviceTime = System.currentTimeMillis() - taskStart val metrics = attemptedTask.flatMap(t => t.metrics) for (m <- metrics) { @@ -268,10 +252,11 @@ private[spark] class Executor( val reason = ExceptionFailure(t.getClass.getName, t.toString, t.getStackTrace, metrics) execBackend.statusUpdate(taskId, TaskState.FAILED, ser.serialize(reason)) - // TODO: Should we exit the whole executor here? On the one hand, the failed task may - // have left some weird state around depending on when the exception was thrown, but on - // the other hand, maybe we could detect that when future tasks fail and exit then. - logError("Exception in task ID " + taskId, t) + // Don't forcibly exit unless the exception was inherently fatal, to avoid + // stopping other tasks unnecessarily. + if (Utils.isFatalError(t)) { + ExecutorUncaughtExceptionHandler.uncaughtException(t) + } } } finally { // TODO: Unregister shuffle memory only for ResultTask diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorUncaughtExceptionHandler.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorUncaughtExceptionHandler.scala new file mode 100644 index 000000000000..b0e984c03964 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorUncaughtExceptionHandler.scala @@ -0,0 +1,53 @@ +/* + * 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.executor + +import org.apache.spark.Logging +import org.apache.spark.util.Utils + +/** + * The default uncaught exception handler for Executors terminates the whole process, to avoid + * getting into a bad state indefinitely. Since Executors are relatively lightweight, it's better + * to fail fast when things go wrong. + */ +private[spark] object ExecutorUncaughtExceptionHandler + extends Thread.UncaughtExceptionHandler with Logging { + + override def uncaughtException(thread: Thread, exception: Throwable) { + try { + logError("Uncaught exception in thread " + thread, exception) + + // We may have been called from a shutdown hook. If so, we must not call System.exit(). + // (If we do, we will deadlock.) + if (!Utils.inShutdown()) { + if (exception.isInstanceOf[OutOfMemoryError]) { + System.exit(ExecutorExitCode.OOM) + } else { + System.exit(ExecutorExitCode.UNCAUGHT_EXCEPTION) + } + } + } catch { + case oom: OutOfMemoryError => Runtime.getRuntime.halt(ExecutorExitCode.OOM) + case t: Throwable => Runtime.getRuntime.halt(ExecutorExitCode.UNCAUGHT_EXCEPTION_TWICE) + } + } + + def uncaughtException(exception: Throwable) { + uncaughtException(Thread.currentThread(), exception) + } +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index 7968a0691db1..a90b0d475c04 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -206,8 +206,8 @@ private[spark] object EventLoggingListener extends Logging { applicationComplete = filePaths.exists { path => isApplicationCompleteFile(path.getName) } ) } catch { - case t: Throwable => - logError("Exception in parsing logging info from directory %s".format(logDir), t) + case e: Exception => + logError("Exception in parsing logging info from directory %s".format(logDir), e) EventLoggingInfo.empty } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala index c9ad2b151daf..99d305b36a95 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala @@ -43,7 +43,7 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul def enqueueSuccessfulTask( taskSetManager: TaskSetManager, tid: Long, serializedData: ByteBuffer) { getTaskResultExecutor.execute(new Runnable { - override def run() { + override def run(): Unit = Utils.logUncaughtExceptions { try { val result = serializer.get().deserialize[TaskResult[_]](serializedData) match { case directResult: DirectTaskResult[_] => directResult @@ -70,7 +70,7 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul case cnf: ClassNotFoundException => val loader = Thread.currentThread.getContextClassLoader taskSetManager.abort("ClassNotFound with classloader: " + loader) - case ex: Throwable => + case ex: Exception => taskSetManager.abort("Exception while deserializing and fetching task: %s".format(ex)) } } @@ -81,7 +81,7 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul serializedData: ByteBuffer) { var reason : TaskEndReason = UnknownReason getTaskResultExecutor.execute(new Runnable { - override def run() { + override def run(): Unit = Utils.logUncaughtExceptions { try { if (serializedData != null && serializedData.limit() > 0) { reason = serializer.get().deserialize[TaskEndReason]( @@ -94,7 +94,7 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul val loader = Utils.getContextOrSparkClassLoader logError( "Could not deserialize TaskEndReason: ClassNotFound with classloader " + loader) - case ex: Throwable => {} + case ex: Exception => {} } scheduler.handleFailedTask(taskSetManager, tid, taskState, reason) } diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala index cf6ef0029a86..3a7243a1ba19 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -148,7 +148,7 @@ private[spark] class DiskBlockManager(shuffleManager: ShuffleBlockManager, rootD private def addShutdownHook() { localDirs.foreach(localDir => Utils.registerShutdownDeleteDir(localDir)) Runtime.getRuntime.addShutdownHook(new Thread("delete Spark local dirs") { - override def run() { + override def run(): Unit = Utils.logUncaughtExceptions { logDebug("Shutdown hook called") DiskBlockManager.this.stop() } @@ -162,8 +162,8 @@ private[spark] class DiskBlockManager(shuffleManager: ShuffleBlockManager, rootD try { if (!Utils.hasRootAsShutdownDeleteDir(localDir)) Utils.deleteRecursively(localDir) } catch { - case t: Throwable => - logError("Exception while deleting local spark dir: " + localDir, t) + case e: Exception => + logError("Exception while deleting local spark dir: " + localDir, e) } } } diff --git a/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala index b0b967485656..a6cbe3aa440f 100644 --- a/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala @@ -25,7 +25,6 @@ import tachyon.client.TachyonFile import org.apache.spark.Logging import org.apache.spark.executor.ExecutorExitCode -import org.apache.spark.network.netty.ShuffleSender import org.apache.spark.util.Utils @@ -137,7 +136,7 @@ private[spark] class TachyonBlockManager( private def addShutdownHook() { tachyonDirs.foreach(tachyonDir => Utils.registerShutdownDeleteDir(tachyonDir)) Runtime.getRuntime.addShutdownHook(new Thread("delete Spark tachyon dirs") { - override def run() { + override def run(): Unit = Utils.logUncaughtExceptions { logDebug("Shutdown hook called") tachyonDirs.foreach { tachyonDir => try { @@ -145,8 +144,8 @@ private[spark] class TachyonBlockManager( Utils.deleteRecursively(tachyonDir, client) } } catch { - case t: Throwable => - logError("Exception while deleting tachyon spark dir: " + tachyonDir, t) + case e: Exception => + logError("Exception while deleting tachyon spark dir: " + tachyonDir, e) } } } diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala index 8afe09a117eb..a8d12bb2a016 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -20,7 +20,7 @@ package org.apache.spark.util import scala.collection.JavaConversions.mapAsJavaMap import scala.concurrent.duration.{Duration, FiniteDuration} -import akka.actor.{ActorSystem, ExtendedActorSystem, IndestructibleActorSystem} +import akka.actor.{ActorSystem, ExtendedActorSystem} import com.typesafe.config.ConfigFactory import org.apache.log4j.{Level, Logger} @@ -41,7 +41,7 @@ private[spark] object AkkaUtils extends Logging { * If indestructible is set to true, the Actor System will continue running in the event * of a fatal exception. This is used by [[org.apache.spark.executor.Executor]]. */ - def createActorSystem(name: String, host: String, port: Int, indestructible: Boolean = false, + def createActorSystem(name: String, host: String, port: Int, conf: SparkConf, securityManager: SecurityManager): (ActorSystem, Int) = { val akkaThreads = conf.getInt("spark.akka.threads", 4) @@ -101,12 +101,7 @@ private[spark] object AkkaUtils extends Logging { |akka.log-dead-letters-during-shutdown = $lifecycleEvents """.stripMargin)) - val actorSystem = if (indestructible) { - IndestructibleActorSystem(name, akkaConf) - } else { - ActorSystem(name, akkaConf) - } - + val actorSystem = ActorSystem(name, akkaConf) val provider = actorSystem.asInstanceOf[ExtendedActorSystem].provider val boundPort = provider.getDefaultAddress.port.get (actorSystem, boundPort) diff --git a/core/src/main/scala/org/apache/spark/util/IndestructibleActorSystem.scala b/core/src/main/scala/org/apache/spark/util/IndestructibleActorSystem.scala deleted file mode 100644 index 4188a869c13d..000000000000 --- a/core/src/main/scala/org/apache/spark/util/IndestructibleActorSystem.scala +++ /dev/null @@ -1,68 +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. - */ - -// Must be in akka.actor package as ActorSystemImpl is protected[akka]. -package akka.actor - -import scala.util.control.{ControlThrowable, NonFatal} - -import com.typesafe.config.Config - -/** - * An akka.actor.ActorSystem which refuses to shut down in the event of a fatal exception - * This is necessary as Spark Executors are allowed to recover from fatal exceptions - * (see org.apache.spark.executor.Executor) - */ -object IndestructibleActorSystem { - def apply(name: String, config: Config): ActorSystem = - apply(name, config, ActorSystem.findClassLoader()) - - def apply(name: String, config: Config, classLoader: ClassLoader): ActorSystem = - new IndestructibleActorSystemImpl(name, config, classLoader).start() -} - -private[akka] class IndestructibleActorSystemImpl( - override val name: String, - applicationConfig: Config, - classLoader: ClassLoader) - extends ActorSystemImpl(name, applicationConfig, classLoader) { - - protected override def uncaughtExceptionHandler: Thread.UncaughtExceptionHandler = { - val fallbackHandler = super.uncaughtExceptionHandler - - new Thread.UncaughtExceptionHandler() { - def uncaughtException(thread: Thread, cause: Throwable): Unit = { - if (isFatalError(cause) && !settings.JvmExitOnFatalError) { - log.error(cause, "Uncaught fatal error from thread [{}] not shutting down " + - "ActorSystem [{}] tolerating and continuing.... ", thread.getName, name) - // shutdown() //TODO make it configurable - } else { - fallbackHandler.uncaughtException(thread, cause) - } - } - } - } - - def isFatalError(e: Throwable): Boolean = { - e match { - case NonFatal(_) | _: InterruptedException | _: NotImplementedError | _: ControlThrowable => - false - case _ => - true - } - } -} diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 95777fbf57d8..8f7594ada2ba 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -29,6 +29,7 @@ import scala.collection.mutable.ArrayBuffer import scala.io.Source import scala.reflect.ClassTag import scala.util.Try +import scala.util.control.{ControlThrowable, NonFatal} import com.google.common.io.Files import com.google.common.util.concurrent.ThreadFactoryBuilder @@ -41,7 +42,6 @@ import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkException} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance} - /** * Various utility methods used by Spark. */ @@ -1125,4 +1125,28 @@ private[spark] object Utils extends Logging { } } + /** + * Executes the given block, printing and re-throwing any uncaught exceptions. + * This is particularly useful for wrapping code that runs in a thread, to ensure + * that exceptions are printed, and to avoid having to catch Throwable. + */ + def logUncaughtExceptions[T](f: => T): T = { + try { + f + } catch { + case t: Throwable => + logError(s"Uncaught exception in thread ${Thread.currentThread().getName}", t) + throw t + } + } + + /** Returns true if the given exception was fatal. See docs for scala.util.control.NonFatal. */ + def isFatalError(e: Throwable): Boolean = { + e match { + case NonFatal(_) | _: InterruptedException | _: NotImplementedError | _: ControlThrowable => + false + case _ => + true + } + } } From d9c97ba39723d36d3dcb53a75a3a2565357c0955 Mon Sep 17 00:00:00 2001 From: Bernardo Gomez Palacio Date: Mon, 12 May 2014 11:10:28 -0700 Subject: [PATCH 13/18] SPARK-1806: Upgrade Mesos dependency to 0.18.1 Enabled Mesos (0.18.1) dependency with shaded protobuf Why is this needed? Avoids any protobuf version collision between Mesos and any other dependency in Spark e.g. Hadoop HDFS 2.2+ or 1.0.4. Ticket: https://issues.apache.org/jira/browse/SPARK-1806 * Should close https://issues.apache.org/jira/browse/SPARK-1433 Author berngp Author: Bernardo Gomez Palacio Closes #741 from berngp/feature/SPARK-1806 and squashes the following commits: 5d70646 [Bernardo Gomez Palacio] SPARK-1806: Upgrade Mesos dependency to 0.18.1 --- core/pom.xml | 3 ++- .../apache/spark/executor/MesosExecutorBackend.scala | 2 +- .../cluster/mesos/MesosSchedulerBackend.scala | 2 +- pom.xml | 10 +++++++++- project/SparkBuild.scala | 2 +- 5 files changed, 14 insertions(+), 5 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index 8fe215ab2428..bab50f5ce288 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -140,6 +140,7 @@ org.apache.mesos mesos + ${mesos.classifier} io.netty @@ -322,7 +323,7 @@ - + src/main/resources diff --git a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala index 9b56f711e0e0..74100498bb2b 100644 --- a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala @@ -19,7 +19,7 @@ package org.apache.spark.executor import java.nio.ByteBuffer -import com.google.protobuf.ByteString +import org.apache.mesos.protobuf.ByteString import org.apache.mesos.{Executor => MesosExecutor, ExecutorDriver, MesosExecutorDriver, MesosNativeLibrary} import org.apache.mesos.Protos.{TaskStatus => MesosTaskStatus, _} diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index c975f312324e..2cea1c8cf154 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -24,7 +24,7 @@ import java.util.Collections import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} -import com.google.protobuf.ByteString +import org.apache.mesos.protobuf.ByteString import org.apache.mesos.{Scheduler => MScheduler} import org.apache.mesos._ import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, TaskState => MesosTaskState, _} diff --git a/pom.xml b/pom.xml index c4e1c6be52a1..dd1d262881b9 100644 --- a/pom.xml +++ b/pom.xml @@ -113,7 +113,8 @@ 2.10.4 2.10 - 0.13.0 + 0.18.1 + shaded-protobuf org.spark-project.akka 2.2.3-shaded-protobuf 1.7.5 @@ -349,6 +350,13 @@ org.apache.mesos mesos ${mesos.version} + ${mesos.classifier} + + + com.google.protobuf + protobuf-java + + commons-net diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 12791e490ae2..57b3e22f8171 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -343,7 +343,7 @@ object SparkBuild extends Build { "org.spark-project.akka" %% "akka-testkit" % akkaVersion % "test", "org.json4s" %% "json4s-jackson" % "3.2.6" excludeAll(excludeScalap), "colt" % "colt" % "1.2.0", - "org.apache.mesos" % "mesos" % "0.13.0", + "org.apache.mesos" % "mesos" % "0.18.1" classifier("shaded-protobuf") exclude("com.google.protobuf", "protobuf-java"), "commons-net" % "commons-net" % "2.2", "net.java.dev.jets3t" % "jets3t" % jets3tVersion excludeAll(excludeCommonsLogging), "org.apache.derby" % "derby" % "10.4.2.0" % "test", From a4a3faa0599f8a4e2238aaec37ce5c5b2cdd0292 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Fri, 9 May 2014 18:54:56 -0700 Subject: [PATCH 14/18] Making EdgePartition serializable. --- .../apache/spark/graphx/GraphKryoRegistrator.scala | 7 ++++++- .../apache/spark/graphx/impl/EdgePartition.scala | 14 +++++++------- .../spark/graphx/impl/EdgePartitionBuilder.scala | 4 ++-- .../spark/graphx/impl/EdgeTripletIterator.scala | 2 +- .../spark/graphx/impl/RoutingTablePartition.scala | 4 ++-- .../graphx/impl/ShippableVertexPartition.scala | 2 +- .../apache/spark/graphx/impl/VertexPartition.scala | 2 +- .../spark/graphx/impl/VertexPartitionBase.scala | 6 +++--- .../spark/graphx/impl/VertexPartitionBaseOps.scala | 4 ++-- ...p.scala => GraphXPrimitiveKeyOpenHashMap.scala} | 2 +- 10 files changed, 26 insertions(+), 21 deletions(-) rename graphx/src/main/scala/org/apache/spark/graphx/util/collection/{PrimitiveKeyOpenHashMap.scala => GraphXPrimitiveKeyOpenHashMap.scala} (98%) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala index d295d0127ac7..e80217b196d3 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala @@ -24,6 +24,9 @@ import org.apache.spark.util.BoundedPriorityQueue import org.apache.spark.util.collection.BitSet import org.apache.spark.graphx.impl._ +import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap +import org.apache.spark.util.collection.OpenHashSet + /** * Registers GraphX classes with Kryo for improved performance. @@ -43,7 +46,9 @@ class GraphKryoRegistrator extends KryoRegistrator { kryo.register(classOf[PartitionStrategy]) kryo.register(classOf[BoundedPriorityQueue[Object]]) kryo.register(classOf[EdgeDirection]) - + kryo.register(classOf[GraphXPrimitiveKeyOpenHashMap[VertexId, Int]]) + kryo.register(classOf[OpenHashSet[Int]]) + kryo.register(classOf[OpenHashSet[Long]]) // This avoids a large number of hash table lookups. kryo.setReferences(false) } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala index 871e81f8d245..a5c9cd1f8b4e 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala @@ -20,7 +20,7 @@ package org.apache.spark.graphx.impl import scala.reflect.{classTag, ClassTag} import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap /** * A collection of edges stored in columnar format, along with any vertex attributes referenced. The @@ -42,12 +42,12 @@ import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap private[graphx] class EdgePartition[ @specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED: ClassTag, VD: ClassTag]( - @transient val srcIds: Array[VertexId], - @transient val dstIds: Array[VertexId], - @transient val data: Array[ED], - @transient val index: PrimitiveKeyOpenHashMap[VertexId, Int], - @transient val vertices: VertexPartition[VD], - @transient val activeSet: Option[VertexSet] = None + val srcIds: Array[VertexId] = null, + val dstIds: Array[VertexId] = null, + val data: Array[ED] = null, + val index: GraphXPrimitiveKeyOpenHashMap[VertexId, Int] = null, + val vertices: VertexPartition[VD] = null, + val activeSet: Option[VertexSet] = None ) extends Serializable { /** Return a new `EdgePartition` with the specified edge data. */ diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala index ecb49bef42e4..4520beb99151 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala @@ -23,7 +23,7 @@ import scala.util.Sorting import org.apache.spark.util.collection.{BitSet, OpenHashSet, PrimitiveVector} import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap private[graphx] class EdgePartitionBuilder[@specialized(Long, Int, Double) ED: ClassTag, VD: ClassTag]( @@ -41,7 +41,7 @@ class EdgePartitionBuilder[@specialized(Long, Int, Double) ED: ClassTag, VD: Cla 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[VertexId, Int] + val index = new GraphXPrimitiveKeyOpenHashMap[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) { diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala index ebb0b9418d65..56f79a7097fc 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala @@ -20,7 +20,7 @@ package org.apache.spark.graphx.impl import scala.reflect.ClassTag import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap /** * The Iterator type returned when constructing edge triplets. This could be an anonymous class in diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala index 927e32ad0f44..d02e9238adba 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala @@ -25,7 +25,7 @@ import org.apache.spark.rdd.ShuffledRDD import org.apache.spark.util.collection.{BitSet, PrimitiveVector} import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap /** * A message from the edge partition `pid` to the vertex partition containing `vid` specifying that @@ -69,7 +69,7 @@ object RoutingTablePartition { : Iterator[RoutingTableMessage] = { // Determine which positions each vertex id appears in using a map where the low 2 bits // represent src and dst - val map = new PrimitiveKeyOpenHashMap[VertexId, Byte] + val map = new GraphXPrimitiveKeyOpenHashMap[VertexId, Byte] edgePartition.srcIds.iterator.foreach { srcId => map.changeValue(srcId, 0x1, (b: Byte) => (b | 0x1).toByte) } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala index f4e221d4e05a..dca54b8a7da8 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala @@ -22,7 +22,7 @@ import scala.reflect.ClassTag import org.apache.spark.util.collection.{BitSet, PrimitiveVector} import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap /** Stores vertex attributes to ship to an edge partition. */ private[graphx] diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala index f1d174720a1b..55c7a19d1bda 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala @@ -22,7 +22,7 @@ import scala.reflect.ClassTag import org.apache.spark.util.collection.BitSet import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap private[graphx] object VertexPartition { /** Construct a `VertexPartition` from the given vertices. */ diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala index 8d9e0204d27f..34939b24440a 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala @@ -23,7 +23,7 @@ import scala.reflect.ClassTag import org.apache.spark.util.collection.BitSet import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap private[graphx] object VertexPartitionBase { /** @@ -32,7 +32,7 @@ private[graphx] object VertexPartitionBase { */ def initFrom[VD: ClassTag](iter: Iterator[(VertexId, VD)]) : (VertexIdToIndexMap, Array[VD], BitSet) = { - val map = new PrimitiveKeyOpenHashMap[VertexId, VD] + val map = new GraphXPrimitiveKeyOpenHashMap[VertexId, VD] iter.foreach { pair => map(pair._1) = pair._2 } @@ -45,7 +45,7 @@ private[graphx] object VertexPartitionBase { */ def initFrom[VD: ClassTag](iter: Iterator[(VertexId, VD)], mergeFunc: (VD, VD) => VD) : (VertexIdToIndexMap, Array[VD], BitSet) = { - val map = new PrimitiveKeyOpenHashMap[VertexId, VD] + val map = new GraphXPrimitiveKeyOpenHashMap[VertexId, VD] iter.foreach { pair => map.setMerge(pair._1, pair._2, mergeFunc) } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala index 21ff615feca6..a4f769b29401 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala @@ -25,7 +25,7 @@ import org.apache.spark.Logging import org.apache.spark.util.collection.BitSet import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap /** * An class containing additional operations for subclasses of VertexPartitionBase that provide @@ -224,7 +224,7 @@ private[graphx] abstract class VertexPartitionBaseOps * Construct a new VertexPartition whose index contains only the vertices in the mask. */ def reindex(): Self[VD] = { - val hashMap = new PrimitiveKeyOpenHashMap[VertexId, VD] + val hashMap = new GraphXPrimitiveKeyOpenHashMap[VertexId, VD] val arbitraryMerge = (a: VD, b: VD) => a for ((k, v) <- self.iterator) { hashMap.setMerge(k, v, arbitraryMerge) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/util/collection/PrimitiveKeyOpenHashMap.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/collection/GraphXPrimitiveKeyOpenHashMap.scala similarity index 98% rename from graphx/src/main/scala/org/apache/spark/graphx/util/collection/PrimitiveKeyOpenHashMap.scala rename to graphx/src/main/scala/org/apache/spark/graphx/util/collection/GraphXPrimitiveKeyOpenHashMap.scala index 7b02e2ed1a9c..57b01b6f2e1f 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/util/collection/PrimitiveKeyOpenHashMap.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/util/collection/GraphXPrimitiveKeyOpenHashMap.scala @@ -29,7 +29,7 @@ import scala.reflect._ * Under the hood, it uses our OpenHashSet implementation. */ private[graphx] -class PrimitiveKeyOpenHashMap[@specialized(Long, Int) K: ClassTag, +class GraphXPrimitiveKeyOpenHashMap[@specialized(Long, Int) K: ClassTag, @specialized(Long, Int, Double) V: ClassTag]( val keySet: OpenHashSet[K], var _values: Array[V]) extends Iterable[(K, V)] From c0a9ae599015f6484d0a7918981d9b319a99f529 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 9 May 2014 20:09:48 -0700 Subject: [PATCH 15/18] Add failing test for EdgePartition Kryo serialization --- .../spark/graphx/impl/EdgePartitionSuite.scala | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala index d2e0c01bc35e..28fd112f2b12 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala @@ -22,6 +22,9 @@ import scala.util.Random import org.scalatest.FunSuite +import org.apache.spark.SparkConf +import org.apache.spark.serializer.KryoSerializer + import org.apache.spark.graphx._ class EdgePartitionSuite extends FunSuite { @@ -120,4 +123,19 @@ class EdgePartitionSuite extends FunSuite { assert(!ep.isActive(-1)) assert(ep.numActives == Some(2)) } + + test("Kryo serialization") { + val aList = List((0, 1, 0), (1, 0, 0), (1, 2, 0), (5, 4, 0), (5, 5, 0)) + val a: EdgePartition[Int, Int] = makeEdgePartition(aList) + val conf = new SparkConf() + .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + .set("spark.kryo.registrator", "org.apache.spark.graphx.GraphKryoRegistrator") + val s = new KryoSerializer(conf).newInstance() + val aSer: EdgePartition[Int, Int] = s.deserialize(s.serialize(a)) + assert(aSer.srcIds.toList === a.srcIds.toList) + assert(aSer.dstIds.toList === a.dstIds.toList) + assert(aSer.data.toList === a.data.toList) + assert(aSer.index != null) + assert(aSer.vertices.iterator.toSet === a.vertices.iterator.toSet) + } } From 5d27824b7a32f3fb84c3db8c6359752ec458e9cd Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 9 May 2014 20:12:38 -0700 Subject: [PATCH 16/18] Disable reference tracking to fix serialization test --- .../scala/org/apache/spark/graphx/GraphKryoRegistrator.scala | 2 -- 1 file changed, 2 deletions(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala index e80217b196d3..f97f329c0e83 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala @@ -49,7 +49,5 @@ class GraphKryoRegistrator extends KryoRegistrator { kryo.register(classOf[GraphXPrimitiveKeyOpenHashMap[VertexId, Int]]) kryo.register(classOf[OpenHashSet[Int]]) kryo.register(classOf[OpenHashSet[Long]]) - // This avoids a large number of hash table lookups. - kryo.setReferences(false) } } From cb2ed3aae3b9f9e7d7a03c64c49c467d4f0331be Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Mon, 12 May 2014 11:20:49 -0700 Subject: [PATCH 17/18] addressing missing exclusion in MimaBuild.scala --- project/MimaBuild.scala | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/project/MimaBuild.scala b/project/MimaBuild.scala index fafc9b36a77d..17ef3d134e66 100644 --- a/project/MimaBuild.scala +++ b/project/MimaBuild.scala @@ -30,7 +30,7 @@ object MimaBuild { // Read package-private excludes from file val excludeFilePath = (base.getAbsolutePath + "/.mima-excludes") - val excludeFile = file(excludeFilePath) + val excludeFile = file(excludeFilePath) val packagePrivateList: Seq[String] = if (!excludeFile.exists()) { Seq() @@ -41,10 +41,10 @@ object MimaBuild { // Exclude a single class and its corresponding object def excludeClass(className: String) = { Seq( - excludePackage(className), + excludePackage(className), ProblemFilters.exclude[MissingClassProblem](className), ProblemFilters.exclude[MissingTypesProblem](className), - excludePackage(className + "$"), + excludePackage(className + "$"), ProblemFilters.exclude[MissingClassProblem](className + "$"), ProblemFilters.exclude[MissingTypesProblem](className + "$") ) @@ -75,6 +75,8 @@ object MimaBuild { excludeSparkClass("rdd.ClassTags") ++ excludeSparkClass("util.XORShiftRandom") ++ excludeSparkClass("graphx.EdgeRDD") ++ + excludeSparkClass("graphx.util.collection.PrimitiveKeyOpenHashMap") + excludeSparkClass("graphx.util.collection.GraphXPrimitiveKeyOpenHashMap") excludeSparkClass("graphx.VertexRDD") ++ excludeSparkClass("graphx.impl.GraphImpl") ++ excludeSparkClass("graphx.impl.RoutingTable") ++ From 8ba6e0d3f224be58d9da661f802efff9448ca31e Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Mon, 12 May 2014 12:11:38 -0700 Subject: [PATCH 18/18] Add concatenation operators to MimaBuild.scala --- project/MimaBuild.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/project/MimaBuild.scala b/project/MimaBuild.scala index 17ef3d134e66..e147be7ddaa6 100644 --- a/project/MimaBuild.scala +++ b/project/MimaBuild.scala @@ -75,11 +75,11 @@ object MimaBuild { excludeSparkClass("rdd.ClassTags") ++ excludeSparkClass("util.XORShiftRandom") ++ excludeSparkClass("graphx.EdgeRDD") ++ - excludeSparkClass("graphx.util.collection.PrimitiveKeyOpenHashMap") - excludeSparkClass("graphx.util.collection.GraphXPrimitiveKeyOpenHashMap") excludeSparkClass("graphx.VertexRDD") ++ excludeSparkClass("graphx.impl.GraphImpl") ++ excludeSparkClass("graphx.impl.RoutingTable") ++ + excludeSparkClass("graphx.util.collection.PrimitiveKeyOpenHashMap") ++ + excludeSparkClass("graphx.util.collection.GraphXPrimitiveKeyOpenHashMap") ++ excludeSparkClass("mllib.recommendation.MFDataGenerator") ++ excludeSparkClass("mllib.optimization.SquaredGradient") ++ excludeSparkClass("mllib.regression.RidgeRegressionWithSGD") ++