From 731f56f309914e3fc7c22c8ef1c8cb9dd40d42c1 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Thu, 9 Jan 2014 14:31:33 -0800 Subject: graph -> graphx --- graphx/pom.xml | 129 +++++ .../scala/org/apache/spark/graphx/Analytics.scala | 593 +++++++++++++++++++++ .../main/scala/org/apache/spark/graphx/Edge.scala | 50 ++ .../org/apache/spark/graphx/EdgeDirection.scala | 36 ++ .../scala/org/apache/spark/graphx/EdgeRDD.scala | 73 +++ .../org/apache/spark/graphx/EdgeTriplet.scala | 63 +++ .../main/scala/org/apache/spark/graphx/Graph.scala | 437 +++++++++++++++ .../apache/spark/graphx/GraphKryoRegistrator.scala | 28 + .../scala/org/apache/spark/graphx/GraphLab.scala | 134 +++++ .../org/apache/spark/graphx/GraphLoader.scala | 113 ++++ .../scala/org/apache/spark/graphx/GraphOps.scala | 277 ++++++++++ .../apache/spark/graphx/PartitionStrategy.scala | 94 ++++ .../scala/org/apache/spark/graphx/Pregel.scala | 122 +++++ .../scala/org/apache/spark/graphx/VertexRDD.scala | 361 +++++++++++++ .../graphx/algorithms/ConnectedComponents.scala | 37 ++ .../apache/spark/graphx/algorithms/PageRank.scala | 205 +++++++ .../spark/graphx/algorithms/SVDPlusPlus.scala | 103 ++++ .../algorithms/StronglyConnectedComponents.scala | 87 +++ .../spark/graphx/algorithms/TriangleCount.scala | 78 +++ .../apache/spark/graphx/impl/EdgePartition.scala | 220 ++++++++ .../spark/graphx/impl/EdgePartitionBuilder.scala | 46 ++ .../spark/graphx/impl/EdgeTripletIterator.scala | 43 ++ .../org/apache/spark/graphx/impl/GraphImpl.scala | 422 +++++++++++++++ .../spark/graphx/impl/MessageToPartition.scala | 93 ++++ .../spark/graphx/impl/ReplicatedVertexView.scala | 182 +++++++ .../apache/spark/graphx/impl/RoutingTable.scala | 64 +++ .../org/apache/spark/graphx/impl/Serializers.scala | 386 ++++++++++++++ .../apache/spark/graphx/impl/VertexPartition.scala | 262 +++++++++ .../scala/org/apache/spark/graphx/package.scala | 22 + .../org/apache/spark/graphx/perf/BagelTest.scala | 76 +++ .../org/apache/spark/graphx/perf/SparkTest.scala | 75 +++ .../apache/spark/graphx/util/BytecodeUtils.scala | 114 ++++ .../apache/spark/graphx/util/GraphGenerators.scala | 282 ++++++++++ .../org/apache/spark/graphx/util/HashUtils.scala | 21 + graphx/src/test/resources/log4j.properties | 28 + .../org/apache/spark/graphx/GraphOpsSuite.scala | 92 ++++ .../scala/org/apache/spark/graphx/GraphSuite.scala | 272 ++++++++++ .../apache/spark/graphx/LocalSparkContext.scala | 28 + .../org/apache/spark/graphx/PregelSuite.scala | 41 ++ .../org/apache/spark/graphx/SerializerSuite.scala | 183 +++++++ .../org/apache/spark/graphx/VertexRDDSuite.scala | 85 +++ .../algorithms/ConnectedComponentsSuite.scala | 83 +++ .../spark/graphx/algorithms/PageRankSuite.scala | 126 +++++ .../spark/graphx/algorithms/SVDPlusPlusSuite.scala | 30 ++ .../StronglyConnectedComponentsSuite.scala | 57 ++ .../graphx/algorithms/TriangleCountSuite.scala | 73 +++ .../spark/graphx/impl/EdgePartitionSuite.scala | 76 +++ .../spark/graphx/impl/VertexPartitionSuite.scala | 113 ++++ .../spark/graphx/util/BytecodeUtilsSuite.scala | 93 ++++ 49 files changed, 6708 insertions(+) create mode 100644 graphx/pom.xml create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/Analytics.scala create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/Edge.scala create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/EdgeDirection.scala create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/EdgeTriplet.scala create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/Graph.scala create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/GraphLab.scala create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/algorithms/ConnectedComponents.scala create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/algorithms/PageRank.scala create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/algorithms/SVDPlusPlus.scala create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/algorithms/StronglyConnectedComponents.scala create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/algorithms/TriangleCount.scala create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala create 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/Serializers.scala create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/package.scala create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/perf/BagelTest.scala create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/perf/SparkTest.scala create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/util/HashUtils.scala create mode 100644 graphx/src/test/resources/log4j.properties create mode 100644 graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala create mode 100644 graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala create mode 100644 graphx/src/test/scala/org/apache/spark/graphx/LocalSparkContext.scala create mode 100644 graphx/src/test/scala/org/apache/spark/graphx/PregelSuite.scala create mode 100644 graphx/src/test/scala/org/apache/spark/graphx/SerializerSuite.scala create mode 100644 graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala create mode 100644 graphx/src/test/scala/org/apache/spark/graphx/algorithms/ConnectedComponentsSuite.scala create mode 100644 graphx/src/test/scala/org/apache/spark/graphx/algorithms/PageRankSuite.scala create mode 100644 graphx/src/test/scala/org/apache/spark/graphx/algorithms/SVDPlusPlusSuite.scala create mode 100644 graphx/src/test/scala/org/apache/spark/graphx/algorithms/StronglyConnectedComponentsSuite.scala create mode 100644 graphx/src/test/scala/org/apache/spark/graphx/algorithms/TriangleCountSuite.scala create mode 100644 graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala create mode 100644 graphx/src/test/scala/org/apache/spark/graphx/impl/VertexPartitionSuite.scala create mode 100644 graphx/src/test/scala/org/apache/spark/graphx/util/BytecodeUtilsSuite.scala (limited to 'graphx') diff --git a/graphx/pom.xml b/graphx/pom.xml new file mode 100644 index 0000000000..fd3dcaad7c --- /dev/null +++ b/graphx/pom.xml @@ -0,0 +1,129 @@ + + + + + 4.0.0 + + org.apache.spark + spark-parent + 0.9.0-incubating-SNAPSHOT + ../pom.xml + + + org.apache.spark + spark-graph_2.9.3 + jar + Spark Graph + http://spark-project.org/ + + + + org.apache.spark + spark-core_2.9.3 + ${project.version} + provided + + + org.eclipse.jetty + jetty-server + + + + org.scalatest + scalatest_${scala.version} + test + + + org.scalacheck + scalacheck_${scala.version} + test + + + + target/scala-${scala.version}/classes + target/scala-${scala.version}/test-classes + + + org.scalatest + scalatest-maven-plugin + + + + + + + hadoop1 + + + org.spark-project + spark-core + ${project.version} + hadoop1 + + + org.apache.hadoop + hadoop-core + provided + + + + + + org.apache.maven.plugins + maven-jar-plugin + + hadoop1 + + + + + + + hadoop2 + + + org.spark-project + spark-core + ${project.version} + hadoop2 + + + org.apache.hadoop + hadoop-core + provided + + + org.apache.hadoop + hadoop-client + provided + + + + + + org.apache.maven.plugins + maven-jar-plugin + + hadoop2 + + + + + + + diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Analytics.scala b/graphx/src/main/scala/org/apache/spark/graphx/Analytics.scala new file mode 100644 index 0000000000..0cafc3fdf9 --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/Analytics.scala @@ -0,0 +1,593 @@ +package org.apache.spark.graphx + +import org.apache.spark._ +import org.apache.spark.graphx.algorithms._ + + +/** + * The Analytics object contains a collection of basic graph analytics + * algorithms that operate largely on the graph structure. + * + * In addition the Analytics object contains a driver `main` which can + * be used to apply the various functions to graphs in standard + * formats. + */ +object Analytics extends Logging { + + def main(args: Array[String]) = { + val host = args(0) + val taskType = args(1) + val fname = args(2) + val options = args.drop(3).map { arg => + arg.dropWhile(_ == '-').split('=') match { + case Array(opt, v) => (opt -> v) + case _ => throw new IllegalArgumentException("Invalid argument: " + arg) + } + } + + def setLogLevels(level: org.apache.log4j.Level, loggers: TraversableOnce[String]) = { + loggers.map{ + loggerName => + val logger = org.apache.log4j.Logger.getLogger(loggerName) + val prevLevel = logger.getLevel() + logger.setLevel(level) + loggerName -> prevLevel + }.toMap + } + + def pickPartitioner(v: String): PartitionStrategy = { + v match { + case "RandomVertexCut" => RandomVertexCut + case "EdgePartition1D" => EdgePartition1D + case "EdgePartition2D" => EdgePartition2D + case "CanonicalRandomVertexCut" => CanonicalRandomVertexCut + case _ => throw new IllegalArgumentException("Invalid Partition Strategy: " + v) + } + } +// setLogLevels(org.apache.log4j.Level.DEBUG, Seq("org.apache.spark")) + + val serializer = "org.apache.spark.serializer.KryoSerializer" + System.setProperty("spark.serializer", serializer) + //System.setProperty("spark.shuffle.compress", "false") + System.setProperty("spark.kryo.registrator", "org.apache.spark.graphx.GraphKryoRegistrator") + + taskType match { + case "pagerank" => { + + var tol:Float = 0.001F + var outFname = "" + var numVPart = 4 + var numEPart = 4 + var partitionStrategy: Option[PartitionStrategy] = None + + options.foreach{ + case ("tol", v) => tol = v.toFloat + case ("output", v) => outFname = v + case ("numVPart", v) => numVPart = v.toInt + case ("numEPart", v) => numEPart = v.toInt + case ("partStrategy", v) => partitionStrategy = Some(pickPartitioner(v)) + case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) + } + + println("======================================") + println("| PageRank |") + println("======================================") + + val sc = new SparkContext(host, "PageRank(" + fname + ")") + + val unpartitionedGraph = GraphLoader.edgeListFile(sc, fname, + minEdgePartitions = numEPart).cache() + val graph = partitionStrategy.foldLeft(unpartitionedGraph)(_.partitionBy(_)) + + println("GRAPHX: Number of vertices " + graph.vertices.count) + println("GRAPHX: Number of edges " + graph.edges.count) + + //val pr = Analytics.pagerank(graph, numIter) + val pr = PageRank.runStandalone(graph, tol) + + println("GRAPHX: Total rank: " + pr.map(_._2).reduce(_+_)) + + if (!outFname.isEmpty) { + logWarning("Saving pageranks of pages to " + outFname) + pr.map{case (id, r) => id + "\t" + r}.saveAsTextFile(outFname) + } + + sc.stop() + } + + case "cc" => { + + var numIter = Int.MaxValue + var numVPart = 4 + var numEPart = 4 + var isDynamic = false + var partitionStrategy: Option[PartitionStrategy] = None + + options.foreach{ + case ("numIter", v) => numIter = v.toInt + case ("dynamic", v) => isDynamic = v.toBoolean + case ("numEPart", v) => numEPart = v.toInt + case ("numVPart", v) => numVPart = v.toInt + case ("partStrategy", v) => partitionStrategy = Some(pickPartitioner(v)) + case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) + } + + if(!isDynamic && numIter == Int.MaxValue) { + println("Set number of iterations!") + sys.exit(1) + } + println("======================================") + println("| Connected Components |") + println("--------------------------------------") + println(" Using parameters:") + println(" \tDynamic: " + isDynamic) + println(" \tNumIter: " + numIter) + println("======================================") + + val sc = new SparkContext(host, "ConnectedComponents(" + fname + ")") + val unpartitionedGraph = GraphLoader.edgeListFile(sc, fname, + minEdgePartitions = numEPart).cache() + val graph = partitionStrategy.foldLeft(unpartitionedGraph)(_.partitionBy(_)) + + val cc = ConnectedComponents.run(graph) + println("Components: " + cc.vertices.map{ case (vid,data) => data}.distinct()) + sc.stop() + } + + case "triangles" => { + var numVPart = 4 + var numEPart = 4 + // TriangleCount requires the graph to be partitioned + var partitionStrategy: PartitionStrategy = RandomVertexCut + + options.foreach{ + case ("numEPart", v) => numEPart = v.toInt + case ("numVPart", v) => numVPart = v.toInt + case ("partStrategy", v) => partitionStrategy = pickPartitioner(v) + case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) + } + println("======================================") + println("| Triangle Count |") + println("--------------------------------------") + val sc = new SparkContext(host, "TriangleCount(" + fname + ")") + val graph = GraphLoader.edgeListFile(sc, fname, canonicalOrientation = true, + minEdgePartitions = numEPart).partitionBy(partitionStrategy).cache() + val triangles = TriangleCount.run(graph) + println("Triangles: " + triangles.vertices.map { + case (vid,data) => data.toLong + }.reduce(_+_) / 3) + sc.stop() + } + +// +// case "shortestpath" => { +// +// var numIter = Int.MaxValue +// var isDynamic = true +// var sources: List[Int] = List.empty +// +// options.foreach{ +// case ("numIter", v) => numIter = v.toInt +// case ("dynamic", v) => isDynamic = v.toBoolean +// case ("source", v) => sources ++= List(v.toInt) +// case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) +// } +// +// +// if(!isDynamic && numIter == Int.MaxValue) { +// println("Set number of iterations!") +// sys.exit(1) +// } +// +// if(sources.isEmpty) { +// println("No sources provided!") +// sys.exit(1) +// } +// +// println("======================================") +// println("| Shortest Path |") +// println("--------------------------------------") +// println(" Using parameters:") +// println(" \tDynamic: " + isDynamic) +// println(" \tNumIter: " + numIter) +// println(" \tSources: [" + sources.mkString(", ") + "]") +// println("======================================") +// +// val sc = new SparkContext(host, "ShortestPath(" + fname + ")") +// val graph = GraphLoader.textFile(sc, fname, a => (if(a.isEmpty) 1.0F else a(0).toFloat ) ) +// //val sp = Analytics.shortestPath(graph, sources, numIter) +// // val cc = if(isDynamic) Analytics.dynamicShortestPath(graph, sources, numIter) +// // else Analytics.shortestPath(graph, sources, numIter) +// println("Longest Path: " + sp.vertices.map(_.data).reduce(math.max(_,_))) +// +// sc.stop() +// } + + + // case "als" => { + + // var numIter = 5 + // var lambda = 0.01 + // var latentK = 10 + // var usersFname = "usersFactors.tsv" + // var moviesFname = "moviesFname.tsv" + // var numVPart = 4 + // var numEPart = 4 + + // options.foreach{ + // case ("numIter", v) => numIter = v.toInt + // case ("lambda", v) => lambda = v.toDouble + // case ("latentK", v) => latentK = v.toInt + // case ("usersFname", v) => usersFname = v + // case ("moviesFname", v) => moviesFname = v + // case ("numVPart", v) => numVPart = v.toInt + // case ("numEPart", v) => numEPart = v.toInt + // case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) + // } + + // println("======================================") + // println("| Alternating Least Squares |") + // println("--------------------------------------") + // println(" Using parameters:") + // println(" \tNumIter: " + numIter) + // println(" \tLambda: " + lambda) + // println(" \tLatentK: " + latentK) + // println(" \tusersFname: " + usersFname) + // println(" \tmoviesFname: " + moviesFname) + // println("======================================") + + // val sc = new SparkContext(host, "ALS(" + fname + ")") + // val graph = GraphLoader.textFile(sc, fname, a => a(0).toDouble ) + // graph.numVPart = numVPart + // graph.numEPart = numEPart + + // val maxUser = graph.edges.map(_._1).reduce(math.max(_,_)) + // val minMovie = graph.edges.map(_._2).reduce(math.min(_,_)) + // assert(maxUser < minMovie) + + // val factors = Analytics.alternatingLeastSquares(graph, latentK, lambda, numIter).cache + // factors.filter(_._1 <= maxUser).map(r => r._1 + "\t" + r._2.mkString("\t")) + // .saveAsTextFile(usersFname) + // factors.filter(_._1 >= minMovie).map(r => r._1 + "\t" + r._2.mkString("\t")) + // .saveAsTextFile(moviesFname) + + // sc.stop() + // } + + + case _ => { + println("Invalid task type.") + } + } + } + + // /** + // * Compute the shortest path to a set of markers + // */ + // def shortestPath[VD: Manifest](graph: Graph[VD, Double], sources: List[Int], numIter: Int) = { + // val sourceSet = sources.toSet + // val spGraph = graph.mapVertices { + // case Vertex(vid, _) => Vertex(vid, (if(sourceSet.contains(vid)) 0.0 else Double.MaxValue)) + // } + // GraphLab.iterateGA[Double, Double, Double](spGraph)( + // (me_id, edge) => edge.otherVertex(me_id).data + edge.data, // gather + // (a: Double, b: Double) => math.min(a, b), // merge + // (v, a: Option[Double]) => math.min(v.data, a.getOrElse(Double.MaxValue)), // apply + // numIter, + // gatherDirection = EdgeDirection.In) + // } + + // /** + // * Compute the connected component membership of each vertex + // * and return an RDD with the vertex value containing the + // * lowest vertex id in the connected component containing + // * that vertex. + // */ + // def dynamicConnectedComponents[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], + // numIter: Int = Int.MaxValue) = { + + // val vertices = graph.vertices.mapPartitions(iter => iter.map { case (vid, _) => (vid, vid) }) + // val edges = graph.edges // .mapValues(v => None) + // val ccGraph = new Graph(vertices, edges) + + // ccGraph.iterateDynamic( + // (me_id, edge) => edge.otherVertex(me_id).data, // gather + // (a: Int, b: Int) => math.min(a, b), // merge + // Integer.MAX_VALUE, + // (v, a: Int) => math.min(v.data, a), // apply + // (me_id, edge) => edge.otherVertex(me_id).data > edge.vertex(me_id).data, // scatter + // numIter, + // gatherEdges = EdgeDirection.Both, + // scatterEdges = EdgeDirection.Both).vertices + // // + // // graph_ret.vertices.collect.foreach(println) + // // graph_ret.edges.take(10).foreach(println) + // } + + + // /** + // * Compute the shortest path to a set of markers + // */ + // def dynamicShortestPath[VD: Manifest, ED: Manifest](graph: Graph[VD, Double], + // sources: List[Int], numIter: Int) = { + // val sourceSet = sources.toSet + // val vertices = graph.vertices.mapPartitions( + // iter => iter.map { + // case (vid, _) => (vid, (if(sourceSet.contains(vid)) 0.0F else Double.MaxValue) ) + // }); + + // val edges = graph.edges // .mapValues(v => None) + // val spGraph = new Graph(vertices, edges) + + // val niterations = Int.MaxValue + // spGraph.iterateDynamic( + // (me_id, edge) => edge.otherVertex(me_id).data + edge.data, // gather + // (a: Double, b: Double) => math.min(a, b), // merge + // Double.MaxValue, + // (v, a: Double) => math.min(v.data, a), // apply + // (me_id, edge) => edge.vertex(me_id).data + edge.data < edge.otherVertex(me_id).data, // scatter + // numIter, + // gatherEdges = EdgeDirection.In, + // scatterEdges = EdgeDirection.Out).vertices + // } + + + // /** + // * + // */ + // def alternatingLeastSquares[VD: ClassTag, ED: ClassTag](graph: Graph[VD, Double], + // latentK: Int, lambda: Double, numIter: Int) = { + // val vertices = graph.vertices.mapPartitions( _.map { + // case (vid, _) => (vid, Array.fill(latentK){ scala.util.Random.nextDouble() } ) + // }).cache + // val maxUser = graph.edges.map(_._1).reduce(math.max(_,_)) + // val edges = graph.edges // .mapValues(v => None) + // val alsGraph = new Graph(vertices, edges) + // alsGraph.numVPart = graph.numVPart + // alsGraph.numEPart = graph.numEPart + + // val niterations = Int.MaxValue + // alsGraph.iterateDynamic[(Array[Double], Array[Double])]( + // (me_id, edge) => { // gather + // val X = edge.otherVertex(me_id).data + // val y = edge.data + // val Xy = X.map(_ * y) + // val XtX = (for(i <- 0 until latentK; j <- i until latentK) yield(X(i) * X(j))).toArray + // (Xy, XtX) + // }, + // (a, b) => { + // // The difference between the while loop and the zip is a FACTOR OF TWO in overall + // // runtime + // var i = 0 + // while(i < a._1.length) { a._1(i) += b._1(i); i += 1 } + // i = 0 + // while(i < a._2.length) { a._2(i) += b._2(i); i += 1 } + // a + // // (a._1.zip(b._1).map{ case (q,r) => q+r }, a._2.zip(b._2).map{ case (q,r) => q+r }) + // }, + // (Array.empty[Double], Array.empty[Double]), // default value is empty + // (vertex, accum) => { // apply + // val XyArray = accum._1 + // val XtXArray = accum._2 + // if(XyArray.isEmpty) vertex.data // no neighbors + // else { + // val XtX = DenseMatrix.tabulate(latentK,latentK){ (i,j) => + // (if(i < j) XtXArray(i + (j+1)*j/2) else XtXArray(i + (j+1)*j/2)) + + // (if(i == j) lambda else 1.0F) //regularization + // } + // val Xy = DenseMatrix.create(latentK,1,XyArray) + // val w = XtX \ Xy + // w.data + // } + // }, + // (me_id, edge) => true, + // numIter, + // gatherEdges = EdgeDirection.Both, + // scatterEdges = EdgeDirection.Both, + // vertex => vertex.id < maxUser).vertices + // } + + // def main(args: Array[String]) = { + // val host = args(0) + // val taskType = args(1) + // val fname = args(2) + // val options = args.drop(3).map { arg => + // arg.dropWhile(_ == '-').split('=') match { + // case Array(opt, v) => (opt -> v) + // case _ => throw new IllegalArgumentException("Invalid argument: " + arg) + // } + // } + + // System.setProperty("spark.serializer", "spark.KryoSerializer") + // //System.setProperty("spark.shuffle.compress", "false") + // System.setProperty("spark.kryo.registrator", "spark.graphx.GraphKryoRegistrator") + + // taskType match { + // case "pagerank" => { + + // var numIter = Int.MaxValue + // var isDynamic = false + // var tol:Double = 0.001 + // var outFname = "" + // var numVPart = 4 + // var numEPart = 4 + + // options.foreach{ + // case ("numIter", v) => numIter = v.toInt + // case ("dynamic", v) => isDynamic = v.toBoolean + // case ("tol", v) => tol = v.toDouble + // case ("output", v) => outFname = v + // case ("numVPart", v) => numVPart = v.toInt + // case ("numEPart", v) => numEPart = v.toInt + // case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) + // } + + // if(!isDynamic && numIter == Int.MaxValue) { + // println("Set number of iterations!") + // sys.exit(1) + // } + // println("======================================") + // println("| PageRank |") + // println("--------------------------------------") + // println(" Using parameters:") + // println(" \tDynamic: " + isDynamic) + // if(isDynamic) println(" \t |-> Tolerance: " + tol) + // println(" \tNumIter: " + numIter) + // println("======================================") + + // val sc = new SparkContext(host, "PageRank(" + fname + ")") + + // val graph = GraphLoader.textFile(sc, fname, a => 1.0).withPartitioner(numVPart, numEPart).cache() + + // val startTime = System.currentTimeMillis + // logInfo("GRAPHX: starting tasks") + // logInfo("GRAPHX: Number of vertices " + graph.vertices.count) + // logInfo("GRAPHX: Number of edges " + graph.edges.count) + + // val pr = Analytics.pagerank(graph, numIter) + // // val pr = if(isDynamic) Analytics.dynamicPagerank(graph, tol, numIter) + // // else Analytics.pagerank(graph, numIter) + // logInfo("GRAPHX: Total rank: " + pr.vertices.map{ case Vertex(id,r) => r }.reduce(_+_) ) + // if (!outFname.isEmpty) { + // println("Saving pageranks of pages to " + outFname) + // pr.vertices.map{case Vertex(id, r) => id + "\t" + r}.saveAsTextFile(outFname) + // } + // logInfo("GRAPHX: Runtime: " + ((System.currentTimeMillis - startTime)/1000.0) + " seconds") + // sc.stop() + // } + + // case "cc" => { + + // var numIter = Int.MaxValue + // var isDynamic = false + + // options.foreach{ + // case ("numIter", v) => numIter = v.toInt + // case ("dynamic", v) => isDynamic = v.toBoolean + // case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) + // } + + // if(!isDynamic && numIter == Int.MaxValue) { + // println("Set number of iterations!") + // sys.exit(1) + // } + // println("======================================") + // println("| Connected Components |") + // println("--------------------------------------") + // println(" Using parameters:") + // println(" \tDynamic: " + isDynamic) + // println(" \tNumIter: " + numIter) + // println("======================================") + + // val sc = new SparkContext(host, "ConnectedComponents(" + fname + ")") + // val graph = GraphLoader.textFile(sc, fname, a => 1.0) + // val cc = Analytics.connectedComponents(graph, numIter) + // // val cc = if(isDynamic) Analytics.dynamicConnectedComponents(graph, numIter) + // // else Analytics.connectedComponents(graph, numIter) + // println("Components: " + cc.vertices.map(_.data).distinct()) + + // sc.stop() + // } + + // case "shortestpath" => { + + // var numIter = Int.MaxValue + // var isDynamic = true + // var sources: List[Int] = List.empty + + // options.foreach{ + // case ("numIter", v) => numIter = v.toInt + // case ("dynamic", v) => isDynamic = v.toBoolean + // case ("source", v) => sources ++= List(v.toInt) + // case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) + // } + + + // if(!isDynamic && numIter == Int.MaxValue) { + // println("Set number of iterations!") + // sys.exit(1) + // } + + // if(sources.isEmpty) { + // println("No sources provided!") + // sys.exit(1) + // } + + // println("======================================") + // println("| Shortest Path |") + // println("--------------------------------------") + // println(" Using parameters:") + // println(" \tDynamic: " + isDynamic) + // println(" \tNumIter: " + numIter) + // println(" \tSources: [" + sources.mkString(", ") + "]") + // println("======================================") + + // val sc = new SparkContext(host, "ShortestPath(" + fname + ")") + // val graph = GraphLoader.textFile(sc, fname, a => (if(a.isEmpty) 1.0 else a(0).toDouble ) ) + // val sp = Analytics.shortestPath(graph, sources, numIter) + // // val cc = if(isDynamic) Analytics.dynamicShortestPath(graph, sources, numIter) + // // else Analytics.shortestPath(graph, sources, numIter) + // println("Longest Path: " + sp.vertices.map(_.data).reduce(math.max(_,_))) + + // sc.stop() + // } + + + // case "als" => { + + // var numIter = 5 + // var lambda = 0.01 + // var latentK = 10 + // var usersFname = "usersFactors.tsv" + // var moviesFname = "moviesFname.tsv" + // var numVPart = 4 + // var numEPart = 4 + + // options.foreach{ + // case ("numIter", v) => numIter = v.toInt + // case ("lambda", v) => lambda = v.toDouble + // case ("latentK", v) => latentK = v.toInt + // case ("usersFname", v) => usersFname = v + // case ("moviesFname", v) => moviesFname = v + // case ("numVPart", v) => numVPart = v.toInt + // case ("numEPart", v) => numEPart = v.toInt + // case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) + // } + + // println("======================================") + // println("| Alternating Least Squares |") + // println("--------------------------------------") + // println(" Using parameters:") + // println(" \tNumIter: " + numIter) + // println(" \tLambda: " + lambda) + // println(" \tLatentK: " + latentK) + // println(" \tusersFname: " + usersFname) + // println(" \tmoviesFname: " + moviesFname) + // println("======================================") + + // val sc = new SparkContext(host, "ALS(" + fname + ")") + // val graph = GraphLoader.textFile(sc, fname, a => a(0).toDouble ) + // graph.numVPart = numVPart + // graph.numEPart = numEPart + + // val maxUser = graph.edges.map(_._1).reduce(math.max(_,_)) + // val minMovie = graph.edges.map(_._2).reduce(math.min(_,_)) + // assert(maxUser < minMovie) + + // val factors = Analytics.alternatingLeastSquares(graph, latentK, lambda, numIter).cache + // factors.filter(_._1 <= maxUser).map(r => r._1 + "\t" + r._2.mkString("\t")) + // .saveAsTextFile(usersFname) + // factors.filter(_._1 >= minMovie).map(r => r._1 + "\t" + r._2.mkString("\t")) + // .saveAsTextFile(moviesFname) + + // sc.stop() + // } + + + // case _ => { + // println("Invalid task type.") + // } + // } + // } + +} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala b/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala new file mode 100644 index 0000000000..29b46674f1 --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala @@ -0,0 +1,50 @@ +package org.apache.spark.graphx + + +/** + * A single directed edge consisting of a source id, target id, + * and the data associated with the Edgee. + * + * @tparam ED type of the edge attribute + */ +case class Edge[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED] ( + /** + * The vertex id of the source vertex + */ + var srcId: VertexID = 0, + /** + * The vertex id of the target vertex. + */ + var dstId: VertexID = 0, + /** + * The attribute associated with the edge. + */ + var attr: ED = nullValue[ED]) extends Serializable { + + /** + * Given one vertex in the edge return the other vertex. + * + * @param vid the id one of the two vertices on the edge. + * @return the id of the other vertex on the edge. + */ + def otherVertexId(vid: VertexID): VertexID = + if (srcId == vid) dstId else { assert(dstId == vid); srcId } + + /** + * Return the relative direction of the edge to the corresponding + * vertex. + * + * @param vid the id of one of the two vertices in the edge. + * @return the relative direction of the edge to the corresponding + * vertex. + */ + def relativeDirection(vid: VertexID): EdgeDirection = + if (vid == srcId) EdgeDirection.Out else { assert(vid == dstId); EdgeDirection.In } +} + +object Edge { + def lexicographicOrdering[ED] = new Ordering[Edge[ED]] { + override def compare(a: Edge[ED], b: Edge[ED]): Int = + Ordering[(VertexID, VertexID)].compare((a.srcId, a.dstId), (b.srcId, b.dstId)) + } +} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/EdgeDirection.scala b/graphx/src/main/scala/org/apache/spark/graphx/EdgeDirection.scala new file mode 100644 index 0000000000..785f941650 --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeDirection.scala @@ -0,0 +1,36 @@ +package org.apache.spark.graphx + + +/** + * The direction of directed edge relative to a vertex used to select + * the set of adjacent neighbors when running a neighborhood query. + */ +sealed abstract class EdgeDirection { + /** + * Reverse the direction of an edge. An in becomes out, + * out becomes in and both remains both. + */ + def reverse: EdgeDirection = this match { + case EdgeDirection.In => EdgeDirection.Out + case EdgeDirection.Out => EdgeDirection.In + case EdgeDirection.Both => EdgeDirection.Both + } +} + + +object EdgeDirection { + /** + * Edges arriving at a vertex. + */ + case object In extends EdgeDirection + + /** + * Edges originating from a vertex + */ + case object Out extends EdgeDirection + + /** + * All edges adjacent to a vertex + */ + case object Both extends EdgeDirection +} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala b/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala new file mode 100644 index 0000000000..e4ef460e6f --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala @@ -0,0 +1,73 @@ +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 + + +class EdgeRDD[@specialized ED: ClassTag]( + val partitionsRDD: RDD[(PartitionID, EdgePartition[ED])]) + extends RDD[Edge[ED]](partitionsRDD.context, List(new OneToOneDependency(partitionsRDD))) { + + partitionsRDD.setName("EdgeRDD") + + override protected def getPartitions: Array[Partition] = partitionsRDD.partitions + + /** + * If partitionsRDD already has a partitioner, use it. Otherwise assume that the PartitionIDs in + * partitionsRDD correspond to the actual partitions and create a new partitioner that allows + * co-partitioning with partitionsRDD. + */ + override val partitioner = + partitionsRDD.partitioner.orElse(Some(Partitioner.defaultPartitioner(partitionsRDD))) + + override def compute(part: Partition, context: TaskContext): Iterator[Edge[ED]] = { + firstParent[(PartitionID, EdgePartition[ED])].iterator(part, context).next._2.iterator + } + + override def collect(): Array[Edge[ED]] = this.map(_.copy()).collect() + + /** + * Caching a VertexRDD causes the index and values to be cached separately. + */ + override def persist(newLevel: StorageLevel): EdgeRDD[ED] = { + partitionsRDD.persist(newLevel) + this + } + + /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */ + override def persist(): EdgeRDD[ED] = persist(StorageLevel.MEMORY_ONLY) + + /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */ + override def cache(): EdgeRDD[ED] = persist() + + def mapEdgePartitions[ED2: ClassTag](f: (PartitionID, EdgePartition[ED]) => EdgePartition[ED2]) + : EdgeRDD[ED2] = { +// iter => iter.map { case (pid, ep) => (pid, f(ep)) } + new EdgeRDD[ED2](partitionsRDD.mapPartitions({ iter => + val (pid, ep) = iter.next() + Iterator(Tuple2(pid, f(pid, ep))) + }, preservesPartitioning = true)) + } + + def innerJoin[ED2: ClassTag, ED3: ClassTag] + (other: EdgeRDD[ED2]) + (f: (VertexID, VertexID, ED, ED2) => ED3): EdgeRDD[ED3] = { + val ed2Tag = classTag[ED2] + val ed3Tag = classTag[ED3] + new EdgeRDD[ED3](partitionsRDD.zipPartitions(other.partitionsRDD, true) { + (thisIter, otherIter) => + val (pid, thisEPart) = thisIter.next() + val (_, otherEPart) = otherIter.next() + Iterator(Tuple2(pid, thisEPart.innerJoin(otherEPart)(f)(ed2Tag, ed3Tag))) + }) + } + + def 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 new file mode 100644 index 0000000000..b0565b7e0e --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeTriplet.scala @@ -0,0 +1,63 @@ +package org.apache.spark.graphx + +import org.apache.spark.graphx.impl.VertexPartition + +/** + * An edge triplet represents two vertices and edge along with their + * attributes. + * + * @tparam VD the type of the vertex attribute. + * @tparam ED the type of the edge attribute + * + * @todo specialize edge triplet for basic types, though when I last + * tried specializing I got a warning about inherenting from a type + * that is not a trait. + */ +class EdgeTriplet[VD, ED] extends Edge[ED] { +// class EdgeTriplet[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) VD: ClassTag, +// @specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED: ClassTag] extends Edge[ED] { + + + /** + * The source vertex attribute + */ + var srcAttr: VD = _ //nullValue[VD] + + /** + * The destination vertex attribute + */ + var dstAttr: VD = _ //nullValue[VD] + + var srcStale: Boolean = false + var dstStale: Boolean = false + + /** + * Set the edge properties of this triplet. + */ + protected[spark] def set(other: Edge[ED]): EdgeTriplet[VD,ED] = { + srcId = other.srcId + dstId = other.dstId + attr = other.attr + this + } + + /** + * Given one vertex in the edge return the other vertex. + * + * @param vid the id one of the two vertices on the edge. + * @return the attribute for the other vertex on the edge. + */ + def otherVertexAttr(vid: VertexID): VD = + if (srcId == vid) dstAttr else { assert(dstId == vid); srcAttr } + + /** + * Get the vertex object for the given vertex in the edge. + * + * @param vid the id of one of the two vertices on the edge + * @return the attr for the vertex with that id. + */ + def vertexAttr(vid: VertexID): VD = + if (srcId == vid) srcAttr else { assert(dstId == vid); dstAttr } + + override def toString() = ((srcId, srcAttr), (dstId, dstAttr), attr).toString() +} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala new file mode 100644 index 0000000000..2b7c0a2583 --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala @@ -0,0 +1,437 @@ +package org.apache.spark.graphx + +import scala.reflect.ClassTag + +import org.apache.spark.graphx.impl._ +import org.apache.spark.rdd.RDD +import org.apache.spark.storage.StorageLevel + + +/** + * The Graph abstractly represents a graph with arbitrary objects + * associated with vertices and edges. The graph provides basic + * operations to access and manipulate the data associated with + * vertices and edges as well as the underlying structure. Like Spark + * RDDs, the graph is a functional data-structure in which mutating + * operations return new graphs. + * + * @see GraphOps for additional graph member functions. + * + * @note The majority of the graph operations are implemented in + * `GraphOps`. All the convenience operations are defined in the + * `GraphOps` class which may be shared across multiple graph + * implementations. + * + * @tparam VD the vertex attribute type + * @tparam ED the edge attribute type + */ +abstract class Graph[VD: ClassTag, ED: ClassTag] { + + /** + * Get the vertices and their data. + * + * @note vertex ids are unique. + * @return An RDD containing the vertices in this graph + * + * @see Vertex for the vertex type. + * + */ + val vertices: VertexRDD[VD] + + /** + * Get the Edges and their data as an RDD. The entries in the RDD + * contain just the source id and target id along with the edge + * data. + * + * @return An RDD containing the edges in this graph + * + * @see Edge for the edge type. + * @see edgesWithVertices to get an RDD which contains all the edges + * along with their vertex data. + * + */ + val edges: EdgeRDD[ED] + + /** + * Get the edges with the vertex data associated with the adjacent + * pair of vertices. + * + * @return An RDD containing edge triplets. + * + * @example This operation might be used to evaluate a graph + * coloring where we would like to check that both vertices are a + * different color. + * {{{ + * type Color = Int + * val graph: Graph[Color, Int] = Graph.textFile("hdfs://file.tsv") + * val numInvalid = graph.edgesWithVertices() + * .map(e => if (e.src.data == e.dst.data) 1 else 0).sum + * }}} + * + * @see edges() If only the edge data and adjacent vertex ids are + * required. + * + */ + val triplets: RDD[EdgeTriplet[VD, ED]] + + /** + * Cache the vertices and edges associated with this graph. + * + * @param newLevel the level at which to cache the graph. + + * @return A reference to this graph for convenience. + * + */ + def persist(newLevel: StorageLevel = StorageLevel.MEMORY_ONLY): Graph[VD, ED] + + /** + * Return a graph that is cached when first created. This is used to + * pin a graph in memory enabling multiple queries to reuse the same + * construction process. + * + * @see RDD.cache() for a more detailed explanation of caching. + */ + def cache(): Graph[VD, ED] + + /** + * Repartition the edges in the graph according to partitionStrategy. + */ + def partitionBy(partitionStrategy: PartitionStrategy): Graph[VD, ED] + + /** + * Compute statistics describing the graph representation. + */ + def statistics: Map[String, Any] + + /** + * Construct a new graph where each vertex value has been + * transformed by the map function. + * + * @note This graph is not changed and that the new graph has the + * same structure. As a consequence the underlying index structures + * can be reused. + * + * @param map the function from a vertex object to a new vertex value. + * + * @tparam VD2 the new vertex data type + * + * @example We might use this operation to change the vertex values + * from one type to another to initialize an algorithm. + * {{{ + * val rawGraph: Graph[(), ()] = Graph.textFile("hdfs://file") + * val root = 42 + * var bfsGraph = rawGraph + * .mapVertices[Int]((vid, data) => if (vid == root) 0 else Math.MaxValue) + * }}} + * + */ + def mapVertices[VD2: ClassTag](map: (VertexID, VD) => VD2): Graph[VD2, ED] + + /** + * Construct a new graph where the value of each edge is + * transformed by the map operation. This function is not passed + * the vertex value for the vertices adjacent to the edge. If + * vertex values are desired use the mapTriplets function. + * + * @note This graph is not changed and that the new graph has the + * same structure. As a consequence the underlying index structures + * can be reused. + * + * @param map the function from an edge object to a new edge value. + * + * @tparam ED2 the new edge data type + * + * @example This function might be used to initialize edge + * attributes. + * + */ + def mapEdges[ED2: ClassTag](map: Edge[ED] => ED2): Graph[VD, ED2] = { + mapEdges((pid, iter) => iter.map(map)) + } + + /** + * Construct a new graph transforming the value of each edge using + * the user defined iterator transform. The iterator transform is + * given an iterator over edge triplets within a logical partition + * and should yield a new iterator over the new values of each edge + * in the order in which they are provided to the iterator transform + * If adjacent vertex values are not required, consider using the + * mapEdges function instead. + * + * @note This that this does not change the structure of the + * graph or modify the values of this graph. As a consequence + * the underlying index structures can be reused. + * + * @param map the function which takes a partition id and an iterator + * over all the edges in the partition and must return an iterator over + * the new values for each edge in the order of the input iterator. + * + * @tparam ED2 the new edge data type + * + */ + def mapEdges[ED2: ClassTag]( + map: (PartitionID, Iterator[Edge[ED]]) => Iterator[ED2]): Graph[VD, ED2] + + /** + * Construct a new graph where the value of each edge is + * transformed by the map operation. This function passes vertex + * values for the adjacent vertices to the map function. If + * adjacent vertex values are not required, consider using the + * mapEdges function instead. + * + * @note This that this does not change the structure of the + * graph or modify the values of this graph. As a consequence + * the underlying index structures can be reused. + * + * @param map the function from an edge object to a new edge value. + * + * @tparam ED2 the new edge data type + * + * @example This function might be used to initialize edge + * attributes based on the attributes associated with each vertex. + * {{{ + * val rawGraph: Graph[Int, Int] = someLoadFunction() + * val graph = rawGraph.mapTriplets[Int]( edge => + * edge.src.data - edge.dst.data) + * }}} + * + */ + def mapTriplets[ED2: ClassTag](map: EdgeTriplet[VD, ED] => ED2): Graph[VD, ED2] = { + mapTriplets((pid, iter) => iter.map(map)) + } + + /** + * Construct a new graph transforming the value of each edge using + * the user defined iterator transform. The iterator transform is + * given an iterator over edge triplets within a logical partition + * and should yield a new iterator over the new values of each edge + * in the order in which they are provided to the iterator transform + * If adjacent vertex values are not required, consider using the + * mapEdges function instead. + * + * @note This that this does not change the structure of the + * graph or modify the values of this graph. As a consequence + * the underlying index structures can be reused. + * + * @param map the function which takes a partition id and an iterator + * over all the edges in the partition and must return an iterator over + * the new values for each edge in the order of the input iterator. + * + * @tparam ED2 the new edge data type + * + */ + def mapTriplets[ED2: ClassTag]( + map: (PartitionID, Iterator[EdgeTriplet[VD, ED]]) => Iterator[ED2]): + Graph[VD, ED2] + + /** + * Construct a new graph with all the edges reversed. If this graph + * contains an edge from a to b then the returned graph contains an + * edge from b to a. + */ + def reverse: Graph[VD, ED] + + /** + * This function takes a vertex and edge predicate and constructs + * the subgraph that consists of vertices and edges that satisfy the + * predict. The resulting graph contains the vertices and edges + * that satisfy: + * + * {{{ + * V' = {v : for all v in V where vpred(v)} + * E' = {(u,v): for all (u,v) in E where epred((u,v)) && vpred(u) && vpred(v)} + * }}} + * + * @param epred the edge predicate which takes a triplet and + * evaluates to true if the edge is to remain in the subgraph. Note + * that only edges in which both vertices satisfy the vertex + * predicate are considered. + * + * @param vpred the vertex predicate which takes a vertex object and + * evaluates to true if the vertex is to be included in the subgraph + * + * @return the subgraph containing only the vertices and edges that + * satisfy the predicates. + */ + def subgraph(epred: EdgeTriplet[VD,ED] => Boolean = (x => true), + vpred: (VertexID, VD) => Boolean = ((v,d) => true) ): Graph[VD, ED] + + /** + * Subgraph of this graph with only vertices and edges from the other graph. + * @param other the graph to project this graph onto + * @return a graph with vertices and edges that exists in both the current graph and other, + * with vertex and edge data from the current graph. + */ + def mask[VD2: ClassTag, ED2: ClassTag](other: Graph[VD2, ED2]): Graph[VD, ED] + + /** + * This function merges multiple edges between two vertices into a single Edge. For correct + * results, the graph must have been partitioned using partitionBy. + * + * @tparam ED2 the type of the resulting edge data after grouping. + * + * @param f the user supplied commutative associative function to merge edge attributes for + * duplicate edges. + * + * @return Graph[VD,ED2] The resulting graph with a single Edge for each source, dest vertex pair. + */ + def groupEdges(merge: (ED, ED) => ED): Graph[VD,ED] + + /** + * The mapReduceTriplets function is used to compute statistics + * about the neighboring edges and vertices of each vertex. The + * user supplied `mapFunc` function is invoked on each edge of the + * graph generating 0 or more "messages" to be "sent" to either + * vertex in the edge. The `reduceFunc` is then used to combine the + * output of the map phase destined to each vertex. + * + * @tparam A the type of "message" to be sent to each vertex + * + * @param mapFunc the user defined map function which returns 0 or + * more messages to neighboring vertices. + * + * @param reduceFunc the user defined reduce function which should + * be commutative and assosciative and is used to combine the output + * of the map phase. + * + * @param activeSet optionally, a set of "active" vertices and a direction of edges to consider + * when running `mapFunc`. For example, if the direction is Out, `mapFunc` will only be run on + * edges originating from vertices in the active set. `activeSet` must have the same index as the + * graph's vertices. + * + * @example We can use this function to compute the inDegree of each + * vertex + * {{{ + * val rawGraph: Graph[(),()] = Graph.textFile("twittergraph") + * val inDeg: RDD[(VertexID, Int)] = + * mapReduceTriplets[Int](et => Array((et.dst.id, 1)), _ + _) + * }}} + * + * @note By expressing computation at the edge level we achieve + * maximum parallelism. This is one of the core functions in the + * Graph API in that enables neighborhood level computation. For + * example this function can be used to count neighbors satisfying a + * predicate or implement PageRank. + * + */ + def mapReduceTriplets[A: ClassTag]( + mapFunc: EdgeTriplet[VD, ED] => Iterator[(VertexID, A)], + reduceFunc: (A, A) => A, + activeSetOpt: Option[(VertexRDD[_], EdgeDirection)] = None) + : VertexRDD[A] + + /** + * Join the vertices with an RDD and then apply a function from the + * the vertex and RDD entry to a new vertex value and type. The + * input table should contain at most one entry for each vertex. If + * no entry is provided the map function is invoked passing none. + * + * @tparam U the type of entry in the table of updates + * @tparam VD2 the new vertex value type + * + * @param table the table to join with the vertices in the graph. + * The table should contain at most one entry for each vertex. + * + * @param mapFunc the function used to compute the new vertex + * values. The map function is invoked for all vertices, even those + * that do not have a corresponding entry in the table. + * + * @example This function is used to update the vertices with new + * values based on external data. For example we could add the out + * degree to each vertex record + * + * {{{ + * val rawGraph: Graph[(),()] = Graph.textFile("webgraph") + * val outDeg: RDD[(VertexID, Int)] = rawGraph.outDegrees() + * val graph = rawGraph.outerJoinVertices(outDeg) { + * (vid, data, optDeg) => optDeg.getOrElse(0) + * } + * }}} + * + */ + def outerJoinVertices[U: ClassTag, VD2: ClassTag](table: RDD[(VertexID, U)]) + (mapFunc: (VertexID, VD, Option[U]) => VD2) + : Graph[VD2, ED] + + // Save a copy of the GraphOps object so there is always one unique GraphOps object + // for a given Graph object, and thus the lazy vals in GraphOps would work as intended. + val ops = new GraphOps(this) +} // end of Graph + + + + +/** + * The Graph object contains a collection of routines used to construct graphs from RDDs. + */ +object Graph { + + /** + * Construct a graph from a collection of edges encoded as vertex id pairs. + * + * @param rawEdges a collection of edges in (src,dst) form. + * @param uniqueEdges if multiple identical edges are found they are combined and the edge + * attribute is set to the sum. Otherwise duplicate edges are treated as separate. To enable + * uniqueEdges, a [[PartitionStrategy]] must be provided. + * + * @return a graph with edge attributes containing either the count of duplicate edges or 1 + * (if `uniqueEdges=None`) and vertex attributes containing the total degree of each vertex. + */ + def fromEdgeTuples[VD: ClassTag]( + rawEdges: RDD[(VertexID, VertexID)], + defaultValue: VD, + uniqueEdges: Option[PartitionStrategy] = None): Graph[VD, Int] = { + val edges = rawEdges.map(p => Edge(p._1, p._2, 1)) + val graph = GraphImpl(edges, defaultValue) + uniqueEdges match { + case Some(p) => graph.partitionBy(p).groupEdges((a, b) => a + b) + case None => graph + } + } + + /** + * Construct a graph from a collection of edges. + * + * @param edges the RDD containing the set of edges in the graph + * @param defaultValue the default vertex attribute to use for each vertex + * + * @return a graph with edge attributes described by `edges` and vertices + * given by all vertices in `edges` with value `defaultValue` + */ + def fromEdges[VD: ClassTag, ED: ClassTag]( + edges: RDD[Edge[ED]], + defaultValue: VD): Graph[VD, ED] = { + GraphImpl(edges, defaultValue) + } + + /** + * Construct a graph from a collection attributed vertices and + * edges. Duplicate vertices are picked arbitrarily and + * vertices found in the edge collection but not in the input + * vertices are the default attribute. + * + * @tparam VD the vertex attribute type + * @tparam ED the edge attribute type + * @param vertices the "set" of vertices and their attributes + * @param edges the collection of edges in the graph + * @param defaultVertexAttr the default vertex attribute to use for + * vertices that are mentioned in edges but not in vertices + * @param partitionStrategy the partition strategy to use when + * partitioning the edges. + */ + def apply[VD: ClassTag, ED: ClassTag]( + vertices: RDD[(VertexID, VD)], + edges: RDD[Edge[ED]], + defaultVertexAttr: VD = null.asInstanceOf[VD]): Graph[VD, ED] = { + GraphImpl(vertices, edges, defaultVertexAttr) + } + + /** + * The implicit graphToGraphOPs function extracts the GraphOps member from a graph. + * + * To improve modularity the Graph type only contains a small set of basic operations. All the + * convenience operations are defined in the GraphOps class which may be shared across multiple + * graph implementations. + */ + implicit def graphToGraphOps[VD: ClassTag, ED: ClassTag](g: Graph[VD, ED]) = g.ops +} // end of Graph object diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala new file mode 100644 index 0000000000..f8aab951f0 --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala @@ -0,0 +1,28 @@ +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 + + +class GraphKryoRegistrator extends KryoRegistrator { + + def registerClasses(kryo: Kryo) { + kryo.register(classOf[Edge[Object]]) + kryo.register(classOf[MessageToPartition[Object]]) + kryo.register(classOf[VertexBroadcastMsg[Object]]) + kryo.register(classOf[(VertexID, Object)]) + kryo.register(classOf[EdgePartition[Object]]) + kryo.register(classOf[BitSet]) + kryo.register(classOf[VertexIdToIndexMap]) + kryo.register(classOf[VertexAttributeBlock[Object]]) + kryo.register(classOf[PartitionStrategy]) + kryo.register(classOf[BoundedPriorityQueue[Object]]) + + // This avoids a large number of hash table lookups. + kryo.setReferences(false) + } +} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphLab.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphLab.scala new file mode 100644 index 0000000000..437288405f --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphLab.scala @@ -0,0 +1,134 @@ +package org.apache.spark.graphx + +import scala.reflect.ClassTag + +import org.apache.spark.Logging +import scala.collection.JavaConversions._ +import org.apache.spark.rdd.RDD + +/** + * This object implements the GraphLab gather-apply-scatter api. + */ +object GraphLab extends Logging { + + /** + * Execute the GraphLab Gather-Apply-Scatter API + * + * @todo finish documenting GraphLab Gather-Apply-Scatter API + * + * @param graph The graph on which to execute the GraphLab API + * @param gatherFunc The gather function is executed on each edge triplet + * adjacent to a vertex and returns an accumulator which + * is then merged using the merge function. + * @param mergeFunc An accumulative associative operation on the result of + * the gather type. + * @param applyFunc Takes a vertex and the final result of the merge operations + * on the adjacent edges and returns a new vertex value. + * @param scatterFunc Executed after the apply function the scatter function takes + * a triplet and signals whether the neighboring vertex program + * must be recomputed. + * @param startVertices predicate to determine which vertices to start the computation on. + * these will be the active vertices in the first iteration. + * @param numIter The maximum number of iterations to run. + * @param gatherDirection The direction of edges to consider during the gather phase + * @param scatterDirection The direction of edges to consider during the scatter phase + * + * @tparam VD The graph vertex attribute type + * @tparam ED The graph edge attribute type + * @tparam A The type accumulated during the gather phase + * @return the resulting graph after the algorithm converges + */ + def apply[VD: ClassTag, ED: ClassTag, A: ClassTag] + (graph: Graph[VD, ED], numIter: Int, + gatherDirection: EdgeDirection = EdgeDirection.In, + scatterDirection: EdgeDirection = EdgeDirection.Out) + (gatherFunc: (VertexID, EdgeTriplet[VD, ED]) => A, + mergeFunc: (A, A) => A, + applyFunc: (VertexID, VD, Option[A]) => VD, + scatterFunc: (VertexID, EdgeTriplet[VD, ED]) => Boolean, + startVertices: (VertexID, VD) => Boolean = (vid: VertexID, data: VD) => true) + : Graph[VD, ED] = { + + + // Add an active attribute to all vertices to track convergence. + var activeGraph: Graph[(Boolean, VD), ED] = graph.mapVertices { + case (id, data) => (startVertices(id, data), data) + }.cache() + + // The gather function wrapper strips the active attribute and + // only invokes the gather function on active vertices + def gather(vid: VertexID, e: EdgeTriplet[(Boolean, VD), ED]): Option[A] = { + if (e.vertexAttr(vid)._1) { + val edgeTriplet = new EdgeTriplet[VD,ED] + edgeTriplet.set(e) + edgeTriplet.srcAttr = e.srcAttr._2 + edgeTriplet.dstAttr = e.dstAttr._2 + Some(gatherFunc(vid, edgeTriplet)) + } else { + None + } + } + + // The apply function wrapper strips the vertex of the active attribute + // and only invokes the apply function on active vertices + def apply(vid: VertexID, data: (Boolean, VD), accum: Option[A]): (Boolean, VD) = { + val (active, vData) = data + if (active) (true, applyFunc(vid, vData, accum)) + else (false, vData) + } + + // The scatter function wrapper strips the vertex of the active attribute + // and only invokes the scatter function on active vertices + def scatter(rawVertexID: VertexID, e: EdgeTriplet[(Boolean, VD), ED]): Option[Boolean] = { + val vid = e.otherVertexId(rawVertexID) + if (e.vertexAttr(vid)._1) { + val edgeTriplet = new EdgeTriplet[VD,ED] + edgeTriplet.set(e) + edgeTriplet.srcAttr = e.srcAttr._2 + edgeTriplet.dstAttr = e.dstAttr._2 + Some(scatterFunc(vid, edgeTriplet)) + } else { + None + } + } + + // Used to set the active status of vertices for the next round + def applyActive( + vid: VertexID, data: (Boolean, VD), newActiveOpt: Option[Boolean]): (Boolean, VD) = { + val (prevActive, vData) = data + (newActiveOpt.getOrElse(false), vData) + } + + // Main Loop --------------------------------------------------------------------- + var i = 0 + var numActive = activeGraph.numVertices + while (i < numIter && numActive > 0) { + + // Gather + val gathered: RDD[(VertexID, A)] = + activeGraph.aggregateNeighbors(gather, mergeFunc, gatherDirection) + + // Apply + activeGraph = activeGraph.outerJoinVertices(gathered)(apply).cache() + + + + // Scatter is basically a gather in the opposite direction so we reverse the edge direction + // activeGraph: Graph[(Boolean, VD), ED] + val scattered: RDD[(VertexID, Boolean)] = + activeGraph.aggregateNeighbors(scatter, _ || _, scatterDirection.reverse) + + activeGraph = activeGraph.outerJoinVertices(scattered)(applyActive).cache() + + // Calculate the number of active vertices + numActive = activeGraph.vertices.map{ + case (vid, data) => if (data._1) 1 else 0 + }.reduce(_ + _) + logInfo("Number active vertices: " + numActive) + i += 1 + } + + // Remove the active attribute from the vertex data before returning the graph + activeGraph.mapVertices{case (vid, data) => data._2 } + } +} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala new file mode 100644 index 0000000000..473cfb18cf --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala @@ -0,0 +1,113 @@ +package org.apache.spark.graphx + +import java.util.{Arrays => JArrays} +import scala.reflect.ClassTag + +import org.apache.spark.graphx.impl.EdgePartitionBuilder +import org.apache.spark.{Logging, SparkContext} +import org.apache.spark.graphx.impl.{EdgePartition, GraphImpl} +import org.apache.spark.util.collection.PrimitiveVector + + +object GraphLoader extends Logging { + + /** + * Load an edge list from file initializing the Graph + * + * @tparam ED the type of the edge data of the resulting Graph + * + * @param sc the SparkContext used to construct RDDs + * @param path the path to the text file containing the edge list + * @param edgeParser a function that takes an array of strings and + * returns an ED object + * @param minEdgePartitions the number of partitions for the + * the Edge RDD + * + */ + def textFile[ED: ClassTag]( + sc: SparkContext, + path: String, + edgeParser: Array[String] => ED, + minEdgePartitions: Int = 1): + Graph[Int, ED] = { + // Parse the edge data table + val edges = sc.textFile(path, minEdgePartitions).mapPartitions( iter => + iter.filter(line => !line.isEmpty && line(0) != '#').map { line => + val lineArray = line.split("\\s+") + if(lineArray.length < 2) { + println("Invalid line: " + line) + assert(false) + } + val source = lineArray(0).trim.toLong + val target = lineArray(1).trim.toLong + val tail = lineArray.drop(2) + val edata = edgeParser(tail) + Edge(source, target, edata) + }) + val defaultVertexAttr = 1 + Graph.fromEdges(edges, defaultVertexAttr) + } + + /** + * Load a graph from an edge list formatted file with each line containing + * two integers: a source Id and a target Id. + * + * @example A file in the following format: + * {{{ + * # Comment Line + * # Source Id <\t> Target Id + * 1 -5 + * 1 2 + * 2 7 + * 1 8 + * }}} + * + * If desired the edges can be automatically oriented in the positive + * direction (source Id < target Id) by setting `canonicalOrientation` to + * true + * + * @param sc + * @param path the path to the file (e.g., /Home/data/file or hdfs://file) + * @param canonicalOrientation whether to orient edges in the positive + * direction. + * @param minEdgePartitions the number of partitions for the + * the Edge RDD + * @tparam ED + * @return + */ + def edgeListFile( + sc: SparkContext, + path: String, + canonicalOrientation: Boolean = false, + minEdgePartitions: Int = 1): + Graph[Int, Int] = { + 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] + iter.foreach { line => + if (!line.isEmpty && line(0) != '#') { + val lineArray = line.split("\\s+") + if (lineArray.length < 2) { + logWarning("Invalid line: " + line) + } + val srcId = lineArray(0).toLong + val dstId = lineArray(1).toLong + if (canonicalOrientation && dstId > srcId) { + builder.add(dstId, srcId, 1) + } else { + builder.add(srcId, dstId, 1) + } + } + } + Iterator((pid, builder.toEdgePartition)) + }.cache() + edges.count() + + logInfo("It took %d ms to load the edges".format(System.currentTimeMillis - startTime)) + + GraphImpl.fromEdgePartitions(edges, defaultVertexAttr = 1) + } // end of edgeListFile + +} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala new file mode 100644 index 0000000000..cacfcb1c90 --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala @@ -0,0 +1,277 @@ +package org.apache.spark.graphx + +import scala.reflect.ClassTag + +import org.apache.spark.rdd.RDD +import org.apache.spark.SparkContext._ +import org.apache.spark.SparkException + + +/** + * `GraphOps` contains additional functionality (syntatic sugar) for + * the graph type and is implicitly constructed for each Graph object. + * All operations in `GraphOps` are expressed in terms of the + * efficient GraphX API. + * + * @tparam VD the vertex attribute type + * @tparam ED the edge attribute type + * + */ +class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) { + + /** + * Compute the number of edges in the graph. + */ + lazy val numEdges: Long = graph.edges.count() + + + /** + * Compute the number of vertices in the graph. + */ + lazy val numVertices: Long = graph.vertices.count() + + + /** + * Compute the in-degree of each vertex in the Graph returning an + * RDD. + * @note Vertices with no in edges are not returned in the resulting RDD. + */ + lazy val inDegrees: VertexRDD[Int] = degreesRDD(EdgeDirection.In) + + + /** + * Compute the out-degree of each vertex in the Graph returning an RDD. + * @note Vertices with no out edges are not returned in the resulting RDD. + */ + lazy val outDegrees: VertexRDD[Int] = degreesRDD(EdgeDirection.Out) + + + /** + * Compute the degrees of each vertex in the Graph returning an RDD. + * @note Vertices with no edges are not returned in the resulting + * RDD. + */ + lazy val degrees: VertexRDD[Int] = degreesRDD(EdgeDirection.Both) + + + /** + * Compute the neighboring vertex degrees. + * + * @param edgeDirection the direction along which to collect + * neighboring vertex attributes. + */ + private def degreesRDD(edgeDirection: EdgeDirection): VertexRDD[Int] = { + if (edgeDirection == EdgeDirection.In) { + graph.mapReduceTriplets(et => Iterator((et.dstId,1)), _ + _) + } else if (edgeDirection == EdgeDirection.Out) { + graph.mapReduceTriplets(et => Iterator((et.srcId,1)), _ + _) + } else { // EdgeDirection.both + graph.mapReduceTriplets(et => Iterator((et.srcId,1), (et.dstId,1)), _ + _) + } + } + + + /** + * This function is used to compute a statistic for the neighborhood + * of each vertex and returns a value for all vertices (including + * those without neighbors). + * + * @note Because the a default value is provided all vertices will + * have a corresponding entry in the returned RDD. + * + * @param mapFunc the function applied to each edge adjacent to each + * vertex. The mapFunc can optionally return None in which case it + * does not contribute to the final sum. + * @param reduceFunc the function used to merge the results of each + * map operation. + * @param default the default value to use for each vertex if it has + * no neighbors or the map function repeatedly evaluates to none + * @param direction the direction of edges to consider (e.g., In, + * Out, Both). + * @tparam VD2 The returned type of the aggregation operation. + * + * @return A Spark.RDD containing tuples of vertex identifiers and + * their resulting value. There will be exactly one entry for ever + * vertex in the original graph. + * + * @example We can use this function to compute the average follower + * age for each user + * + * {{{ + * val graph: Graph[Int,Int] = loadGraph() + * val averageFollowerAge: RDD[(Int, Int)] = + * graph.aggregateNeighbors[(Int,Double)]( + * (vid, edge) => (edge.otherVertex(vid).data, 1), + * (a, b) => (a._1 + b._1, a._2 + b._2), + * -1, + * EdgeDirection.In) + * .mapValues{ case (sum,followers) => sum.toDouble / followers} + * }}} + * + * @todo Should this return a graph with the new vertex values? + * + */ + def aggregateNeighbors[A: ClassTag]( + mapFunc: (VertexID, EdgeTriplet[VD, ED]) => Option[A], + reduceFunc: (A, A) => A, + dir: EdgeDirection) + : VertexRDD[A] = { + + // Define a new map function over edge triplets + val mf = (et: EdgeTriplet[VD,ED]) => { + // Compute the message to the dst vertex + val dst = + if (dir == EdgeDirection.In || dir == EdgeDirection.Both) { + mapFunc(et.dstId, et) + } else { Option.empty[A] } + // Compute the message to the source vertex + val src = + if (dir == EdgeDirection.Out || dir == EdgeDirection.Both) { + mapFunc(et.srcId, et) + } else { Option.empty[A] } + // construct the return array + (src, dst) match { + case (None, None) => Iterator.empty + case (Some(srcA),None) => Iterator((et.srcId, srcA)) + case (None, Some(dstA)) => Iterator((et.dstId, dstA)) + case (Some(srcA), Some(dstA)) => Iterator((et.srcId, srcA), (et.dstId, dstA)) + } + } + + graph.mapReduceTriplets(mf, reduceFunc) + } // end of aggregateNeighbors + + + /** + * Return the Ids of the neighboring vertices. + * + * @param edgeDirection the direction along which to collect + * neighboring vertices + * + * @return the vertex set of neighboring ids for each vertex. + */ + def collectNeighborIds(edgeDirection: EdgeDirection) : + VertexRDD[Array[VertexID]] = { + val nbrs = + if (edgeDirection == EdgeDirection.Both) { + graph.mapReduceTriplets[Array[VertexID]]( + mapFunc = et => Iterator((et.srcId, Array(et.dstId)), (et.dstId, Array(et.srcId))), + reduceFunc = _ ++ _ + ) + } else if (edgeDirection == EdgeDirection.Out) { + graph.mapReduceTriplets[Array[VertexID]]( + mapFunc = et => Iterator((et.srcId, Array(et.dstId))), + reduceFunc = _ ++ _) + } else if (edgeDirection == EdgeDirection.In) { + graph.mapReduceTriplets[Array[VertexID]]( + mapFunc = et => Iterator((et.dstId, Array(et.srcId))), + reduceFunc = _ ++ _) + } else { + throw new SparkException("It doesn't make sense to collect neighbor ids without a direction.") + } + graph.vertices.leftZipJoin(nbrs) { (vid, vdata, nbrsOpt) => + nbrsOpt.getOrElse(Array.empty[VertexID]) + } + } // end of collectNeighborIds + + + /** + * Collect the neighbor vertex attributes for each vertex. + * + * @note This function could be highly inefficient on power-law + * graphs where high degree vertices may force a large ammount of + * information to be collected to a single location. + * + * @param edgeDirection the direction along which to collect + * neighboring vertices + * + * @return the vertex set of neighboring vertex attributes for each + * vertex. + */ + def collectNeighbors(edgeDirection: EdgeDirection) : + VertexRDD[ Array[(VertexID, VD)] ] = { + val nbrs = graph.aggregateNeighbors[Array[(VertexID,VD)]]( + (vid, edge) => + Some(Array( (edge.otherVertexId(vid), edge.otherVertexAttr(vid)) )), + (a, b) => a ++ b, + edgeDirection) + + graph.vertices.leftZipJoin(nbrs) { (vid, vdata, nbrsOpt) => + nbrsOpt.getOrElse(Array.empty[(VertexID, VD)]) + } + } // end of collectNeighbor + + + /** + * Join the vertices with an RDD and then apply a function from the + * the vertex and RDD entry to a new vertex value. The input table + * should contain at most one entry for each vertex. If no entry is + * provided the map function is skipped and the old value is used. + * + * @tparam U the type of entry in the table of updates + * @param table the table to join with the vertices in the graph. + * The table should contain at most one entry for each vertex. + * @param mapFunc the function used to compute the new vertex + * values. The map function is invoked only for vertices with a + * corresponding entry in the table otherwise the old vertex value + * is used. + * + * @note for small tables this function can be much more efficient + * than leftJoinVertices + * + * @example This function is used to update the vertices with new + * values based on external data. For example we could add the out + * degree to each vertex record + * + * {{{ + * val rawGraph: Graph[Int,()] = Graph.textFile("webgraph") + * .mapVertices(v => 0) + * val outDeg: RDD[(Int, Int)] = rawGraph.outDegrees() + * val graph = rawGraph.leftJoinVertices[Int,Int](outDeg, + * (v, deg) => deg ) + * }}} + * + */ + def joinVertices[U: ClassTag](table: RDD[(VertexID, U)])(mapFunc: (VertexID, VD, U) => VD) + : Graph[VD, ED] = { + val uf = (id: VertexID, data: VD, o: Option[U]) => { + o match { + case Some(u) => mapFunc(id, data, u) + case None => data + } + } + graph.outerJoinVertices(table)(uf) + } + + /** + * Filter the graph by computing some values to filter on, and applying the predicates. + * + * @param preprocess a function to compute new vertex and edge data before filtering + * @param epred edge pred to filter on after preprocess, see more details under Graph#subgraph + * @param vpred vertex pred to filter on after prerocess, see more details under Graph#subgraph + * @tparam VD2 vertex type the vpred operates on + * @tparam ED2 edge type the epred operates on + * @return a subgraph of the orginal graph, with its data unchanged + * + * @example This function can be used to filter the graph based on some property, without + * changing the vertex and edge values in your program. For example, we could remove the vertices + * in a graph with 0 outdegree + * + * {{{ + * graph.filter( + * graph => { + * val degrees: VertexSetRDD[Int] = graph.outDegrees + * graph.outerJoinVertices(degrees) {(vid, data, deg) => deg.getOrElse(0)} + * }, + * vpred = (vid: VertexID, deg:Int) => deg > 0 + * ) + * }}} + * + */ + def filter[VD2: ClassTag, ED2: ClassTag]( + preprocess: Graph[VD, ED] => Graph[VD2, ED2], + epred: (EdgeTriplet[VD2, ED2]) => Boolean = (x: EdgeTriplet[VD2, ED2]) => true, + vpred: (VertexID, VD2) => Boolean = (v:VertexID, d:VD2) => true): Graph[VD, ED] = { + graph.mask(preprocess(graph).subgraph(epred, vpred)) + } +} // end of GraphOps diff --git a/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala b/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala new file mode 100644 index 0000000000..5e80a535f1 --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala @@ -0,0 +1,94 @@ +package org.apache.spark.graphx + + +sealed trait PartitionStrategy extends Serializable { + def getPartition(src: VertexID, dst: VertexID, numParts: PartitionID): PartitionID +} + + +/** + * This function implements a classic 2D-Partitioning of a sparse matrix. + * Suppose we have a graph with 11 vertices that we want to partition + * over 9 machines. We can use the following sparse matrix representation: + * + * __________________________________ + * v0 | P0 * | P1 | P2 * | + * v1 | **** | * | | + * v2 | ******* | ** | **** | + * v3 | ***** | * * | * | + * ---------------------------------- + * v4 | P3 * | P4 *** | P5 ** * | + * v5 | * * | * | | + * v6 | * | ** | **** | + * v7 | * * * | * * | * | + * ---------------------------------- + * v8 | P6 * | P7 * | P8 * *| + * v9 | * | * * | | + * v10 | * | ** | * * | + * v11 | * <-E | *** | ** | + * ---------------------------------- + * + * The edge denoted by E connects v11 with v1 and is assigned to + * processor P6. To get the processor number we divide the matrix + * into sqrt(numProc) by sqrt(numProc) blocks. Notice that edges + * adjacent to v11 can only be in the first colum of + * blocks (P0, P3, P6) or the last row of blocks (P6, P7, P8). + * As a consequence we can guarantee that v11 will need to be + * replicated to at most 2 * sqrt(numProc) machines. + * + * Notice that P0 has many edges and as a consequence this + * partitioning would lead to poor work balance. To improve + * balance we first multiply each vertex id by a large prime + * to effectively shuffle the vertex locations. + * + * One of the limitations of this approach is that the number of + * machines must either be a perfect square. We partially address + * this limitation by computing the machine assignment to the next + * largest perfect square and then mapping back down to the actual + * number of machines. Unfortunately, this can also lead to work + * imbalance and so it is suggested that a perfect square is used. + * + * + */ +case object EdgePartition2D extends PartitionStrategy { + override def getPartition(src: VertexID, dst: VertexID, numParts: PartitionID): PartitionID = { + val ceilSqrtNumParts: PartitionID = math.ceil(math.sqrt(numParts)).toInt + val mixingPrime: VertexID = 1125899906842597L + val col: PartitionID = ((math.abs(src) * mixingPrime) % ceilSqrtNumParts).toInt + val row: PartitionID = ((math.abs(dst) * mixingPrime) % ceilSqrtNumParts).toInt + (col * ceilSqrtNumParts + row) % numParts + } +} + + +case object EdgePartition1D extends PartitionStrategy { + override def getPartition(src: VertexID, dst: VertexID, numParts: PartitionID): PartitionID = { + val mixingPrime: VertexID = 1125899906842597L + (math.abs(src) * mixingPrime).toInt % numParts + } +} + + +/** + * Assign edges to an aribtrary machine corresponding to a + * random vertex cut. + */ +case object RandomVertexCut extends PartitionStrategy { + override def getPartition(src: VertexID, dst: VertexID, numParts: PartitionID): PartitionID = { + math.abs((src, dst).hashCode()) % numParts + } +} + + +/** + * Assign edges to an arbitrary machine corresponding to a random vertex cut. This + * function ensures that edges of opposite direction between the same two vertices + * will end up on the same partition. + */ +case object CanonicalRandomVertexCut extends PartitionStrategy { + override def getPartition(src: VertexID, dst: VertexID, numParts: PartitionID): PartitionID = { + val lower = math.min(src, dst) + val higher = math.max(src, dst) + math.abs((lower, higher).hashCode()) % numParts + } +} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala b/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala new file mode 100644 index 0000000000..8ddb788135 --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala @@ -0,0 +1,122 @@ +package org.apache.spark.graphx + +import scala.reflect.ClassTag + + +/** + * This object implements a Pregel-like bulk-synchronous + * message-passing API. However, unlike the original Pregel API the + * GraphX pregel API factors the sendMessage computation over edges, + * enables the message sending computation to read both vertex + * attributes, and finally constrains messages to the graph structure. + * These changes allow for substantially more efficient distributed + * execution while also exposing greater flexibility for graph based + * computation. + * + * @example We can use the Pregel abstraction to implement PageRank + * {{{ + * val pagerankGraph: Graph[Double, Double] = graph + * // Associate the degree with each vertex + * .outerJoinVertices(graph.outDegrees){ + * (vid, vdata, deg) => deg.getOrElse(0) + * } + * // Set the weight on the edges based on the degree + * .mapTriplets( e => 1.0 / e.srcAttr ) + * // Set the vertex attributes to the initial pagerank values + * .mapVertices( (id, attr) => 1.0 ) + * + * def vertexProgram(id: VertexID, attr: Double, msgSum: Double): Double = + * resetProb + (1.0 - resetProb) * msgSum + * def sendMessage(id: VertexID, edge: EdgeTriplet[Double, Double]): Option[Double] = + * Some(edge.srcAttr * edge.attr) + * def messageCombiner(a: Double, b: Double): Double = a + b + * val initialMessage = 0.0 + * // Execute pregel for a fixed number of iterations. + * Pregel(pagerankGraph, initialMessage, numIter)( + * vertexProgram, sendMessage, messageCombiner) + * }}} + * + */ +object Pregel { + + /** + * Execute a Pregel-like iterative vertex-parallel abstraction. The + * user-defined vertex-program `vprog` is executed in parallel on + * each vertex receiving any inbound messages and computing a new + * value for the vertex. The `sendMsg` function is then invoked on + * all out-edges and is used to compute an optional message to the + * destination vertex. The `mergeMsg` function is a commutative + * associative function used to combine messages destined to the + * same vertex. + * + * On the first iteration all vertices receive the `initialMsg` and + * on subsequent iterations if a vertex does not receive a message + * then the vertex-program is not invoked. + * + * This function iterates until there are no remaining messages, or + * for maxIterations iterations. + * + * @tparam VD the vertex data type + * @tparam ED the edge data type + * @tparam A the Pregel message type + * + * @param graph the input graph. + * + * @param initialMsg the message each vertex will receive at the on + * the first iteration. + * + * @param maxIterations the maximum number of iterations to run for. + * + * @param vprog the user-defined vertex program which runs on each + * vertex and receives the inbound message and computes a new vertex + * value. On the first iteration the vertex program is invoked on + * all vertices and is passed the default message. On subsequent + * iterations the vertex program is only invoked on those vertices + * that receive messages. + * + * @param sendMsg a user supplied function that is applied to out + * edges of vertices that received messages in the current + * iteration. + * + * @param mergeMsg a user supplied function that takes two incoming + * messages of type A and merges them into a single message of type + * A. ''This function must be commutative and associative and + * ideally the size of A should not increase.'' + * + * @return the resulting graph at the end of the computation + * + */ + def apply[VD: ClassTag, ED: ClassTag, A: ClassTag] + (graph: Graph[VD, ED], initialMsg: A, maxIterations: Int = Int.MaxValue)( + vprog: (VertexID, VD, A) => VD, + sendMsg: EdgeTriplet[VD, ED] => Iterator[(VertexID,A)], + mergeMsg: (A, A) => A) + : Graph[VD, ED] = { + + var g = graph.mapVertices( (vid, vdata) => vprog(vid, vdata, initialMsg) ) + // compute the messages + var messages = g.mapReduceTriplets(sendMsg, mergeMsg).cache() + var activeMessages = messages.count() + // Loop + var i = 0 + while (activeMessages > 0 && i < maxIterations) { + // Receive the messages. Vertices that didn't get any messages do not appear in newVerts. + val newVerts = g.vertices.innerJoin(messages)(vprog).cache() + // Update the graph with the new vertices. + g = g.outerJoinVertices(newVerts) { (vid, old, newOpt) => newOpt.getOrElse(old) } + + val oldMessages = messages + // Send new messages. Vertices that didn't get any messages don't appear in newVerts, so don't + // get to send messages. + messages = g.mapReduceTriplets(sendMsg, mergeMsg, Some((newVerts, EdgeDirection.Out))).cache() + activeMessages = messages.count() + // after counting we can unpersist the old messages + oldMessages.unpersist(blocking=false) + // count the iteration + i += 1 + } + + g + } // end of apply + +} // end of class Pregel diff --git a/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala new file mode 100644 index 0000000000..cfee9b089f --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala @@ -0,0 +1,361 @@ +/* + * 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 + +import scala.reflect.ClassTag + +import org.apache.spark._ +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 + + +/** + * A `VertexRDD[VD]` extends the `RDD[(VertexID, VD)]` by ensuring that there is + * only one entry for each vertex and by pre-indexing the entries for fast, + * efficient joins. + * + * @tparam VD the vertex attribute associated with each vertex in the set. + * + * To construct a `VertexRDD` use the singleton object: + * + * @example Construct a `VertexRDD` from a plain RDD + * {{{ + * // Construct an intial vertex set + * val someData: RDD[(VertexID, SomeType)] = loadData(someFile) + * val vset = VertexRDD(someData) + * // If there were redundant values in someData we would use a reduceFunc + * val vset2 = VertexRDD(someData, reduceFunc) + * // Finally we can use the VertexRDD to index another dataset + * val otherData: RDD[(VertexID, OtherType)] = loadData(otherFile) + * val vset3 = VertexRDD(otherData, vset.index) + * // Now we can construct very fast joins between the two sets + * val vset4: VertexRDD[(SomeType, OtherType)] = vset.leftJoin(vset3) + * }}} + * + */ +class VertexRDD[@specialized VD: ClassTag]( + val partitionsRDD: RDD[VertexPartition[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 keys in the RDD. + * The resulting VertexRDD will be based on a different index and can + * no longer be quickly joined with this RDD. + */ + def reindex(): VertexRDD[VD] = new VertexRDD(partitionsRDD.map(_.reindex())) + + /** + * The partitioner is defined by the index. + */ + override val partitioner = partitionsRDD.partitioner + + /** + * The actual partitions are defined by the tuples. + */ + override protected def getPartitions: Array[Partition] = partitionsRDD.partitions + + /** + * The preferred locations are computed based on the preferred + * locations of the tuples. + */ + override protected def getPreferredLocations(s: Partition): Seq[String] = + partitionsRDD.preferredLocations(s) + + /** + * Caching a VertexRDD causes the index and values to be cached separately. + */ + override def persist(newLevel: StorageLevel): VertexRDD[VD] = { + partitionsRDD.persist(newLevel) + this + } + + /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */ + override def persist(): VertexRDD[VD] = persist(StorageLevel.MEMORY_ONLY) + + /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */ + override def cache(): VertexRDD[VD] = persist() + + /** Return the number of vertices in this set. */ + override def count(): Long = { + partitionsRDD.map(_.size).reduce(_ + _) + } + + /** + * Provide the `RDD[(VertexID, VD)]` equivalent output. + */ + override def compute(part: Partition, context: TaskContext): Iterator[(VertexID, VD)] = { + firstParent[VertexPartition[VD]].iterator(part, context).next.iterator + } + + /** + * Return a new VertexRDD by applying a function to each VertexPartition of this RDD. + */ + def mapVertexPartitions[VD2: ClassTag](f: VertexPartition[VD] => VertexPartition[VD2]) + : VertexRDD[VD2] = { + val newPartitionsRDD = partitionsRDD.mapPartitions(_.map(f), preservesPartitioning = true) + new VertexRDD(newPartitionsRDD) + } + + + /** + * Restrict the vertex set to the set of vertices satisfying the + * given predicate. + * + * @param pred the user defined predicate, which takes a tuple to conform to + * the RDD[(VertexID, VD)] interface + * + * @note The vertex set preserves the original index structure + * which means that the returned RDD can be easily joined with + * the original vertex-set. Furthermore, the filter only + * modifies the bitmap index and so no new values are allocated. + */ + override def filter(pred: Tuple2[VertexID, VD] => Boolean): VertexRDD[VD] = + this.mapVertexPartitions(_.filter(Function.untupled(pred))) + + /** + * Pass each vertex attribute through a map function and retain the + * original RDD's partitioning and index. + * + * @tparam VD2 the type returned by the map function + * + * @param f the function applied to each value in the RDD + * @return a new VertexRDD with values obtained by applying `f` to + * each of the entries in the original VertexRDD. The resulting + * VertexRDD retains the same index. + */ + def mapValues[VD2: ClassTag](f: VD => VD2): VertexRDD[VD2] = + this.mapVertexPartitions(_.map((vid, attr) => f(attr))) + + /** + * Pass each vertex attribute through a map function and retain the + * original RDD's partitioning and index. + * + * @tparam VD2 the type returned by the map function + * + * @param f the function applied to each value in the RDD + * @return a new VertexRDD with values obtained by applying `f` to + * each of the entries in the original VertexRDD. The resulting + * VertexRDD retains the same index. + */ + def mapValues[VD2: ClassTag](f: (VertexID, VD) => VD2): VertexRDD[VD2] = + this.mapVertexPartitions(_.map(f)) + + /** + * Hides vertices that are the same between this and other. For vertices that are different, keeps + * the values from `other`. + */ + def diff(other: VertexRDD[VD]): VertexRDD[VD] = { + val newPartitionsRDD = partitionsRDD.zipPartitions( + other.partitionsRDD, preservesPartitioning = true + ) { (thisIter, otherIter) => + val thisPart = thisIter.next() + val otherPart = otherIter.next() + Iterator(thisPart.diff(otherPart)) + } + new VertexRDD(newPartitionsRDD) + } + + /** + * Left join this VertexSet with another VertexSet which has the + * same Index. This function will fail if both VertexSets do not + * share the same index. The resulting vertex set contains an entry + * for each vertex in this set. If the other VertexSet is missing + * any vertex in this VertexSet then a `None` attribute is generated + * + * @tparam VD2 the attribute type of the other VertexSet + * @tparam VD3 the attribute type of the resulting VertexSet + * + * @param other the other VertexSet with which to join. + * @param f the function mapping a vertex id and its attributes in + * this and the other vertex set to a new vertex attribute. + * @return a VertexRDD containing all the vertices in this + * VertexSet with `None` attributes used for Vertices missing in the + * other VertexSet. + * + */ + def leftZipJoin[VD2: ClassTag, VD3: ClassTag] + (other: VertexRDD[VD2])(f: (VertexID, VD, Option[VD2]) => VD3): VertexRDD[VD3] = { + val newPartitionsRDD = partitionsRDD.zipPartitions( + other.partitionsRDD, preservesPartitioning = true + ) { (thisIter, otherIter) => + val thisPart = thisIter.next() + val otherPart = otherIter.next() + Iterator(thisPart.leftJoin(otherPart)(f)) + } + new VertexRDD(newPartitionsRDD) + } + + /** + * Left join this VertexRDD with an RDD containing vertex attribute + * pairs. If the other RDD is backed by a VertexRDD with the same + * index than the efficient leftZipJoin implementation is used. The + * resulting vertex set contains an entry for each vertex in this + * set. If the other VertexRDD is missing any vertex in this + * VertexRDD then a `None` attribute is generated. + * + * If there are duplicates, the vertex is picked at random. + * + * @tparam VD2 the attribute type of the other VertexRDD + * @tparam VD3 the attribute type of the resulting VertexRDD + * + * @param other the other VertexRDD with which to join. + * @param f the function mapping a vertex id and its attributes in + * this and the other vertex set to a new vertex attribute. + * @return a VertexRDD containing all the vertices in this + * VertexRDD with the attribute emitted by f. + */ + def leftJoin[VD2: ClassTag, VD3: ClassTag] + (other: RDD[(VertexID, VD2)]) + (f: (VertexID, VD, Option[VD2]) => VD3) + : VertexRDD[VD3] = + { + // Test if the other vertex is a VertexRDD to choose the optimal join strategy. + // If the other set is a VertexRDD then we use the much more efficient leftZipJoin + other match { + case other: VertexRDD[_] => + leftZipJoin(other)(f) + 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)) + } + ) + } + } + + /** + * Same effect as leftJoin(other) { (vid, a, bOpt) => bOpt.getOrElse(a) }, but `this` and `other` + * must have the same index. + */ + def innerZipJoin[U: ClassTag, VD2: ClassTag](other: VertexRDD[U]) + (f: (VertexID, VD, U) => VD2): VertexRDD[VD2] = { + val newPartitionsRDD = partitionsRDD.zipPartitions( + other.partitionsRDD, preservesPartitioning = true + ) { (thisIter, otherIter) => + val thisPart = thisIter.next() + val otherPart = otherIter.next() + Iterator(thisPart.innerJoin(otherPart)(f)) + } + new VertexRDD(newPartitionsRDD) + } + + /** + * Replace vertices with corresponding vertices in `other`, and drop vertices without a + * corresponding vertex in `other`. + */ + def innerJoin[U: ClassTag, VD2: ClassTag](other: RDD[(VertexID, U)]) + (f: (VertexID, VD, U) => VD2): VertexRDD[VD2] = { + // Test if the other vertex is a VertexRDD to choose the optimal join strategy. + // If the other set is a VertexRDD then we use the much more efficient innerZipJoin + other match { + case other: VertexRDD[_] => + innerZipJoin(other)(f) + 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)) + } + ) + } + } + + /** + * Aggregate messages with the same ids using `reduceFunc`, returning a VertexRDD that is + * co-indexed with this one. + */ + def aggregateUsingIndex[VD2: ClassTag]( + messages: RDD[(VertexID, VD2)], reduceFunc: (VD2, VD2) => VD2): VertexRDD[VD2] = + { + val shuffled = MsgRDDFunctions.partitionForAggregation(messages, this.partitioner.get) + val parts = partitionsRDD.zipPartitions(shuffled, true) { (thisIter, msgIter) => + val vertexPartition: VertexPartition[VD] = thisIter.next() + Iterator(vertexPartition.aggregateUsingIndex(msgIter, reduceFunc)) + } + new VertexRDD[VD2](parts) + } + +} // end of VertexRDD + + +/** + * The VertexRDD singleton is used to construct VertexRDDs + */ +object VertexRDD { + + /** + * Construct a vertex set 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 + */ + 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)) + } + val vertexPartitions = partitioned.mapPartitions( + iter => Iterator(VertexPartition(iter)), + preservesPartitioning = true) + new VertexRDD(vertexPartitions) + } + + /** + * Construct a vertex set from an RDD of vertex-attribute pairs. + * Duplicate entries are merged using mergeFunc. + * + * @tparam VD the vertex attribute type + * + * @param rdd the collection of vertex-attribute pairs + * @param mergeFunc the associative, commutative merge function. + */ + 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)) + } + val vertexPartitions = partitioned.mapPartitions( + iter => Iterator(VertexPartition(iter)), + preservesPartitioning = true) + new VertexRDD(vertexPartitions) + } + + 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) + } + } +} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/ConnectedComponents.scala b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/ConnectedComponents.scala new file mode 100644 index 0000000000..a0dd36da60 --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/ConnectedComponents.scala @@ -0,0 +1,37 @@ +package org.apache.spark.graphx.algorithms + +import org.apache.spark.graphx._ + + +object ConnectedComponents { + /** + * Compute the connected component membership of each vertex and return an RDD with the vertex + * value containing the lowest vertex id in the connected component containing that vertex. + * + * @tparam VD the vertex attribute type (discarded in the computation) + * @tparam ED the edge attribute type (preserved in the computation) + * + * @param graph the graph for which to compute the connected components + * + * @return a graph with vertex attributes containing the smallest vertex in each + * connected component + */ + def run[VD: Manifest, ED: Manifest](graph: Graph[VD, ED]): Graph[VertexID, ED] = { + val ccGraph = graph.mapVertices { case (vid, _) => vid } + + def sendMessage(edge: EdgeTriplet[VertexID, ED]) = { + if (edge.srcAttr < edge.dstAttr) { + Iterator((edge.dstId, edge.srcAttr)) + } else if (edge.srcAttr > edge.dstAttr) { + Iterator((edge.srcId, edge.dstAttr)) + } else { + Iterator.empty + } + } + val initialMessage = Long.MaxValue + Pregel(ccGraph, initialMessage)( + vprog = (id, attr, msg) => math.min(attr, msg), + sendMsg = sendMessage, + mergeMsg = (a, b) => math.min(a, b)) + } // end of connectedComponents +} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/PageRank.scala b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/PageRank.scala new file mode 100644 index 0000000000..0292b7316d --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/PageRank.scala @@ -0,0 +1,205 @@ +package org.apache.spark.graphx.algorithms + +import org.apache.spark.Logging +import org.apache.spark.graphx._ + + +object PageRank extends Logging { + + /** + * Run PageRank for a fixed number of iterations returning a graph + * with vertex attributes containing the PageRank and edge + * attributes the normalized edge weight. + * + * The following PageRank fixed point is computed for each vertex. + * + * {{{ + * var PR = Array.fill(n)( 1.0 ) + * val oldPR = Array.fill(n)( 1.0 ) + * for( iter <- 0 until numIter ) { + * swap(oldPR, PR) + * for( i <- 0 until n ) { + * PR[i] = alpha + (1 - alpha) * inNbrs[i].map(j => oldPR[j] / outDeg[j]).sum + * } + * } + * }}} + * + * where `alpha` is the random reset probability (typically 0.15), + * `inNbrs[i]` is the set of neighbors whick link to `i` and + * `outDeg[j]` is the out degree of vertex `j`. + * + * Note that this is not the "normalized" PageRank and as a consequence pages that have no + * inlinks will have a PageRank of alpha. + * + * @tparam VD the original vertex attribute (not used) + * @tparam ED the original edge attribute (not used) + * + * @param graph the graph on which to compute PageRank + * @param numIter the number of iterations of PageRank to run + * @param resetProb the random reset probability (alpha) + * + * @return the graph containing with each vertex containing the PageRank and each edge + * containing the normalized weight. + * + */ + def run[VD: Manifest, ED: Manifest]( + graph: Graph[VD, ED], numIter: Int, resetProb: Double = 0.15): Graph[Double, Double] = + { + + /** + * Initialize the pagerankGraph with each edge attribute having + * weight 1/outDegree and each vertex with attribute 1.0. + */ + val pagerankGraph: Graph[Double, Double] = graph + // Associate the degree with each vertex + .outerJoinVertices(graph.outDegrees){ + (vid, vdata, deg) => deg.getOrElse(0) + } + // Set the weight on the edges based on the degree + .mapTriplets( e => 1.0 / e.srcAttr ) + // Set the vertex attributes to the initial pagerank values + .mapVertices( (id, attr) => 1.0 ) + + // Display statistics about pagerank + logInfo(pagerankGraph.statistics.toString) + + // Define the three functions needed to implement PageRank in the GraphX + // version of Pregel + def vertexProgram(id: VertexID, attr: Double, msgSum: Double): Double = + resetProb + (1.0 - resetProb) * msgSum + def sendMessage(edge: EdgeTriplet[Double, Double]) = + Iterator((edge.dstId, edge.srcAttr * edge.attr)) + def messageCombiner(a: Double, b: Double): Double = a + b + // The initial message received by all vertices in PageRank + val initialMessage = 0.0 + + // Execute pregel for a fixed number of iterations. + Pregel(pagerankGraph, initialMessage, numIter)( + vertexProgram, sendMessage, messageCombiner) + } + + /** + * Run a dynamic version of PageRank returning a graph with vertex attributes containing the + * PageRank and edge attributes containing the normalized edge weight. + * + * {{{ + * var PR = Array.fill(n)( 1.0 ) + * val oldPR = Array.fill(n)( 0.0 ) + * while( max(abs(PR - oldPr)) > tol ) { + * swap(oldPR, PR) + * for( i <- 0 until n if abs(PR[i] - oldPR[i]) > tol ) { + * PR[i] = alpha + (1 - \alpha) * inNbrs[i].map(j => oldPR[j] / outDeg[j]).sum + * } + * } + * }}} + * + * where `alpha` is the random reset probability (typically 0.15), `inNbrs[i]` is the set of + * neighbors whick link to `i` and `outDeg[j]` is the out degree of vertex `j`. + * + * Note that this is not the "normalized" PageRank and as a consequence pages that have no + * inlinks will have a PageRank of alpha. + * + * @tparam VD the original vertex attribute (not used) + * @tparam ED the original edge attribute (not used) + * + * @param graph the graph on which to compute PageRank + * @param tol the tolerance allowed at convergence (smaller => more * accurate). + * @param resetProb the random reset probability (alpha) + * + * @return the graph containing with each vertex containing the PageRank and each edge + * containing the normalized weight. + */ + def runUntillConvergence[VD: Manifest, ED: Manifest]( + graph: Graph[VD, ED], tol: Double, resetProb: Double = 0.15): Graph[Double, Double] = + { + // Initialize the pagerankGraph with each edge attribute + // having weight 1/outDegree and each vertex with attribute 1.0. + val pagerankGraph: Graph[(Double, Double), Double] = graph + // Associate the degree with each vertex + .outerJoinVertices(graph.outDegrees) { + (vid, vdata, deg) => deg.getOrElse(0) + } + // Set the weight on the edges based on the degree + .mapTriplets( e => 1.0 / e.srcAttr ) + // Set the vertex attributes to (initalPR, delta = 0) + .mapVertices( (id, attr) => (0.0, 0.0) ) + + // Display statistics about pagerank + logInfo(pagerankGraph.statistics.toString) + + // Define the three functions needed to implement PageRank in the GraphX + // version of Pregel + def vertexProgram(id: VertexID, attr: (Double, Double), msgSum: Double): (Double, Double) = { + val (oldPR, lastDelta) = attr + val newPR = oldPR + (1.0 - resetProb) * msgSum + (newPR, newPR - oldPR) + } + + def sendMessage(edge: EdgeTriplet[(Double, Double), Double]) = { + if (edge.srcAttr._2 > tol) { + Iterator((edge.dstId, edge.srcAttr._2 * edge.attr)) + } else { + Iterator.empty + } + } + + def messageCombiner(a: Double, b: Double): Double = a + b + + // The initial message received by all vertices in PageRank + val initialMessage = resetProb / (1.0 - resetProb) + + // Execute a dynamic version of Pregel. + Pregel(pagerankGraph, initialMessage)(vertexProgram, sendMessage, messageCombiner) + .mapVertices((vid, attr) => attr._1) + } // end of deltaPageRank + + def runStandalone[VD: Manifest, ED: Manifest]( + graph: Graph[VD, ED], tol: Double, resetProb: Double = 0.15): VertexRDD[Double] = { + + // Initialize the ranks + var ranks: VertexRDD[Double] = graph.vertices.mapValues((vid, attr) => resetProb).cache() + + // Initialize the delta graph where each vertex stores its delta and each edge knows its weight + var deltaGraph: Graph[Double, Double] = + graph.outerJoinVertices(graph.outDegrees)((vid, vdata, deg) => deg.getOrElse(0)) + .mapTriplets(e => 1.0 / e.srcAttr) + .mapVertices((vid, degree) => resetProb).cache() + var numDeltas: Long = ranks.count() + + var prevDeltas: Option[VertexRDD[Double]] = None + + var i = 0 + val weight = (1.0 - resetProb) + while (numDeltas > 0) { + // Compute new deltas. Only deltas that existed in the last round (i.e., were greater than + // `tol`) get to send messages; those that were less than `tol` would send messages less than + // `tol` as well. + val deltas = deltaGraph + .mapReduceTriplets[Double]( + et => Iterator((et.dstId, et.srcAttr * et.attr * weight)), + _ + _, + prevDeltas.map((_, EdgeDirection.Out))) + .filter { case (vid, delta) => delta > tol } + .cache() + prevDeltas = Some(deltas) + numDeltas = deltas.count() + logInfo("Standalone PageRank: iter %d has %d deltas".format(i, numDeltas)) + + // Update deltaGraph with the deltas + deltaGraph = deltaGraph.outerJoinVertices(deltas) { (vid, old, newOpt) => + newOpt.getOrElse(old) + }.cache() + + // Update ranks + ranks = ranks.leftZipJoin(deltas) { (vid, oldRank, deltaOpt) => + oldRank + deltaOpt.getOrElse(0.0) + } + ranks.foreach(x => {}) // force the iteration for ease of debugging + + i += 1 + } + + ranks + } + +} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/SVDPlusPlus.scala b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/SVDPlusPlus.scala new file mode 100644 index 0000000000..8fdfa3d907 --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/SVDPlusPlus.scala @@ -0,0 +1,103 @@ +package org.apache.spark.graphx.algorithms + +import org.apache.spark.rdd._ +import org.apache.spark.graphx._ +import scala.util.Random +import org.apache.commons.math.linear._ + +class SVDPlusPlusConf( // SVDPlusPlus parameters + var rank: Int, + var maxIters: Int, + var minVal: Double, + var maxVal: Double, + var gamma1: Double, + var gamma2: Double, + var gamma6: Double, + var gamma7: Double) extends Serializable + +object SVDPlusPlus { + /** + * Implement SVD++ based on "Factorization Meets the Neighborhood: a Multifaceted Collaborative Filtering Model", + * paper is available at [[http://public.research.att.com/~volinsky/netflix/kdd08koren.pdf]]. + * The prediction rule is rui = u + bu + bi + qi*(pu + |N(u)|^(-0.5)*sum(y)), see the details on page 6. + * + * @param edges edges for constructing the graph + * + * @param conf SVDPlusPlus parameters + * + * @return a graph with vertex attributes containing the trained model + */ + + def run(edges: RDD[Edge[Double]], conf: SVDPlusPlusConf): (Graph[(RealVector, RealVector, Double, Double), Double], Double) = { + + // generate default vertex attribute + def defaultF(rank: Int): (RealVector, RealVector, Double, Double) = { + val v1 = new ArrayRealVector(rank) + val v2 = new ArrayRealVector(rank) + for (i <- 0 until rank) { + v1.setEntry(i, Random.nextDouble) + v2.setEntry(i, Random.nextDouble) + } + (v1, v2, 0.0, 0.0) + } + + // calculate global rating mean + val (rs, rc) = edges.map(e => (e.attr, 1L)).reduce((a, b) => (a._1 + b._1, a._2 + b._2)) + val u = rs / rc + + // construct graph + var g = Graph.fromEdges(edges, defaultF(conf.rank)).cache() + + // calculate initial bias and norm + var t0 = g.mapReduceTriplets(et => + Iterator((et.srcId, (1L, et.attr)), (et.dstId, (1L, et.attr))), (g1: (Long, Double), g2: (Long, Double)) => (g1._1 + g2._1, g1._2 + g2._2)) + g = g.outerJoinVertices(t0) { (vid: VertexID, vd: (RealVector, RealVector, Double, Double), msg: Option[(Long, Double)]) => + (vd._1, vd._2, msg.get._2 / msg.get._1, 1.0 / scala.math.sqrt(msg.get._1)) + } + + def mapTrainF(conf: SVDPlusPlusConf, u: Double)(et: EdgeTriplet[(RealVector, RealVector, Double, Double), Double]) + : Iterator[(VertexID, (RealVector, RealVector, Double))] = { + val (usr, itm) = (et.srcAttr, et.dstAttr) + val (p, q) = (usr._1, itm._1) + var pred = u + usr._3 + itm._3 + q.dotProduct(usr._2) + pred = math.max(pred, conf.minVal) + pred = math.min(pred, conf.maxVal) + val err = et.attr - pred + val updateP = ((q.mapMultiply(err)).subtract(p.mapMultiply(conf.gamma7))).mapMultiply(conf.gamma2) + val updateQ = ((usr._2.mapMultiply(err)).subtract(q.mapMultiply(conf.gamma7))).mapMultiply(conf.gamma2) + val updateY = ((q.mapMultiply(err * usr._4)).subtract((itm._2).mapMultiply(conf.gamma7))).mapMultiply(conf.gamma2) + Iterator((et.srcId, (updateP, updateY, (err - conf.gamma6 * usr._3) * conf.gamma1)), + (et.dstId, (updateQ, updateY, (err - conf.gamma6 * itm._3) * conf.gamma1))) + } + + for (i <- 0 until conf.maxIters) { + // phase 1, calculate pu + |N(u)|^(-0.5)*sum(y) for user nodes + var t1 = g.mapReduceTriplets(et => Iterator((et.srcId, et.dstAttr._2)), (g1: RealVector, g2: RealVector) => g1.add(g2)) + g = g.outerJoinVertices(t1) { (vid: VertexID, vd: (RealVector, RealVector, Double, Double), msg: Option[RealVector]) => + if (msg.isDefined) (vd._1, vd._1.add(msg.get.mapMultiply(vd._4)), vd._3, vd._4) else vd + } + // phase 2, update p for user nodes and q, y for item nodes + val t2 = g.mapReduceTriplets(mapTrainF(conf, u), (g1: (RealVector, RealVector, Double), g2: (RealVector, RealVector, Double)) => + (g1._1.add(g2._1), g1._2.add(g2._2), g1._3 + g2._3)) + g = g.outerJoinVertices(t2) { (vid: VertexID, vd: (RealVector, RealVector, Double, Double), msg: Option[(RealVector, RealVector, Double)]) => + (vd._1.add(msg.get._1), vd._2.add(msg.get._2), vd._3 + msg.get._3, vd._4) + } + } + + // calculate error on training set + def mapTestF(conf: SVDPlusPlusConf, u: Double)(et: EdgeTriplet[(RealVector, RealVector, Double, Double), Double]): Iterator[(VertexID, Double)] = { + val (usr, itm) = (et.srcAttr, et.dstAttr) + val (p, q) = (usr._1, itm._1) + var pred = u + usr._3 + itm._3 + q.dotProduct(usr._2) + pred = math.max(pred, conf.minVal) + pred = math.min(pred, conf.maxVal) + val err = (et.attr - pred) * (et.attr - pred) + Iterator((et.dstId, err)) + } + val t3 = g.mapReduceTriplets(mapTestF(conf, u), (g1: Double, g2: Double) => g1 + g2) + g = g.outerJoinVertices(t3) { (vid: VertexID, vd: (RealVector, RealVector, Double, Double), msg: Option[Double]) => + if (msg.isDefined) (vd._1, vd._2, vd._3, msg.get) else vd + } + (g, u) + } +} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/StronglyConnectedComponents.scala b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/StronglyConnectedComponents.scala new file mode 100644 index 0000000000..f64fc3ef0f --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/StronglyConnectedComponents.scala @@ -0,0 +1,87 @@ +package org.apache.spark.graphx.algorithms + +import org.apache.spark.graphx._ + +object StronglyConnectedComponents { + + /** + * Compute the strongly connected component (SCC) of each vertex and return an RDD with the vertex + * value containing the lowest vertex id in the SCC containing that vertex. + * + * @tparam VD the vertex attribute type (discarded in the computation) + * @tparam ED the edge attribute type (preserved in the computation) + * + * @param graph the graph for which to compute the SCC + * + * @return a graph with vertex attributes containing the smallest vertex id in each SCC + */ + def run[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], numIter: Int): Graph[VertexID, ED] = { + + // the graph we update with final SCC ids, and the graph we return at the end + var sccGraph = graph.mapVertices { case (vid, _) => vid } + // graph we are going to work with in our iterations + var sccWorkGraph = graph.mapVertices { case (vid, _) => (vid, false) } + + var numVertices = sccWorkGraph.numVertices + var iter = 0 + while (sccWorkGraph.numVertices > 0 && iter < numIter) { + iter += 1 + do { + numVertices = sccWorkGraph.numVertices + sccWorkGraph = sccWorkGraph.outerJoinVertices(sccWorkGraph.outDegrees) { + (vid, data, degreeOpt) => if (degreeOpt.isDefined) data else (vid, true) + } + sccWorkGraph = sccWorkGraph.outerJoinVertices(sccWorkGraph.inDegrees) { + (vid, data, degreeOpt) => if (degreeOpt.isDefined) data else (vid, true) + } + + // get all vertices to be removed + val finalVertices = sccWorkGraph.vertices + .filter { case (vid, (scc, isFinal)) => isFinal} + .mapValues { (vid, data) => data._1} + + // write values to sccGraph + sccGraph = sccGraph.outerJoinVertices(finalVertices) { + (vid, scc, opt) => opt.getOrElse(scc) + } + // only keep vertices that are not final + sccWorkGraph = sccWorkGraph.subgraph(vpred = (vid, data) => !data._2) + } while (sccWorkGraph.numVertices < numVertices) + + sccWorkGraph = sccWorkGraph.mapVertices{ case (vid, (color, isFinal)) => (vid, isFinal) } + + // collect min of all my neighbor's scc values, update if it's smaller than mine + // then notify any neighbors with scc values larger than mine + sccWorkGraph = GraphLab[(VertexID, Boolean), ED, VertexID](sccWorkGraph, Integer.MAX_VALUE)( + (vid, e) => e.otherVertexAttr(vid)._1, + (vid1, vid2) => math.min(vid1, vid2), + (vid, scc, optScc) => + (math.min(scc._1, optScc.getOrElse(scc._1)), scc._2), + (vid, e) => e.vertexAttr(vid)._1 < e.otherVertexAttr(vid)._1 + ) + + // start at root of SCCs. Traverse values in reverse, notify all my neighbors + // do not propagate if colors do not match! + sccWorkGraph = GraphLab[(VertexID, Boolean), ED, Boolean]( + sccWorkGraph, + Integer.MAX_VALUE, + EdgeDirection.Out, + EdgeDirection.In + )( + // vertex is final if it is the root of a color + // or it has the same color as a neighbor that is final + (vid, e) => (vid == e.vertexAttr(vid)._1) || (e.vertexAttr(vid)._1 == e.otherVertexAttr(vid)._1), + (final1, final2) => final1 || final2, + (vid, scc, optFinal) => + (scc._1, scc._2 || optFinal.getOrElse(false)), + // activate neighbor if they are not final, you are, and you have the same color + (vid, e) => e.vertexAttr(vid)._2 && + !e.otherVertexAttr(vid)._2 && (e.vertexAttr(vid)._1 == e.otherVertexAttr(vid)._1), + // start at root of colors + (vid, data) => vid == data._1 + ) + } + sccGraph + } + +} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/TriangleCount.scala b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/TriangleCount.scala new file mode 100644 index 0000000000..b5a93c1bd1 --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/TriangleCount.scala @@ -0,0 +1,78 @@ +package org.apache.spark.graphx.algorithms + +import scala.reflect.ClassTag + +import org.apache.spark.graphx._ + + +object TriangleCount { + /** + * Compute the number of triangles passing through each vertex. + * + * The algorithm is relatively straightforward and can be computed in three steps: + * + * 1) Compute the set of neighbors for each vertex + * 2) For each edge compute the intersection of the sets and send the + * count to both vertices. + * 3) Compute the sum at each vertex and divide by two since each + * triangle is counted twice. + * + * + * @param graph a graph with `sourceId` less than `destId`. The graph must have been partitioned + * using Graph.partitionBy. + * + * @return + */ + def run[VD: ClassTag, ED: ClassTag](graph: Graph[VD,ED]): Graph[Int, ED] = { + // Remove redundant edges + val g = graph.groupEdges((a, b) => a).cache + + // Construct set representations of the neighborhoods + val nbrSets: VertexRDD[VertexSet] = + g.collectNeighborIds(EdgeDirection.Both).mapValues { (vid, nbrs) => + val set = new VertexSet(4) + var i = 0 + while (i < nbrs.size) { + // prevent self cycle + if(nbrs(i) != vid) { + set.add(nbrs(i)) + } + i += 1 + } + set + } + // join the sets with the graph + val setGraph: Graph[VertexSet, ED] = g.outerJoinVertices(nbrSets) { + (vid, _, optSet) => optSet.getOrElse(null) + } + // Edge function computes intersection of smaller vertex with larger vertex + def edgeFunc(et: EdgeTriplet[VertexSet, ED]): Iterator[(VertexID, Int)] = { + assert(et.srcAttr != null) + assert(et.dstAttr != null) + val (smallSet, largeSet) = if (et.srcAttr.size < et.dstAttr.size) { + (et.srcAttr, et.dstAttr) + } else { + (et.dstAttr, et.srcAttr) + } + val iter = smallSet.iterator + var counter: Int = 0 + while (iter.hasNext) { + val vid = iter.next + if (vid != et.srcId && vid != et.dstId && largeSet.contains(vid)) { counter += 1 } + } + Iterator((et.srcId, counter), (et.dstId, counter)) + } + // compute the intersection along edges + val counters: VertexRDD[Int] = setGraph.mapReduceTriplets(edgeFunc, _ + _) + // Merge counters with the graph and divide by two since each triangle is counted twice + g.outerJoinVertices(counters) { + (vid, _, optCounter: Option[Int]) => + val dblCount = optCounter.getOrElse(0) + // double count should be even (divisible by two) + assert((dblCount & 1) == 0) + dblCount / 2 + } + + } // end of TriangleCount + +} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala new file mode 100644 index 0000000000..4176563d22 --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala @@ -0,0 +1,220 @@ +package org.apache.spark.graphx.impl + +import scala.reflect.ClassTag + +import org.apache.spark.graphx._ +import org.apache.spark.util.collection.PrimitiveKeyOpenHashMap + +/** + * A collection of edges stored in 3 large columnar arrays (src, dst, attribute). The arrays are + * clustered by src. + * + * @param srcIds the source vertex id of each edge + * @param dstIds the destination vertex id of each edge + * @param data the attribute associated with each edge + * @param index a clustered index on source vertex id + * @tparam ED the edge attribute type. + */ +class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED: ClassTag]( + val srcIds: Array[VertexID], + val dstIds: Array[VertexID], + val data: Array[ED], + val index: PrimitiveKeyOpenHashMap[VertexID, Int]) extends Serializable { + + /** + * 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) + for (e <- iterator) { + builder.add(e.dstId, e.srcId, e.attr) + } + builder.toEdgePartition + } + + /** + * Construct a new edge partition by applying the function f to all + * edges in this partition. + * + * @param f a function from an edge to a new attribute + * @tparam ED2 the type of the new attribute + * @return a new edge partition with the result of the function `f` + * applied to each edge + */ + def map[ED2: ClassTag](f: Edge[ED] => ED2): EdgePartition[ED2] = { + val newData = new Array[ED2](data.size) + val edge = new Edge[ED]() + val size = data.size + var i = 0 + while (i < size) { + edge.srcId = srcIds(i) + edge.dstId = dstIds(i) + edge.attr = data(i) + newData(i) = f(edge) + i += 1 + } + new EdgePartition(srcIds, dstIds, newData, index) + } + + /** + * Construct a new edge partition by using the edge attributes + * contained in the iterator. + * + * @note The input iterator should return edge attributes in the + * order of the edges returned by `EdgePartition.iterator` and + * should return attributes equal to the number of edges. + * + * @param f a function from an edge to a new attribute + * @tparam ED2 the type of the new attribute + * @return a new edge partition with the result of the function `f` + * applied to each edge + */ + def map[ED2: ClassTag](iter: Iterator[ED2]): EdgePartition[ED2] = { + val newData = new Array[ED2](data.size) + var i = 0 + while (iter.hasNext) { + newData(i) = iter.next() + i += 1 + } + assert(newData.size == i) + new EdgePartition(srcIds, dstIds, newData, index) + } + + /** + * Apply the function f to all edges in this partition. + * + * @param f an external state mutating user defined function. + */ + def foreach(f: Edge[ED] => Unit) { + iterator.foreach(f) + } + + /** + * Merge all the edges with the same src and dest id into a single + * edge using the `merge` function + * + * @param merge a commutative associative merge operation + * @return a new edge partition without duplicate edges + */ + def groupEdges(merge: (ED, ED) => ED): EdgePartition[ED] = { + val builder = new EdgePartitionBuilder[ED] + var firstIter: Boolean = true + var currSrcId: VertexID = nullValue[VertexID] + var currDstId: VertexID = nullValue[VertexID] + var currAttr: ED = nullValue[ED] + var i = 0 + while (i < size) { + if (i > 0 && currSrcId == srcIds(i) && currDstId == dstIds(i)) { + currAttr = merge(currAttr, data(i)) + } else { + if (i > 0) { + builder.add(currSrcId, currDstId, currAttr) + } + currSrcId = srcIds(i) + currDstId = dstIds(i) + currAttr = data(i) + } + i += 1 + } + if (size > 0) { + builder.add(currSrcId, currDstId, currAttr) + } + builder.toEdgePartition + } + + /** + * Apply `f` to all edges present in both `this` and `other` and return a new EdgePartition + * containing the resulting edges. + * + * If there are multiple edges with the same src and dst in `this`, `f` will be invoked once for + * each edge, but each time it may be invoked on any corresponding edge in `other`. + * + * If there are multiple edges with the same src and dst in `other`, `f` will only be invoked + * once. + */ + def innerJoin[ED2: ClassTag, ED3: ClassTag] + (other: EdgePartition[ED2]) + (f: (VertexID, VertexID, ED, ED2) => ED3): EdgePartition[ED3] = { + val builder = new EdgePartitionBuilder[ED3] + var i = 0 + var j = 0 + // For i = index of each edge in `this`... + while (i < size && j < other.size) { + val srcId = this.srcIds(i) + val dstId = this.dstIds(i) + // ... forward j to the index of the corresponding edge in `other`, and... + while (j < other.size && other.srcIds(j) < srcId) { j += 1 } + if (j < other.size && other.srcIds(j) == srcId) { + while (j < other.size && other.srcIds(j) == srcId && other.dstIds(j) < dstId) { j += 1 } + if (j < other.size && other.srcIds(j) == srcId && other.dstIds(j) == dstId) { + // ... run `f` on the matching edge + builder.add(srcId, dstId, f(srcId, dstId, this.data(i), other.data(j))) + } + } + i += 1 + } + builder.toEdgePartition + } + + /** + * The number of edges in this partition + * + * @return size of the partition + */ + def size: Int = srcIds.size + + /** The number of unique source vertices in the partition. */ + def indexSize: Int = index.size + + /** + * Get an iterator over the edges in this partition. + * + * @return an iterator over edges in the partition + */ + def iterator = new Iterator[Edge[ED]] { + private[this] val edge = new Edge[ED] + private[this] var pos = 0 + + override def hasNext: Boolean = pos < EdgePartition.this.size + + override def next(): Edge[ED] = { + edge.srcId = srcIds(pos) + edge.dstId = dstIds(pos) + edge.attr = data(pos) + pos += 1 + edge + } + } + + /** + * Get an iterator over the edges in this partition whose source vertex ids match srcIdPred. The + * iterator is generated using an index scan, so it is efficient at skipping edges that don't + * match srcIdPred. + */ + def indexIterator(srcIdPred: VertexID => Boolean): Iterator[Edge[ED]] = + index.iterator.filter(kv => srcIdPred(kv._1)).flatMap(Function.tupled(clusterIterator)) + + /** + * Get an iterator over the cluster of edges in this partition with source vertex id `srcId`. The + * cluster must start at position `index`. + */ + private def clusterIterator(srcId: VertexID, index: Int) = new Iterator[Edge[ED]] { + private[this] val edge = new Edge[ED] + private[this] var pos = index + + override def hasNext: Boolean = { + pos >= 0 && pos < EdgePartition.this.size && srcIds(pos) == srcId + } + + override def next(): Edge[ED] = { + assert(srcIds(pos) == srcId) + edge.srcId = srcIds(pos) + edge.dstId = dstIds(pos) + edge.attr = data(pos) + pos += 1 + edge + } + } +} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala new file mode 100644 index 0000000000..d4f08497a2 --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala @@ -0,0 +1,46 @@ +package org.apache.spark.graphx.impl + +import scala.reflect.ClassTag +import scala.util.Sorting + +import org.apache.spark.graphx._ +import org.apache.spark.util.collection.{PrimitiveKeyOpenHashMap, PrimitiveVector} + + +//private[graph] +class EdgePartitionBuilder[@specialized(Long, Int, Double) ED: ClassTag](size: Int = 64) { + + var edges = new PrimitiveVector[Edge[ED]](size) + + /** Add a new edge to the partition. */ + def add(src: VertexID, dst: VertexID, d: ED) { + edges += Edge(src, dst, d) + } + + def toEdgePartition: EdgePartition[ED] = { + val edgeArray = edges.trim().array + Sorting.quickSort(edgeArray)(Edge.lexicographicOrdering) + val srcIds = new Array[VertexID](edgeArray.size) + val dstIds = new Array[VertexID](edgeArray.size) + val data = new Array[ED](edgeArray.size) + val index = new PrimitiveKeyOpenHashMap[VertexID, Int] + // Copy edges into columnar structures, tracking the beginnings of source vertex id clusters and + // adding them to the index + if (edgeArray.length > 0) { + index.update(srcIds(0), 0) + var currSrcId: VertexID = srcIds(0) + var i = 0 + while (i < edgeArray.size) { + srcIds(i) = edgeArray(i).srcId + dstIds(i) = edgeArray(i).dstId + data(i) = edgeArray(i).attr + if (edgeArray(i).srcId != currSrcId) { + currSrcId = edgeArray(i).srcId + index.update(currSrcId, i) + } + i += 1 + } + } + new EdgePartition(srcIds, dstIds, data, index) + } +} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala new file mode 100644 index 0000000000..79fd962ffd --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala @@ -0,0 +1,43 @@ +package org.apache.spark.graphx.impl + +import scala.reflect.ClassTag + +import org.apache.spark.graphx._ +import org.apache.spark.util.collection.PrimitiveKeyOpenHashMap + + +/** + * The Iterator type returned when constructing edge triplets. This class technically could be + * an anonymous class in GraphImpl.triplets, but we name it here explicitly so it is easier to + * debug / profile. + */ +private[impl] +class EdgeTripletIterator[VD: ClassTag, ED: ClassTag]( + val vidToIndex: VertexIdToIndexMap, + val vertexArray: Array[VD], + val edgePartition: EdgePartition[ED]) + extends Iterator[EdgeTriplet[VD, ED]] { + + // Current position in the array. + private var pos = 0 + + // A triplet object that this iterator.next() call returns. We reuse this object to avoid + // allocating too many temporary Java objects. + private val triplet = new EdgeTriplet[VD, ED] + + private val vmap = new PrimitiveKeyOpenHashMap[VertexID, VD](vidToIndex, vertexArray) + + override def hasNext: Boolean = pos < edgePartition.size + + override def next() = { + triplet.srcId = edgePartition.srcIds(pos) + // assert(vmap.containsKey(e.src.id)) + triplet.srcAttr = vmap(triplet.srcId) + triplet.dstId = edgePartition.dstIds(pos) + // assert(vmap.containsKey(e.dst.id)) + triplet.dstAttr = vmap(triplet.dstId) + triplet.attr = edgePartition.data(pos) + pos += 1 + triplet + } +} 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 new file mode 100644 index 0000000000..be9f188150 --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala @@ -0,0 +1,422 @@ +package org.apache.spark.graphx.impl + +import scala.reflect.{classTag, ClassTag} + +import org.apache.spark.util.collection.PrimitiveVector +import org.apache.spark.{HashPartitioner, Partitioner} +import org.apache.spark.SparkContext._ +import org.apache.spark.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 RDD that supports computation on graphs. + * + * Graphs are represented using two classes of data: vertex-partitioned and + * edge-partitioned. `vertices` contains vertex attributes, which are vertex-partitioned. `edges` + * 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. + */ +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]) + extends Graph[VD, ED] with Serializable { + + def this( + vertices: VertexRDD[VD], + edges: EdgeRDD[ED], + routingTable: RoutingTable) = { + this(vertices, edges, routingTable, new ReplicatedVertexView(vertices, edges, routingTable)) + } + + def this( + vertices: VertexRDD[VD], + edges: EdgeRDD[ED]) = { + this(vertices, edges, new RoutingTable(edges, vertices)) + } + + /** Return a RDD that brings edges together with their source and destination vertices. */ + @transient override val triplets: RDD[EdgeTriplet[VD, ED]] = { + val vdTag = classTag[VD] + 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) + } + } + + override def persist(newLevel: StorageLevel): Graph[VD, ED] = { + vertices.persist(newLevel) + edges.persist(newLevel) + this + } + + override def cache(): Graph[VD, ED] = persist(StorageLevel.MEMORY_ONLY) + + override def partitionBy(partitionStrategy: PartitionStrategy): Graph[VD, ED] = { + val numPartitions = edges.partitions.size + val edTag = classTag[ED] + val newEdges = new EdgeRDD(edges.map { e => + val part: PartitionID = partitionStrategy.getPartition(e.srcId, e.dstId, numPartitions) + + // Should we be using 3-tuple or an optimized class + new MessageToPartition(part, (e.srcId, e.dstId, e.attr)) + } + .partitionBy(new HashPartitioner(numPartitions)) + .mapPartitionsWithIndex( { (pid, iter) => + val builder = new EdgePartitionBuilder[ED]()(edTag) + iter.foreach { message => + val data = message.data + builder.add(data._1, data._2, data._3) + } + val edgePartition = builder.toEdgePartition + Iterator((pid, edgePartition)) + }, preservesPartitioning = true).cache()) + new GraphImpl(vertices, newEdges) + } + + override def statistics: Map[String, Any] = { + // Get the total number of vertices after replication, used to compute the replication ratio. + def numReplicatedVertices(vid2pids: RDD[Array[Array[VertexID]]]): Double = { + vid2pids.map(_.map(_.size).sum.toLong).reduce(_ + _).toDouble + } + + val numVertices = this.ops.numVertices + val numEdges = this.ops.numEdges + val replicationRatioBoth = numReplicatedVertices(routingTable.bothAttrs) / numVertices + val replicationRatioSrcOnly = numReplicatedVertices(routingTable.srcAttrOnly) / numVertices + val replicationRatioDstOnly = numReplicatedVertices(routingTable.dstAttrOnly) / numVertices + // One entry for each partition, indicate the total number of edges on that partition. + val loadArray = edges.partitionsRDD.map(_._2.size).collect().map(_.toDouble / numEdges) + val minLoad = loadArray.min + val maxLoad = loadArray.max + Map( + "Num Vertices" -> numVertices, + "Num Edges" -> numEdges, + "Replication (both)" -> replicationRatioBoth, + "Replication (src only)" -> replicationRatioSrcOnly, + "Replication (dest only)" -> replicationRatioDstOnly, + "Load Array" -> loadArray, + "Min Load" -> minLoad, + "Max Load" -> maxLoad) + } + + /** + * Display the lineage information for this graph. + */ + def printLineage() = { + def traverseLineage( + rdd: RDD[_], + indent: String = "", + visited: Map[Int, String] = Map.empty[Int, String]) { + if (visited.contains(rdd.id)) { + println(indent + visited(rdd.id)) + println(indent) + } else { + val locs = rdd.partitions.map( p => rdd.preferredLocations(p) ) + val cacheLevel = rdd.getStorageLevel + val name = rdd.id + val deps = rdd.dependencies + val partitioner = rdd.partitioner + val numparts = partitioner match { case Some(p) => p.numPartitions; case None => 0} + println(indent + name + ": " + cacheLevel.description + " (partitioner: " + partitioner + + ", " + numparts +")") + println(indent + " |---> Deps: " + deps.map(d => (d, d.rdd.id) ).toString) + println(indent + " |---> PrefLoc: " + locs.map(x=> x.toString).mkString(", ")) + deps.foreach(d => traverseLineage(d.rdd, indent + " | ", visited)) + } + } + println("edges ------------------------------------------") + traverseLineage(edges, " ") + var visited = Map(edges.id -> "edges") + println("\n\nvertices ------------------------------------------") + traverseLineage(vertices, " ", visited) + visited += (vertices.id -> "vertices") + println("\n\nroutingTable.bothAttrs -------------------------------") + traverseLineage(routingTable.bothAttrs, " ", visited) + visited += (routingTable.bothAttrs.id -> "routingTable.bothAttrs") + println("\n\ntriplets ----------------------------------------") + traverseLineage(triplets, " ", visited) + println(visited) + } // end of printLineage + + override def reverse: Graph[VD, ED] = { + val newETable = edges.mapEdgePartitions((pid, part) => part.reverse) + new GraphImpl(vertices, newETable, routingTable, replicatedVertexView) + } + + override def mapVertices[VD2: ClassTag](f: (VertexID, VD) => VD2): Graph[VD2, ED] = { + if (classTag[VD] equals classTag[VD2]) { + // The map preserves type, so we can use incremental replication + val newVerts = vertices.mapVertexPartitions(_.map(f)) + val 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) + } else { + // The map does not preserve type, so we must re-replicate all vertices + new GraphImpl(vertices.mapVertexPartitions(_.map(f)), edges, routingTable) + } + } + + 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) + } + + override def mapTriplets[ED2: ClassTag]( + f: (PartitionID, Iterator[EdgeTriplet[VD, ED]]) => Iterator[ED2]): Graph[VD, ED2] = { + // Use an explicit manifest in PrimitiveKeyOpenHashMap init so we don't pull in the implicit + // manifest from GraphImpl (which would require serializing GraphImpl). + val vdTag = classTag[VD] + 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) + } + + override def subgraph( + epred: EdgeTriplet[VD, ED] => Boolean = x => true, + vpred: (VertexID, VD) => Boolean = (a, b) => true): Graph[VD, ED] = { + // Filter the vertices, reusing the partitioner and the index from this graph + val newVerts = vertices.mapVertexPartitions(_.filter(vpred)) + + // 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 + + 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) + } + + 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) + } + + ////////////////////////////////////////////////////////////////////////////////////////////////// + // Lower level transformation methods + ////////////////////////////////////////////////////////////////////////////////////////////////// + + override def mapReduceTriplets[A: ClassTag]( + mapFunc: EdgeTriplet[VD, ED] => Iterator[(VertexID, A)], + reduceFunc: (A, A) => A, + activeSetOpt: Option[(VertexRDD[_], EdgeDirection)] = None) = { + + ClosureCleaner.clean(mapFunc) + ClosureCleaner.clean(reduceFunc) + + // For each vertex, replicate its attribute only to partitions where it is + // in the relevant position in an edge. + val mapUsesSrcAttr = accessesVertexAttr[VD, ED](mapFunc, "srcAttr") + val mapUsesDstAttr = accessesVertexAttr[VD, ED](mapFunc, "dstAttr") + val vs = activeSetOpt match { + case Some((activeSet, _)) => + replicatedVertexView.get(mapUsesSrcAttr, mapUsesDstAttr, activeSet) + case None => + replicatedVertexView.get(mapUsesSrcAttr, mapUsesDstAttr) + } + val activeDirectionOpt = activeSetOpt.map(_._2) + + // Map and combine. + val preAgg = edges.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.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) + } + mapFunc(et) + } + // Note: This doesn't allow users to send messages to arbitrary vertices. + vPart.aggregateUsingIndex(mapOutputs, reduceFunc).iterator + } + + // do the final reduction reusing the index map + vertices.aggregateUsingIndex(preAgg, reduceFunc) + } // end of mapReduceTriplets + + override def outerJoinVertices[U: ClassTag, VD2: ClassTag] + (updates: RDD[(VertexID, U)])(updateF: (VertexID, VD, Option[U]) => VD2): Graph[VD2, ED] = { + if (classTag[VD] equals classTag[VD2]) { + // updateF preserves type, so we can use incremental replication + val newVerts = vertices.leftJoin(updates)(updateF) + 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) + } else { + // updateF does not preserve type, so we must re-replicate all vertices + val newVerts = vertices.leftJoin(updates)(updateF) + new GraphImpl(newVerts, edges, routingTable) + } + } + + private def accessesVertexAttr[VD, ED](closure: AnyRef, attrName: String): Boolean = { + try { + BytecodeUtils.invokedMethod(closure, classOf[EdgeTriplet[VD, ED]], attrName) + } catch { + case _: ClassNotFoundException => true // if we don't know, be conservative + } + } +} // end of class GraphImpl + + +object GraphImpl { + + def apply[VD: ClassTag, ED: ClassTag]( + edges: RDD[Edge[ED]], + defaultVertexAttr: VD): GraphImpl[VD, ED] = + { + fromEdgeRDD(createEdgeRDD(edges), defaultVertexAttr) + } + + def fromEdgePartitions[VD: ClassTag, ED: ClassTag]( + edgePartitions: RDD[(PartitionID, EdgePartition[ED])], + defaultVertexAttr: VD): GraphImpl[VD, ED] = { + fromEdgeRDD(new EdgeRDD(edgePartitions), 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) + + new GraphImpl(vertexRDD, edgeRDD) + } + + /** + * 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. + */ + private def createEdgeRDD[ED: ClassTag]( + edges: RDD[Edge[ED]]): EdgeRDD[ED] = { + val edgePartitions = edges.mapPartitionsWithIndex { (pid, iter) => + val builder = new EdgePartitionBuilder[ED] + iter.foreach { e => + builder.add(e.srcId, e.dstId, e.attr) + } + Iterator((pid, builder.toEdgePartition)) + } + new EdgeRDD(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)) + new 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(classOf[VertexIDMsgSerializer].getName) + } +} // 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 new file mode 100644 index 0000000000..ad5daf8f6a --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala @@ -0,0 +1,93 @@ +package org.apache.spark.graphx.impl + +import scala.reflect.{classTag, ClassTag} + +import org.apache.spark.Partitioner +import org.apache.spark.graphx.{PartitionID, VertexID} +import org.apache.spark.rdd.{ShuffledRDD, RDD} + + +class VertexBroadcastMsg[@specialized(Int, Long, Double, Boolean) T]( + @transient var partition: PartitionID, + var vid: VertexID, + var data: T) + extends Product2[PartitionID, (VertexID, T)] with Serializable { + + override def _1 = partition + + override def _2 = (vid, data) + + override def canEqual(that: Any): Boolean = that.isInstanceOf[VertexBroadcastMsg[_]] +} + + +/** + * A message used to send a specific value to a partition. + * @param partition index of the target partition. + * @param data value to send + */ +class MessageToPartition[@specialized(Int, Long, Double, Char, Boolean/*, AnyRef*/) T]( + @transient var partition: PartitionID, + var data: T) + extends Product2[PartitionID, T] with Serializable { + + override def _1 = partition + + override def _2 = data + + override def canEqual(that: Any): Boolean = that.isInstanceOf[MessageToPartition[_]] +} + + +class VertexBroadcastMsgRDDFunctions[T: ClassTag](self: RDD[VertexBroadcastMsg[T]]) { + def partitionBy(partitioner: Partitioner): RDD[VertexBroadcastMsg[T]] = { + val rdd = new ShuffledRDD[PartitionID, (VertexID, T), VertexBroadcastMsg[T]](self, partitioner) + + // Set a custom serializer if the data is of int or double type. + if (classTag[T] == ClassTag.Int) { + rdd.setSerializer(classOf[IntVertexBroadcastMsgSerializer].getName) + } else if (classTag[T] == ClassTag.Long) { + rdd.setSerializer(classOf[LongVertexBroadcastMsgSerializer].getName) + } else if (classTag[T] == ClassTag.Double) { + rdd.setSerializer(classOf[DoubleVertexBroadcastMsgSerializer].getName) + } + rdd + } +} + + +class MsgRDDFunctions[T: ClassTag](self: RDD[MessageToPartition[T]]) { + + /** + * Return a copy of the RDD partitioned using the specified partitioner. + */ + def partitionBy(partitioner: Partitioner): RDD[MessageToPartition[T]] = { + new ShuffledRDD[PartitionID, T, MessageToPartition[T]](self, partitioner) + } + +} + + +object MsgRDDFunctions { + implicit def rdd2PartitionRDDFunctions[T: ClassTag](rdd: RDD[MessageToPartition[T]]) = { + new MsgRDDFunctions(rdd) + } + + 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) + + // Set a custom serializer if the data is of int or double type. + if (classTag[T] == ClassTag.Int) { + rdd.setSerializer(classOf[IntAggMsgSerializer].getName) + } else if (classTag[T] == ClassTag.Long) { + rdd.setSerializer(classOf[LongAggMsgSerializer].getName) + } else if (classTag[T] == ClassTag.Double) { + rdd.setSerializer(classOf[DoubleAggMsgSerializer].getName) + } + 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 new file mode 100644 index 0000000000..63180bc3af --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala @@ -0,0 +1,182 @@ +package org.apache.spark.graphx.impl + +import scala.reflect.{classTag, ClassTag} + +import org.apache.spark.SparkContext._ +import org.apache.spark.rdd.RDD +import org.apache.spark.util.collection.{PrimitiveVector, OpenHashSet} + +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 created, it remains materialized). This avoids + * constructing it twice if the user calls graph.triplets followed by graph.mapReduceTriplets, for + * example. + */ +private[impl] +class ReplicatedVertexView[VD: ClassTag]( + updatedVerts: VertexRDD[VD], + edges: EdgeRDD[_], + routingTable: RoutingTable, + prevViewOpt: Option[ReplicatedVertexView[VD]] = None) { + + /** + * Within each edge partition, create a local map from vid to an index into the attribute + * array. Each map contains a superset of the vertices that it will receive, because it stores + * vids from both the source and destination of edges. It must always include both source and + * destination vids because some operations, such as GraphImpl.mapReduceTriplets, rely on this. + */ + 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 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 + } + } + + 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)) + } + } + + 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)) + .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)) + } + } +} + +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 + } + (pid, new VertexAttributeBlock(vids.trim().array, attrs.trim().array)) + } + } + + 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() + + 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 + } + (pid, actives.trim().array) + } + } +} + +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 new file mode 100644 index 0000000000..3bd8b24133 --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTable.scala @@ -0,0 +1,64 @@ +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. + */ +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 numPartitions = vertices.partitions.size + vid2pid.partitionBy(vertices.partitioner.get).mapPartitions { iter => + val pid2vid = Array.fill(numPartitions)(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/Serializers.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala new file mode 100644 index 0000000000..1c3c87f08d --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala @@ -0,0 +1,386 @@ +package org.apache.spark.graphx.impl + +import java.io.{EOFException, InputStream, OutputStream} +import java.nio.ByteBuffer + +import org.apache.spark.SparkConf +import org.apache.spark.graphx._ +import org.apache.spark.serializer._ + +class VertexIDMsgSerializer(conf: SparkConf) extends Serializer { + override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { + + override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { + def writeObject[T](t: T) = { + val msg = t.asInstanceOf[(VertexID, _)] + writeVarLong(msg._1, optimizePositive = false) + this + } + } + + override def deserializeStream(s: InputStream) = new ShuffleDeserializationStream(s) { + override def readObject[T](): T = { + (readVarLong(optimizePositive = false), null).asInstanceOf[T] + } + } + } +} + +/** A special shuffle serializer for VertexBroadcastMessage[Int]. */ +class IntVertexBroadcastMsgSerializer(conf: SparkConf) extends Serializer { + override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { + + override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { + def writeObject[T](t: T) = { + val msg = t.asInstanceOf[VertexBroadcastMsg[Int]] + writeVarLong(msg.vid, optimizePositive = false) + writeInt(msg.data) + this + } + } + + override def deserializeStream(s: InputStream) = new ShuffleDeserializationStream(s) { + override def readObject[T](): T = { + val a = readVarLong(optimizePositive = false) + val b = readInt() + new VertexBroadcastMsg[Int](0, a, b).asInstanceOf[T] + } + } + } +} + +/** A special shuffle serializer for VertexBroadcastMessage[Long]. */ +class LongVertexBroadcastMsgSerializer(conf: SparkConf) extends Serializer { + override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { + + override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { + def writeObject[T](t: T) = { + val msg = t.asInstanceOf[VertexBroadcastMsg[Long]] + writeVarLong(msg.vid, optimizePositive = false) + writeLong(msg.data) + this + } + } + + override def deserializeStream(s: InputStream) = new ShuffleDeserializationStream(s) { + override def readObject[T](): T = { + val a = readVarLong(optimizePositive = false) + val b = readLong() + new VertexBroadcastMsg[Long](0, a, b).asInstanceOf[T] + } + } + } +} + +/** A special shuffle serializer for VertexBroadcastMessage[Double]. */ +class DoubleVertexBroadcastMsgSerializer(conf: SparkConf) extends Serializer { + override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { + + override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { + def writeObject[T](t: T) = { + val msg = t.asInstanceOf[VertexBroadcastMsg[Double]] + writeVarLong(msg.vid, optimizePositive = false) + writeDouble(msg.data) + this + } + } + + override def deserializeStream(s: InputStream) = new ShuffleDeserializationStream(s) { + def readObject[T](): T = { + val a = readVarLong(optimizePositive = false) + val b = readDouble() + new VertexBroadcastMsg[Double](0, a, b).asInstanceOf[T] + } + } + } +} + +/** A special shuffle serializer for AggregationMessage[Int]. */ +class IntAggMsgSerializer(conf: SparkConf) extends Serializer { + override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { + + override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { + def writeObject[T](t: T) = { + val msg = t.asInstanceOf[(VertexID, Int)] + writeVarLong(msg._1, optimizePositive = false) + writeUnsignedVarInt(msg._2) + this + } + } + + override def deserializeStream(s: InputStream) = new ShuffleDeserializationStream(s) { + override def readObject[T](): T = { + val a = readVarLong(optimizePositive = false) + val b = readUnsignedVarInt() + (a, b).asInstanceOf[T] + } + } + } +} + +/** A special shuffle serializer for AggregationMessage[Long]. */ +class LongAggMsgSerializer(conf: SparkConf) extends Serializer { + override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { + + override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { + def writeObject[T](t: T) = { + val msg = t.asInstanceOf[(VertexID, Long)] + writeVarLong(msg._1, optimizePositive = false) + writeVarLong(msg._2, optimizePositive = true) + this + } + } + + override def deserializeStream(s: InputStream) = new ShuffleDeserializationStream(s) { + override def readObject[T](): T = { + val a = readVarLong(optimizePositive = false) + val b = readVarLong(optimizePositive = true) + (a, b).asInstanceOf[T] + } + } + } +} + +/** A special shuffle serializer for AggregationMessage[Double]. */ +class DoubleAggMsgSerializer(conf: SparkConf) extends Serializer { + override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { + + override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { + def writeObject[T](t: T) = { + val msg = t.asInstanceOf[(VertexID, Double)] + writeVarLong(msg._1, optimizePositive = false) + writeDouble(msg._2) + this + } + } + + override def deserializeStream(s: InputStream) = new ShuffleDeserializationStream(s) { + def readObject[T](): T = { + val a = readVarLong(optimizePositive = false) + val b = readDouble() + (a, b).asInstanceOf[T] + } + } + } +} + +//////////////////////////////////////////////////////////////////////////////// +// Helper classes to shorten the implementation of those special serializers. +//////////////////////////////////////////////////////////////////////////////// + +abstract class ShuffleSerializationStream(s: OutputStream) extends SerializationStream { + // The implementation should override this one. + def writeObject[T](t: T): SerializationStream + + def writeInt(v: Int) { + s.write(v >> 24) + s.write(v >> 16) + s.write(v >> 8) + s.write(v) + } + + def writeUnsignedVarInt(value: Int) { + if ((value >>> 7) == 0) { + s.write(value.toInt) + } else if ((value >>> 14) == 0) { + s.write((value & 0x7F) | 0x80) + s.write(value >>> 7) + } else if ((value >>> 21) == 0) { + s.write((value & 0x7F) | 0x80) + s.write(value >>> 7 | 0x80) + s.write(value >>> 14) + } else if ((value >>> 28) == 0) { + s.write((value & 0x7F) | 0x80) + s.write(value >>> 7 | 0x80) + s.write(value >>> 14 | 0x80) + s.write(value >>> 21) + } else { + s.write((value & 0x7F) | 0x80) + s.write(value >>> 7 | 0x80) + s.write(value >>> 14 | 0x80) + s.write(value >>> 21 | 0x80) + s.write(value >>> 28) + } + } + + def writeVarLong(value: Long, optimizePositive: Boolean) { + val v = if (!optimizePositive) (value << 1) ^ (value >> 63) else value + if ((v >>> 7) == 0) { + s.write(v.toInt) + } else if ((v >>> 14) == 0) { + s.write(((v & 0x7F) | 0x80).toInt) + s.write((v >>> 7).toInt) + } else if ((v >>> 21) == 0) { + s.write(((v & 0x7F) | 0x80).toInt) + s.write((v >>> 7 | 0x80).toInt) + s.write((v >>> 14).toInt) + } else if ((v >>> 28) == 0) { + s.write(((v & 0x7F) | 0x80).toInt) + s.write((v >>> 7 | 0x80).toInt) + s.write((v >>> 14 | 0x80).toInt) + s.write((v >>> 21).toInt) + } else if ((v >>> 35) == 0) { + s.write(((v & 0x7F) | 0x80).toInt) + s.write((v >>> 7 | 0x80).toInt) + s.write((v >>> 14 | 0x80).toInt) + s.write((v >>> 21 | 0x80).toInt) + s.write((v >>> 28).toInt) + } else if ((v >>> 42) == 0) { + s.write(((v & 0x7F) | 0x80).toInt) + s.write((v >>> 7 | 0x80).toInt) + s.write((v >>> 14 | 0x80).toInt) + s.write((v >>> 21 | 0x80).toInt) + s.write((v >>> 28 | 0x80).toInt) + s.write((v >>> 35).toInt) + } else if ((v >>> 49) == 0) { + s.write(((v & 0x7F) | 0x80).toInt) + s.write((v >>> 7 | 0x80).toInt) + s.write((v >>> 14 | 0x80).toInt) + s.write((v >>> 21 | 0x80).toInt) + s.write((v >>> 28 | 0x80).toInt) + s.write((v >>> 35 | 0x80).toInt) + s.write((v >>> 42).toInt) + } else if ((v >>> 56) == 0) { + s.write(((v & 0x7F) | 0x80).toInt) + s.write((v >>> 7 | 0x80).toInt) + s.write((v >>> 14 | 0x80).toInt) + s.write((v >>> 21 | 0x80).toInt) + s.write((v >>> 28 | 0x80).toInt) + s.write((v >>> 35 | 0x80).toInt) + s.write((v >>> 42 | 0x80).toInt) + s.write((v >>> 49).toInt) + } else { + s.write(((v & 0x7F) | 0x80).toInt) + s.write((v >>> 7 | 0x80).toInt) + s.write((v >>> 14 | 0x80).toInt) + s.write((v >>> 21 | 0x80).toInt) + s.write((v >>> 28 | 0x80).toInt) + s.write((v >>> 35 | 0x80).toInt) + s.write((v >>> 42 | 0x80).toInt) + s.write((v >>> 49 | 0x80).toInt) + s.write((v >>> 56).toInt) + } + } + + def writeLong(v: Long) { + s.write((v >>> 56).toInt) + s.write((v >>> 48).toInt) + s.write((v >>> 40).toInt) + s.write((v >>> 32).toInt) + s.write((v >>> 24).toInt) + s.write((v >>> 16).toInt) + s.write((v >>> 8).toInt) + s.write(v.toInt) + } + + //def writeDouble(v: Double): Unit = writeUnsignedVarLong(java.lang.Double.doubleToLongBits(v)) + def writeDouble(v: Double): Unit = writeLong(java.lang.Double.doubleToLongBits(v)) + + override def flush(): Unit = s.flush() + + override def close(): Unit = s.close() +} + +abstract class ShuffleDeserializationStream(s: InputStream) extends DeserializationStream { + // The implementation should override this one. + def readObject[T](): T + + def readInt(): Int = { + val first = s.read() + if (first < 0) throw new EOFException + (first & 0xFF) << 24 | (s.read() & 0xFF) << 16 | (s.read() & 0xFF) << 8 | (s.read() & 0xFF) + } + + def readUnsignedVarInt(): Int = { + var value: Int = 0 + var i: Int = 0 + def readOrThrow(): Int = { + val in = s.read() + if (in < 0) throw new EOFException + in & 0xFF + } + var b: Int = readOrThrow() + while ((b & 0x80) != 0) { + value |= (b & 0x7F) << i + i += 7 + if (i > 35) throw new IllegalArgumentException("Variable length quantity is too long") + b = readOrThrow() + } + value | (b << i) + } + + def readVarLong(optimizePositive: Boolean): Long = { + def readOrThrow(): Int = { + val in = s.read() + if (in < 0) throw new EOFException + in & 0xFF + } + var b = readOrThrow() + var ret: Long = b & 0x7F + if ((b & 0x80) != 0) { + b = readOrThrow() + ret |= (b & 0x7F) << 7 + if ((b & 0x80) != 0) { + b = readOrThrow() + ret |= (b & 0x7F) << 14 + if ((b & 0x80) != 0) { + b = readOrThrow() + ret |= (b & 0x7F) << 21 + if ((b & 0x80) != 0) { + b = readOrThrow() + ret |= (b & 0x7F).toLong << 28 + if ((b & 0x80) != 0) { + b = readOrThrow() + ret |= (b & 0x7F).toLong << 35 + if ((b & 0x80) != 0) { + b = readOrThrow() + ret |= (b & 0x7F).toLong << 42 + if ((b & 0x80) != 0) { + b = readOrThrow() + ret |= (b & 0x7F).toLong << 49 + if ((b & 0x80) != 0) { + b = readOrThrow() + ret |= b.toLong << 56 + } + } + } + } + } + } + } + } + if (!optimizePositive) (ret >>> 1) ^ -(ret & 1) else ret + } + + def readLong(): Long = { + val first = s.read() + if (first < 0) throw new EOFException() + (first.toLong << 56) | + (s.read() & 0xFF).toLong << 48 | + (s.read() & 0xFF).toLong << 40 | + (s.read() & 0xFF).toLong << 32 | + (s.read() & 0xFF).toLong << 24 | + (s.read() & 0xFF) << 16 | + (s.read() & 0xFF) << 8 | + (s.read() & 0xFF) + } + + //def readDouble(): Double = java.lang.Double.longBitsToDouble(readUnsignedVarLong()) + def readDouble(): Double = java.lang.Double.longBitsToDouble(readLong()) + + override def close(): Unit = s.close() +} + +sealed trait ShuffleSerializerInstance extends SerializerInstance { + + override def serialize[T](t: T): ByteBuffer = throw new UnsupportedOperationException + + override def deserialize[T](bytes: ByteBuffer): T = throw new UnsupportedOperationException + + override def deserialize[T](bytes: ByteBuffer, loader: ClassLoader): T = + throw new UnsupportedOperationException + + // The implementation should override the following two. + override def serializeStream(s: OutputStream): SerializationStream + override def deserializeStream(s: InputStream): DeserializationStream +} 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 new file mode 100644 index 0000000000..7c83497ca9 --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala @@ -0,0 +1,262 @@ +package org.apache.spark.graphx.impl + +import scala.reflect.ClassTag + +import org.apache.spark.util.collection.{BitSet, PrimitiveKeyOpenHashMap} + +import org.apache.spark.Logging +import org.apache.spark.graphx._ + + +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) + } + + /** The number of active vertices, if any exist. */ + def numActives: Option[Int] = activeSet.map(_.size) + + /** + * Pass each vertex attribute along with the vertex id through a map + * function and retain the original RDD's partitioning and index. + * + * @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): 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) + } + + /** + * Hides vertices that are the same between this and other. For vertices that are different, keeps + * the values from `other`. The indices of `this` and `other` must be the same. + */ + def diff(other: VertexPartition[VD]): VertexPartition[VD] = { + 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) + } + + /** + * 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) + } + + /** + * 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) + } + + 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 replaceActives(iter: Iterator[VertexID]): VertexPartition[VD] = { + val newActiveSet = new VertexSet + iter.foreach(newActiveSet.add(_)) + new VertexPartition(index, values, mask, Some(newActiveSet)) + } + + /** + * Construct a new VertexPartition whose index contains only the vertices in the mask. + */ + 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 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/package.scala b/graphx/src/main/scala/org/apache/spark/graphx/package.scala new file mode 100644 index 0000000000..96f0d91c9b --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/package.scala @@ -0,0 +1,22 @@ +package org.apache.spark + +import org.apache.spark.util.collection.OpenHashSet + + +package object graphx { + + type VertexID = Long + + // TODO: Consider using Char. + type PartitionID = Int + + type VertexSet = OpenHashSet[VertexID] + + // type VertexIdToIndexMap = it.unimi.dsi.fastutil.longs.Long2IntOpenHashMap + type VertexIdToIndexMap = OpenHashSet[VertexID] + + /** + * Return the default null-like value for a data type T. + */ + def nullValue[T] = null.asInstanceOf[T] +} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/perf/BagelTest.scala b/graphx/src/main/scala/org/apache/spark/graphx/perf/BagelTest.scala new file mode 100644 index 0000000000..81332e0800 --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/perf/BagelTest.scala @@ -0,0 +1,76 @@ +///// This file creates circular dependencies between examples bagle and graph + +// package org.apache.spark.graphx.perf + +// import org.apache.spark._ +// import org.apache.spark.SparkContext._ +// import org.apache.spark.bagel.Bagel + +// import org.apache.spark.examples.bagel +// //import org.apache.spark.bagel.examples._ +// import org.apache.spark.graphx._ + + +// object BagelTest { + +// def main(args: Array[String]) { +// val host = args(0) +// val taskType = args(1) +// val fname = args(2) +// val options = args.drop(3).map { arg => +// arg.dropWhile(_ == '-').split('=') match { +// case Array(opt, v) => (opt -> v) +// case _ => throw new IllegalArgumentException("Invalid argument: " + arg) +// } +// } + +// System.setProperty("spark.serializer", "org.apache.spark.serializer.KryoSerializer") +// //System.setProperty("spark.shuffle.compress", "false") +// System.setProperty("spark.kryo.registrator", "org.apache.spark.bagel.examples.PRKryoRegistrator") + +// var numIter = Int.MaxValue +// var isDynamic = false +// var tol:Float = 0.001F +// var outFname = "" +// var numVPart = 4 +// var numEPart = 4 + +// options.foreach{ +// case ("numIter", v) => numIter = v.toInt +// case ("dynamic", v) => isDynamic = v.toBoolean +// case ("tol", v) => tol = v.toFloat +// case ("output", v) => outFname = v +// case ("numVPart", v) => numVPart = v.toInt +// case ("numEPart", v) => numEPart = v.toInt +// case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) +// } + +// val sc = new SparkContext(host, "PageRank(" + fname + ")") +// val g = GraphLoader.textFile(sc, fname, a => 1.0F).withPartitioner(numVPart, numEPart).cache() +// val startTime = System.currentTimeMillis + +// val numVertices = g.vertices.count() + +// val vertices = g.collectNeighborIds(EdgeDirection.Out).map { case (vid, neighbors) => +// (vid.toString, new PRVertex(1.0, neighbors.map(_.toString))) +// } + +// // Do the computation +// val epsilon = 0.01 / numVertices +// val messages = sc.parallelize(Array[(String, PRMessage)]()) +// val utils = new PageRankUtils +// val result = +// Bagel.run( +// sc, vertices, messages, combiner = new PRCombiner(), +// numPartitions = numVPart)( +// utils.computeWithCombiner(numVertices, epsilon, numIter)) + +// println("Total rank: " + result.map{ case (id, r) => r.value }.reduce(_+_) ) +// if (!outFname.isEmpty) { +// println("Saving pageranks of pages to " + outFname) +// result.map{ case (id, r) => id + "\t" + r.value }.saveAsTextFile(outFname) +// } +// println("Runtime: " + ((System.currentTimeMillis - startTime)/1000.0) + " seconds") +// sc.stop() +// } +// } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/perf/SparkTest.scala b/graphx/src/main/scala/org/apache/spark/graphx/perf/SparkTest.scala new file mode 100644 index 0000000000..24262640ab --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/perf/SparkTest.scala @@ -0,0 +1,75 @@ +///// This file creates circular dependencies between examples bagle and graph + + +// package org.apache.spark.graphx.perf + +// import org.apache.spark._ +// import org.apache.spark.SparkContext._ +// import org.apache.spark.bagel.Bagel +// import org.apache.spark.bagel.examples._ +// import org.apache.spark.graphx._ + + +// object SparkTest { + +// def main(args: Array[String]) { +// val host = args(0) +// val taskType = args(1) +// val fname = args(2) +// val options = args.drop(3).map { arg => +// arg.dropWhile(_ == '-').split('=') match { +// case Array(opt, v) => (opt -> v) +// case _ => throw new IllegalArgumentException("Invalid argument: " + arg) +// } +// } + +// System.setProperty("spark.serializer", "org.apache.spark.KryoSerializer") +// //System.setProperty("spark.shuffle.compress", "false") +// System.setProperty("spark.kryo.registrator", "spark.bagel.examples.PRKryoRegistrator") + +// var numIter = Int.MaxValue +// var isDynamic = false +// var tol:Float = 0.001F +// var outFname = "" +// var numVPart = 4 +// var numEPart = 4 + +// options.foreach{ +// case ("numIter", v) => numIter = v.toInt +// case ("dynamic", v) => isDynamic = v.toBoolean +// case ("tol", v) => tol = v.toFloat +// case ("output", v) => outFname = v +// case ("numVPart", v) => numVPart = v.toInt +// case ("numEPart", v) => numEPart = v.toInt +// case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) +// } + +// val sc = new SparkContext(host, "PageRank(" + fname + ")") +// val g = GraphLoader.textFile(sc, fname, a => 1.0F).withPartitioner(numVPart, numEPart).cache() +// val startTime = System.currentTimeMillis + +// val numVertices = g.vertices.count() + +// val vertices = g.collectNeighborIds(EdgeDirection.Out).map { case (vid, neighbors) => +// (vid.toString, new PRVertex(1.0, neighbors.map(_.toString))) +// } + +// // Do the computation +// val epsilon = 0.01 / numVertices +// val messages = sc.parallelize(Array[(String, PRMessage)]()) +// val utils = new PageRankUtils +// val result = +// Bagel.run( +// sc, vertices, messages, combiner = new PRCombiner(), +// numPartitions = numVPart)( +// utils.computeWithCombiner(numVertices, epsilon, numIter)) + +// println("Total rank: " + result.map{ case (id, r) => r.value }.reduce(_+_) ) +// if (!outFname.isEmpty) { +// println("Saving pageranks of pages to " + outFname) +// result.map{ case (id, r) => id + "\t" + r.value }.saveAsTextFile(outFname) +// } +// println("Runtime: " + ((System.currentTimeMillis - startTime)/1000.0) + " seconds") +// sc.stop() +// } +// } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala new file mode 100644 index 0000000000..ec8d534333 --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala @@ -0,0 +1,114 @@ +package org.apache.spark.graphx.util + +import java.io.{ByteArrayInputStream, ByteArrayOutputStream} + +import scala.collection.mutable.HashSet + +import org.apache.spark.util.Utils + +import org.objectweb.asm.{ClassReader, ClassVisitor, MethodVisitor} +import org.objectweb.asm.Opcodes._ + + + +private[spark] object BytecodeUtils { + + /** + * Test whether the given closure invokes the specified method in the specified class. + */ + def invokedMethod(closure: AnyRef, targetClass: Class[_], targetMethod: String): Boolean = { + if (_invokedMethod(closure.getClass, "apply", targetClass, targetMethod)) { + true + } else { + // look at closures enclosed in this closure + for (f <- closure.getClass.getDeclaredFields + if f.getType.getName.startsWith("scala.Function")) { + f.setAccessible(true) + if (invokedMethod(f.get(closure), targetClass, targetMethod)) { + return true + } + } + return false + } + } + + private def _invokedMethod(cls: Class[_], method: String, + targetClass: Class[_], targetMethod: String): Boolean = { + + val seen = new HashSet[(Class[_], String)] + var stack = List[(Class[_], String)]((cls, method)) + + while (stack.nonEmpty) { + val (c, m) = stack.head + stack = stack.tail + seen.add((c, m)) + val finder = new MethodInvocationFinder(c.getName, m) + getClassReader(c).accept(finder, 0) + for (classMethod <- finder.methodsInvoked) { + //println(classMethod) + if (classMethod._1 == targetClass && classMethod._2 == targetMethod) { + return true + } else if (!seen.contains(classMethod)) { + stack = classMethod :: stack + } + } + } + return false + } + + /** + * Get an ASM class reader for a given class from the JAR that loaded it. + */ + private def getClassReader(cls: Class[_]): ClassReader = { + // Copy data over, before delegating to ClassReader - else we can run out of open file handles. + val className = cls.getName.replaceFirst("^.*\\.", "") + ".class" + val resourceStream = cls.getResourceAsStream(className) + // todo: Fixme - continuing with earlier behavior ... + if (resourceStream == null) return new ClassReader(resourceStream) + + val baos = new ByteArrayOutputStream(128) + Utils.copyStream(resourceStream, baos, true) + new ClassReader(new ByteArrayInputStream(baos.toByteArray)) + } + + /** + * Given the class name, return whether we should look into the class or not. This is used to + * skip examing a large quantity of Java or Scala classes that we know for sure wouldn't access + * the closures. Note that the class name is expected in ASM style (i.e. use "/" instead of "."). + */ + private def skipClass(className: String): Boolean = { + val c = className + c.startsWith("java/") || c.startsWith("scala/") || c.startsWith("javax/") + } + + /** + * Find the set of methods invoked by the specified method in the specified class. + * For example, after running the visitor, + * MethodInvocationFinder("spark/graph/Foo", "test") + * its methodsInvoked variable will contain the set of methods invoked directly by + * Foo.test(). Interface invocations are not returned as part of the result set because we cannot + * determine the actual metod invoked by inspecting the bytecode. + */ + private class MethodInvocationFinder(className: String, methodName: String) + extends ClassVisitor(ASM4) { + + val methodsInvoked = new HashSet[(Class[_], String)] + + override def visitMethod(access: Int, name: String, desc: String, + sig: String, exceptions: Array[String]): MethodVisitor = { + if (name == methodName) { + new MethodVisitor(ASM4) { + override def visitMethodInsn(op: Int, owner: String, name: String, desc: String) { + if (op == INVOKEVIRTUAL || op == INVOKESPECIAL || op == INVOKESTATIC) { + if (!skipClass(owner)) { + methodsInvoked.add((Class.forName(owner.replace("/", ".")), name)) + } + } + } + } + } else { + null + } + } + } +} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala new file mode 100644 index 0000000000..57117241ad --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala @@ -0,0 +1,282 @@ +package org.apache.spark.graphx.util + +import scala.annotation.tailrec +import scala.math._ +import scala.reflect.ClassTag +import scala.util._ + +import org.apache.spark._ +import org.apache.spark.serializer._ +import org.apache.spark.rdd.RDD +import org.apache.spark.SparkContext +import org.apache.spark.SparkContext._ +import org.apache.spark.graphx._ +import org.apache.spark.graphx.Graph +import org.apache.spark.graphx.Edge +import org.apache.spark.graphx.impl.GraphImpl + +/** + * @todo cleanup and modularize code + */ +object GraphGenerators { + + val RMATa = 0.45 + val RMATb = 0.15 + val RMATc = 0.15 + val RMATd = 0.25 + + def main(args: Array[String]) { + + + val serializer = "org.apache.spark.serializer.KryoSerializer" + System.setProperty("spark.serializer", serializer) + //System.setProperty("spark.shuffle.compress", "false") + System.setProperty("spark.kryo.registrator", "spark.graphx.GraphKryoRegistrator") + val host = "local[4]" + val sc = new SparkContext(host, "Lognormal graph generator") + + val lnGraph = logNormalGraph(sc, 10000) + + val rmat = rmatGraph(sc, 1000, 3000) + + //for (v <- lnGraph.vertices) { + // println(v.id + ":\t" + v.data) + //} + + val times = 100000 + //val nums = (1 to times).flatMap { n => List(sampleLogNormal(4.0, 1.3, times)) }.toList + //val avg = nums.sum / nums.length + //val sumSquares = nums.foldLeft(0.0) {(total, next) => + // (total + math.pow((next - avg), 2)) } + //val stdev = math.sqrt(sumSquares/(nums.length - 1)) + + //println("avg: " + avg + "+-" + stdev) + + + //for (i <- 1 to 1000) { + // println(sampleLogNormal(4.0, 1.3, 1000)) + //} + + sc.stop() + + } + + + // Right now it just generates a bunch of edges where + // the edge data is the weight (default 1) + def logNormalGraph(sc: SparkContext, numVertices: Int): Graph[Int, Int] = { + // based on Pregel settings + val mu = 4 + val sigma = 1.3 + //val vertsAndEdges = (0 until numVertices).flatMap { src => { + + val vertices: RDD[(VertexID, Int)] = sc.parallelize(0 until numVertices).map{ + src => (src, sampleLogNormal(mu, sigma, numVertices)) + } + + val edges = vertices.flatMap{ + v => generateRandomEdges(v._1.toInt, v._2, numVertices) + } + + Graph(vertices, edges, 0) + //println("Vertices:") + //for (v <- vertices) { + // println(v.id) + //} + + //println("Edges") + //for (e <- edges) { + // println(e.src, e.dst, e.data) + //} + + } + + + def generateRandomEdges(src: Int, numEdges: Int, maxVertexID: Int): Array[Edge[Int]] = { + val rand = new Random() + var dsts: Set[Int] = Set() + while (dsts.size < numEdges) { + val nextDst = rand.nextInt(maxVertexID) + if (nextDst != src) { + dsts += nextDst + } + } + dsts.map {dst => Edge[Int](src, dst, 1) }.toArray + } + + + /** + * Randomly samples from a log normal distribution + * whose corresponding normal distribution has the + * the given mean and standard deviation. It uses + * the formula X = exp(m+s*Z) where m, s are the + * mean, standard deviation of the lognormal distribution + * and Z~N(0, 1). In this function, + * m = e^(mu+sigma^2/2) and + * s = sqrt[(e^(sigma^2) - 1)(e^(2*mu+sigma^2))]. + * + * @param mu the mean of the normal distribution + * @param sigma the standard deviation of the normal distribution + * @param macVal exclusive upper bound on the value of the sample + */ + def sampleLogNormal(mu: Double, sigma: Double, maxVal: Int): Int = { + val rand = new Random() + val m = math.exp(mu+(sigma*sigma)/2.0) + val s = math.sqrt((math.exp(sigma*sigma) - 1) * math.exp(2*mu + sigma*sigma)) + // Z ~ N(0, 1) + var X: Double = maxVal + + while (X >= maxVal) { + val Z = rand.nextGaussian() + //X = math.exp((m + s*Z)) + X = math.exp((mu + sigma*Z)) + } + math.round(X.toFloat) + } + + + + def rmatGraph(sc: SparkContext, requestedNumVertices: Int, numEdges: Int): Graph[Int, Int] = { + // let N = requestedNumVertices + // the number of vertices is 2^n where n=ceil(log2[N]) + // This ensures that the 4 quadrants are the same size at all recursion levels + val numVertices = math.round(math.pow(2.0, math.ceil(math.log(requestedNumVertices)/math.log(2.0)))).toInt + var edges: Set[Edge[Int]] = Set() + while (edges.size < numEdges) { + if (edges.size % 100 == 0) { + println(edges.size + " edges") + } + edges += addEdge(numVertices) + + } + val graph = outDegreeFromEdges(sc.parallelize(edges.toList)) + graph + + } + + def outDegreeFromEdges[ED: ClassTag](edges: RDD[Edge[ED]]): Graph[Int, ED] = { + + val vertices = edges.flatMap { edge => List((edge.srcId, 1)) } + .reduceByKey(_ + _) + .map{ case (vid, degree) => (vid, degree) } + Graph(vertices, edges, 0) + } + + /** + * @param numVertices Specifies the total number of vertices in the graph (used to get + * the dimensions of the adjacency matrix + */ + def addEdge(numVertices: Int): Edge[Int] = { + //val (src, dst) = chooseCell(numVertices/2.0, numVertices/2.0, numVertices/2.0) + val v = math.round(numVertices.toFloat/2.0).toInt + + val (src, dst) = chooseCell(v, v, v) + Edge[Int](src, dst, 1) + } + + + /** + * This method recursively subdivides the the adjacency matrix into quadrants + * until it picks a single cell. The naming conventions in this paper match + * those of the R-MAT paper. There are a power of 2 number of nodes in the graph. + * The adjacency matrix looks like: + * + * dst -> + * (x,y) *************** _ + * | | | | + * | a | b | | + * src | | | | + * | *************** | T + * \|/ | | | | + * | c | d | | + * | | | | + * *************** - + * + * where this represents the subquadrant of the adj matrix currently being + * subdivided. (x,y) represent the upper left hand corner of the subquadrant, + * and T represents the side length (guaranteed to be a power of 2). + * + * After choosing the next level subquadrant, we get the resulting sets + * of parameters: + * quad = a, x'=x, y'=y, T'=T/2 + * quad = b, x'=x+T/2, y'=y, T'=T/2 + * quad = c, x'=x, y'=y+T/2, T'=T/2 + * quad = d, x'=x+T/2, y'=y+T/2, T'=T/2 + * + * @param src is the + */ + @tailrec + def chooseCell(x: Int, y: Int, t: Int): (Int, Int) = { + if (t <= 1) + (x,y) + else { + val newT = math.round(t.toFloat/2.0).toInt + pickQuadrant(RMATa, RMATb, RMATc, RMATd) match { + case 0 => chooseCell(x, y, newT) + case 1 => chooseCell(x+newT, y, newT) + case 2 => chooseCell(x, y+newT, newT) + case 3 => chooseCell(x+newT, y+newT, newT) + } + } + } + + // TODO(crankshaw) turn result into an enum (or case class for pattern matching} + def pickQuadrant(a: Double, b: Double, c: Double, d: Double): Int = { + if (a+b+c+d != 1.0) { + throw new IllegalArgumentException("R-MAT probability parameters sum to " + (a+b+c+d) + ", should sum to 1.0") + } + val rand = new Random() + val result = rand.nextDouble() + result match { + case x if x < a => 0 // 0 corresponds to quadrant a + case x if (x >= a && x < a+b) => 1 // 1 corresponds to b + case x if (x >= a+b && x < a+b+c) => 2 // 2 corresponds to c + case _ => 3 // 3 corresponds to d + } + } + + + + /** + * Create `rows` by `cols` grid graph with each vertex connected to its + * row+1 and col+1 neighbors. Vertex ids are assigned in row major + * order. + * + * @param sc the spark context in which to construct the graph + * @param rows the number of rows + * @param cols the number of columns + * + * @return A graph containing vertices with the row and column ids + * as their attributes and edge values as 1.0. + */ + def gridGraph(sc: SparkContext, rows: Int, cols: Int): Graph[(Int,Int), Double] = { + // Convert row column address into vertex ids (row major order) + def sub2ind(r: Int, c: Int): VertexID = r * cols + c + + val vertices: RDD[(VertexID, (Int,Int))] = + sc.parallelize(0 until rows).flatMap( r => (0 until cols).map( c => (sub2ind(r,c), (r,c)) ) ) + val edges: RDD[Edge[Double]] = + vertices.flatMap{ case (vid, (r,c)) => + (if (r+1 < rows) { Seq( (sub2ind(r, c), sub2ind(r+1, c))) } else { Seq.empty }) ++ + (if (c+1 < cols) { Seq( (sub2ind(r, c), sub2ind(r, c+1))) } else { Seq.empty }) + }.map{ case (src, dst) => Edge(src, dst, 1.0) } + Graph(vertices, edges) + } // end of gridGraph + + /** + * Create a star graph with vertex 0 being the center. + * + * @param sc the spark context in which to construct the graph + * @param nverts the number of vertices in the star + * + * @return A star graph containing `nverts` vertices with vertex 0 + * being the center vertex. + */ + def starGraph(sc: SparkContext, nverts: Int): Graph[Int, Int] = { + val edges: RDD[(VertexID, VertexID)] = sc.parallelize(1 until nverts).map(vid => (vid, 0)) + Graph.fromEdgeTuples(edges, 1) + } // end of starGraph + + + +} // end of Graph Generators diff --git a/graphx/src/main/scala/org/apache/spark/graphx/util/HashUtils.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/HashUtils.scala new file mode 100644 index 0000000000..7a79d33350 --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/util/HashUtils.scala @@ -0,0 +1,21 @@ +package org.apache.spark.graphx.util + + +object HashUtils { + + /** + * Compute a 64-bit hash value for the given string. + * See http://stackoverflow.com/questions/1660501/what-is-a-good-64bit-hash-function-in-java-for-textual-strings + */ + def hash(str: String): Long = { + var h = 1125899906842597L + val len = str.length + var i = 0 + + while (i < len) { + h = 31 * h + str(i) + i += 1 + } + h + } +} diff --git a/graphx/src/test/resources/log4j.properties b/graphx/src/test/resources/log4j.properties new file mode 100644 index 0000000000..896936d8c4 --- /dev/null +++ b/graphx/src/test/resources/log4j.properties @@ -0,0 +1,28 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Set everything to be logged to the file core/target/unit-tests.log +log4j.rootCategory=INFO, file +log4j.appender.file=org.apache.log4j.FileAppender +log4j.appender.file.append=false +log4j.appender.file.file=graph/target/unit-tests.log +log4j.appender.file.layout=org.apache.log4j.PatternLayout +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n + +# Ignore messages below warning level from Jetty, because it's a bit verbose +log4j.logger.org.eclipse.jetty=WARN +org.eclipse.jetty.LEVEL=WARN diff --git a/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala new file mode 100644 index 0000000000..cc281fce99 --- /dev/null +++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala @@ -0,0 +1,92 @@ +package org.apache.spark.graphx + +import org.apache.spark.SparkContext +import org.apache.spark.graphx.Graph._ +import org.apache.spark.graphx.impl.EdgePartition +import org.apache.spark.rdd._ +import org.scalatest.FunSuite + +class GraphOpsSuite extends FunSuite with LocalSparkContext { + + test("aggregateNeighbors") { + withSpark { sc => + val n = 3 + val star = + Graph.fromEdgeTuples(sc.parallelize((1 to n).map(x => (0: VertexID, x: VertexID))), 1) + + val indegrees = star.aggregateNeighbors( + (vid, edge) => Some(1), + (a: Int, b: Int) => a + b, + EdgeDirection.In) + assert(indegrees.collect().toSet === (1 to n).map(x => (x, 1)).toSet) + + val outdegrees = star.aggregateNeighbors( + (vid, edge) => Some(1), + (a: Int, b: Int) => a + b, + EdgeDirection.Out) + assert(outdegrees.collect().toSet === Set((0, n))) + + val noVertexValues = star.aggregateNeighbors[Int]( + (vid: VertexID, edge: EdgeTriplet[Int, Int]) => None, + (a: Int, b: Int) => throw new Exception("reduceFunc called unexpectedly"), + EdgeDirection.In) + assert(noVertexValues.collect().toSet === Set.empty[(VertexID, Int)]) + } + } + + test("joinVertices") { + withSpark { sc => + val vertices = + sc.parallelize(Seq[(VertexID, String)]((1, "one"), (2, "two"), (3, "three")), 2) + val edges = sc.parallelize((Seq(Edge(1, 2, "onetwo")))) + val g: Graph[String, String] = Graph(vertices, edges) + + val tbl = sc.parallelize(Seq[(VertexID, Int)]((1, 10), (2, 20))) + val g1 = g.joinVertices(tbl) { (vid: VertexID, attr: String, u: Int) => attr + u } + + val v = g1.vertices.collect().toSet + assert(v === Set((1, "one10"), (2, "two20"), (3, "three"))) + } + } + + test("collectNeighborIds") { + withSpark { sc => + val chain = (0 until 100).map(x => (x, (x+1)%100) ) + val rawEdges = sc.parallelize(chain, 3).map { case (s,d) => (s.toLong, d.toLong) } + val graph = Graph.fromEdgeTuples(rawEdges, 1.0) + val nbrs = graph.collectNeighborIds(EdgeDirection.Both) + assert(nbrs.count === chain.size) + assert(graph.numVertices === nbrs.count) + nbrs.collect.foreach { case (vid, nbrs) => assert(nbrs.size === 2) } + nbrs.collect.foreach { case (vid, nbrs) => + val s = nbrs.toSet + assert(s.contains((vid + 1) % 100)) + assert(s.contains(if (vid > 0) vid - 1 else 99 )) + } + } + } + + test ("filter") { + withSpark { sc => + val n = 5 + val vertices = sc.parallelize((0 to n).map(x => (x:VertexID, x))) + val edges = sc.parallelize((1 to n).map(x => Edge(0, x, x))) + val graph: Graph[Int, Int] = Graph(vertices, edges) + val filteredGraph = graph.filter( + graph => { + val degrees: VertexRDD[Int] = graph.outDegrees + graph.outerJoinVertices(degrees) {(vid, data, deg) => deg.getOrElse(0)} + }, + vpred = (vid: VertexID, deg:Int) => deg > 0 + ) + + val v = filteredGraph.vertices.collect().toSet + assert(v === Set((0,0))) + + // the map is necessary because of object-reuse in the edge iterator + val e = filteredGraph.edges.map(e => Edge(e.srcId, e.dstId, e.attr)).collect().toSet + assert(e.isEmpty) + } + } + +} diff --git a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala new file mode 100644 index 0000000000..094fa722a0 --- /dev/null +++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala @@ -0,0 +1,272 @@ +package org.apache.spark.graphx + +import org.scalatest.FunSuite + +import org.apache.spark.SparkContext +import org.apache.spark.graphx.Graph._ +import org.apache.spark.rdd._ + +class GraphSuite extends FunSuite with LocalSparkContext { + + def starGraph(sc: SparkContext, n: Int): Graph[String, Int] = { + Graph.fromEdgeTuples(sc.parallelize((1 to n).map(x => (0: VertexID, x: VertexID)), 3), "v") + } + + test("Graph.fromEdgeTuples") { + withSpark { sc => + val ring = (0L to 100L).zip((1L to 99L) :+ 0L) + val doubleRing = ring ++ ring + val graph = Graph.fromEdgeTuples(sc.parallelize(doubleRing), 1) + assert(graph.edges.count() === doubleRing.size) + assert(graph.edges.collect.forall(e => e.attr == 1)) + + // uniqueEdges option should uniquify edges and store duplicate count in edge attributes + val uniqueGraph = Graph.fromEdgeTuples(sc.parallelize(doubleRing), 1, Some(RandomVertexCut)) + assert(uniqueGraph.edges.count() === ring.size) + assert(uniqueGraph.edges.collect.forall(e => e.attr == 2)) + } + } + + test("Graph.fromEdges") { + withSpark { sc => + val ring = (0L to 100L).zip((1L to 99L) :+ 0L).map { case (a, b) => Edge(a, b, 1) } + val graph = Graph.fromEdges(sc.parallelize(ring), 1.0F) + assert(graph.edges.count() === ring.size) + } + } + + test("Graph.apply") { + withSpark { sc => + val rawEdges = (0L to 98L).zip((1L to 99L) :+ 0L) + val edges: RDD[Edge[Int]] = sc.parallelize(rawEdges).map { case (s, t) => Edge(s, t, 1) } + val vertices: RDD[(VertexID, Boolean)] = sc.parallelize((0L until 10L).map(id => (id, true))) + val graph = Graph(vertices, edges, false) + assert( graph.edges.count() === rawEdges.size ) + // Vertices not explicitly provided but referenced by edges should be created automatically + assert( graph.vertices.count() === 100) + graph.triplets.map { et => + assert((et.srcId < 10 && et.srcAttr) || (et.srcId >= 10 && !et.srcAttr)) + assert((et.dstId < 10 && et.dstAttr) || (et.dstId >= 10 && !et.dstAttr)) + } + } + } + + test("triplets") { + withSpark { sc => + val n = 5 + val star = starGraph(sc, n) + assert(star.triplets.map(et => (et.srcId, et.dstId, et.srcAttr, et.dstAttr)).collect.toSet === + (1 to n).map(x => (0: VertexID, x: VertexID, "v", "v")).toSet) + } + } + + test("partitionBy") { + withSpark { sc => + def mkGraph(edges: List[(Long, Long)]) = Graph.fromEdgeTuples(sc.parallelize(edges, 2), 0) + def nonemptyParts(graph: Graph[Int, Int]) = { + graph.edges.partitionsRDD.mapPartitions { iter => + Iterator(iter.next()._2.iterator.toList) + }.filter(_.nonEmpty) + } + val identicalEdges = List((0L, 1L), (0L, 1L)) + val canonicalEdges = List((0L, 1L), (1L, 0L)) + val sameSrcEdges = List((0L, 1L), (0L, 2L)) + + // The two edges start out in different partitions + for (edges <- List(identicalEdges, canonicalEdges, sameSrcEdges)) { + assert(nonemptyParts(mkGraph(edges)).count === 2) + } + // partitionBy(RandomVertexCut) puts identical edges in the same partition + assert(nonemptyParts(mkGraph(identicalEdges).partitionBy(RandomVertexCut)).count === 1) + // partitionBy(EdgePartition1D) puts same-source edges in the same partition + assert(nonemptyParts(mkGraph(sameSrcEdges).partitionBy(EdgePartition1D)).count === 1) + // partitionBy(CanonicalRandomVertexCut) puts edges that are identical modulo direction into + // the same partition + assert(nonemptyParts(mkGraph(canonicalEdges).partitionBy(CanonicalRandomVertexCut)).count === 1) + // partitionBy(EdgePartition2D) puts identical edges in the same partition + assert(nonemptyParts(mkGraph(identicalEdges).partitionBy(EdgePartition2D)).count === 1) + + // partitionBy(EdgePartition2D) ensures that vertices need only be replicated to 2 * sqrt(p) + // partitions + val n = 100 + val p = 100 + val verts = 1 to n + val graph = Graph.fromEdgeTuples(sc.parallelize(verts.flatMap(x => + verts.filter(y => y % x == 0).map(y => (x: VertexID, y: VertexID))), p), 0) + assert(graph.edges.partitions.length === p) + val partitionedGraph = graph.partitionBy(EdgePartition2D) + assert(graph.edges.partitions.length === p) + val bound = 2 * math.sqrt(p) + // Each vertex should be replicated to at most 2 * sqrt(p) partitions + val partitionSets = partitionedGraph.edges.partitionsRDD.mapPartitions { iter => + val part = iter.next()._2 + Iterator((part.srcIds ++ part.dstIds).toSet) + }.collect + assert(verts.forall(id => partitionSets.count(_.contains(id)) <= bound)) + // This should not be true for the default hash partitioning + val partitionSetsUnpartitioned = graph.edges.partitionsRDD.mapPartitions { iter => + val part = iter.next()._2 + Iterator((part.srcIds ++ part.dstIds).toSet) + }.collect + assert(verts.exists(id => partitionSetsUnpartitioned.count(_.contains(id)) > bound)) + } + } + + test("mapVertices") { + withSpark { sc => + val n = 5 + val star = starGraph(sc, n) + // mapVertices preserving type + val mappedVAttrs = star.mapVertices((vid, attr) => attr + "2") + assert(mappedVAttrs.vertices.collect.toSet === (0 to n).map(x => (x: VertexID, "v2")).toSet) + // mapVertices changing type + val mappedVAttrs2 = star.mapVertices((vid, attr) => attr.length) + assert(mappedVAttrs2.vertices.collect.toSet === (0 to n).map(x => (x: VertexID, 1)).toSet) + } + } + + test("mapEdges") { + withSpark { sc => + val n = 3 + val star = starGraph(sc, n) + val starWithEdgeAttrs = star.mapEdges(e => e.dstId) + + val edges = starWithEdgeAttrs.edges.collect() + assert(edges.size === n) + assert(edges.toSet === (1 to n).map(x => Edge(0, x, x)).toSet) + } + } + + test("mapTriplets") { + withSpark { sc => + val n = 5 + val star = starGraph(sc, n) + assert(star.mapTriplets(et => et.srcAttr + et.dstAttr).edges.collect.toSet === + (1L to n).map(x => Edge(0, x, "vv")).toSet) + } + } + + test("reverse") { + withSpark { sc => + val n = 5 + val star = starGraph(sc, n) + assert(star.reverse.outDegrees.collect.toSet === (1 to n).map(x => (x: VertexID, 1)).toSet) + } + } + + test("subgraph") { + withSpark { sc => + // Create a star graph of 10 veritces. + val n = 10 + val star = starGraph(sc, n) + // Take only vertices whose vids are even + val subgraph = star.subgraph(vpred = (vid, attr) => vid % 2 == 0) + + // We should have 5 vertices. + assert(subgraph.vertices.collect().toSet === (0 to n by 2).map(x => (x, "v")).toSet) + + // And 4 edges. + assert(subgraph.edges.map(_.copy()).collect().toSet === (2 to n by 2).map(x => Edge(0, x, 1)).toSet) + } + } + + test("mask") { + withSpark { sc => + val n = 5 + val vertices = sc.parallelize((0 to n).map(x => (x:VertexID, x))) + val edges = sc.parallelize((1 to n).map(x => Edge(0, x, x))) + val graph: Graph[Int, Int] = Graph(vertices, edges) + + val subgraph = graph.subgraph( + e => e.dstId != 4L, + (vid, vdata) => vid != 3L + ).mapVertices((vid, vdata) => -1).mapEdges(e => -1) + + val projectedGraph = graph.mask(subgraph) + + val v = projectedGraph.vertices.collect().toSet + assert(v === Set((0,0), (1,1), (2,2), (4,4), (5,5))) + + // the map is necessary because of object-reuse in the edge iterator + val e = projectedGraph.edges.map(e => Edge(e.srcId, e.dstId, e.attr)).collect().toSet + assert(e === Set(Edge(0,1,1), Edge(0,2,2), Edge(0,5,5))) + + } + } + + test("groupEdges") { + withSpark { sc => + val n = 5 + val star = starGraph(sc, n) + val doubleStar = Graph.fromEdgeTuples( + sc.parallelize((1 to n).flatMap(x => + List((0: VertexID, x: VertexID), (0: VertexID, x: VertexID))), 1), "v") + val star2 = doubleStar.groupEdges { (a, b) => a} + assert(star2.edges.collect.toArray.sorted(Edge.lexicographicOrdering[Int]) === + star.edges.collect.toArray.sorted(Edge.lexicographicOrdering[Int])) + assert(star2.vertices.collect.toSet === star.vertices.collect.toSet) + } + } + + test("mapReduceTriplets") { + withSpark { sc => + val n = 5 + val star = starGraph(sc, n).mapVertices { (_, _) => 0 } + val starDeg = star.joinVertices(star.degrees){ (vid, oldV, deg) => deg } + val neighborDegreeSums = starDeg.mapReduceTriplets( + edge => Iterator((edge.srcId, edge.dstAttr), (edge.dstId, edge.srcAttr)), + (a: Int, b: Int) => a + b) + assert(neighborDegreeSums.collect().toSet === (0 to n).map(x => (x, n)).toSet) + + // activeSetOpt + val allPairs = for (x <- 1 to n; y <- 1 to n) yield (x: VertexID, y: VertexID) + val complete = Graph.fromEdgeTuples(sc.parallelize(allPairs, 3), 0) + val vids = complete.mapVertices((vid, attr) => vid).cache() + val active = vids.vertices.filter { case (vid, attr) => attr % 2 == 0 } + val numEvenNeighbors = vids.mapReduceTriplets(et => { + // Map function should only run on edges with destination in the active set + if (et.dstId % 2 != 0) { + throw new Exception("map ran on edge with dst vid %d, which is odd".format(et.dstId)) + } + Iterator((et.srcId, 1)) + }, (a: Int, b: Int) => a + b, Some((active, EdgeDirection.In))).collect.toSet + assert(numEvenNeighbors === (1 to n).map(x => (x: VertexID, n / 2)).toSet) + + // outerJoinVertices followed by mapReduceTriplets(activeSetOpt) + val ringEdges = sc.parallelize((0 until n).map(x => (x: VertexID, (x+1) % n: VertexID)), 3) + val ring = Graph.fromEdgeTuples(ringEdges, 0) .mapVertices((vid, attr) => vid).cache() + val changed = ring.vertices.filter { case (vid, attr) => attr % 2 == 1 }.mapValues(-_) + val changedGraph = ring.outerJoinVertices(changed) { (vid, old, newOpt) => newOpt.getOrElse(old) } + val numOddNeighbors = changedGraph.mapReduceTriplets(et => { + // Map function should only run on edges with source in the active set + if (et.srcId % 2 != 1) { + throw new Exception("map ran on edge with src vid %d, which is even".format(et.dstId)) + } + Iterator((et.dstId, 1)) + }, (a: Int, b: Int) => a + b, Some(changed, EdgeDirection.Out)).collect.toSet + assert(numOddNeighbors === (2 to n by 2).map(x => (x: VertexID, 1)).toSet) + + } + } + + test("outerJoinVertices") { + withSpark { sc => + val n = 5 + val reverseStar = starGraph(sc, n).reverse + // outerJoinVertices changing type + val reverseStarDegrees = + reverseStar.outerJoinVertices(reverseStar.outDegrees) { (vid, a, bOpt) => bOpt.getOrElse(0) } + val neighborDegreeSums = reverseStarDegrees.mapReduceTriplets( + et => Iterator((et.srcId, et.dstAttr), (et.dstId, et.srcAttr)), + (a: Int, b: Int) => a + b).collect.toSet + assert(neighborDegreeSums === Set((0: VertexID, n)) ++ (1 to n).map(x => (x: VertexID, 0))) + // outerJoinVertices preserving type + val messages = reverseStar.vertices.mapValues { (vid, attr) => vid.toString } + val newReverseStar = + reverseStar.outerJoinVertices(messages) { (vid, a, bOpt) => a + bOpt.getOrElse("") } + assert(newReverseStar.vertices.map(_._2).collect.toSet === + (0 to n).map(x => "v%d".format(x)).toSet) + } + } + +} diff --git a/graphx/src/test/scala/org/apache/spark/graphx/LocalSparkContext.scala b/graphx/src/test/scala/org/apache/spark/graphx/LocalSparkContext.scala new file mode 100644 index 0000000000..6aec2ea8a9 --- /dev/null +++ b/graphx/src/test/scala/org/apache/spark/graphx/LocalSparkContext.scala @@ -0,0 +1,28 @@ +package org.apache.spark.graphx + +import org.scalatest.Suite +import org.scalatest.BeforeAndAfterEach + +import org.apache.spark.SparkContext + + +/** + * Provides a method to run tests against a {@link SparkContext} variable that is correctly stopped + * after each test. +*/ +trait LocalSparkContext { + System.setProperty("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + System.setProperty("spark.kryo.registrator", "org.apache.spark.graphx.GraphKryoRegistrator") + + /** Runs `f` on a new SparkContext and ensures that it is stopped afterwards. */ + def withSpark[T](f: SparkContext => T) = { + val sc = new SparkContext("local", "test") + try { + f(sc) + } finally { + sc.stop() + // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown + System.clearProperty("spark.driver.port") + } + } +} diff --git a/graphx/src/test/scala/org/apache/spark/graphx/PregelSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/PregelSuite.scala new file mode 100644 index 0000000000..429622357f --- /dev/null +++ b/graphx/src/test/scala/org/apache/spark/graphx/PregelSuite.scala @@ -0,0 +1,41 @@ +package org.apache.spark.graphx + +import org.scalatest.FunSuite + +import org.apache.spark.SparkContext +import org.apache.spark.rdd._ + +class PregelSuite extends FunSuite with LocalSparkContext { + + test("1 iteration") { + withSpark { sc => + val n = 5 + val star = + Graph.fromEdgeTuples(sc.parallelize((1 to n).map(x => (0: VertexID, x: VertexID)), 3), "v") + val result = Pregel(star, 0)( + (vid, attr, msg) => attr, + et => Iterator.empty, + (a: Int, b: Int) => throw new Exception("mergeMsg run unexpectedly")) + assert(result.vertices.collect.toSet === star.vertices.collect.toSet) + } + } + + test("chain propagation") { + withSpark { sc => + val n = 5 + val chain = Graph.fromEdgeTuples( + sc.parallelize((1 until n).map(x => (x: VertexID, x + 1: VertexID)), 3), + 0).cache() + assert(chain.vertices.collect.toSet === (1 to n).map(x => (x: VertexID, 0)).toSet) + val chainWithSeed = chain.mapVertices { (vid, attr) => if (vid == 1) 1 else 0 } + assert(chainWithSeed.vertices.collect.toSet === + Set((1: VertexID, 1)) ++ (2 to n).map(x => (x: VertexID, 0)).toSet) + val result = Pregel(chainWithSeed, 0)( + (vid, attr, msg) => math.max(msg, attr), + et => Iterator((et.dstId, et.srcAttr)), + (a: Int, b: Int) => math.max(a, b)) + assert(result.vertices.collect.toSet === + chain.vertices.mapValues { (vid, attr) => attr + 1 }.collect.toSet) + } + } +} diff --git a/graphx/src/test/scala/org/apache/spark/graphx/SerializerSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/SerializerSuite.scala new file mode 100644 index 0000000000..3ba412c1f8 --- /dev/null +++ b/graphx/src/test/scala/org/apache/spark/graphx/SerializerSuite.scala @@ -0,0 +1,183 @@ +package org.apache.spark.graphx + +import java.io.{EOFException, ByteArrayInputStream, ByteArrayOutputStream} + +import scala.util.Random + +import org.scalatest.FunSuite + +import org.apache.spark._ +import org.apache.spark.graphx.impl._ +import org.apache.spark.graphx.impl.MsgRDDFunctions._ +import org.apache.spark.serializer.SerializationStream + + +class SerializerSuite extends FunSuite with LocalSparkContext { + + test("IntVertexBroadcastMsgSerializer") { + val conf = new SparkConf(false) + val outMsg = new VertexBroadcastMsg[Int](3, 4, 5) + val bout = new ByteArrayOutputStream + val outStrm = new IntVertexBroadcastMsgSerializer(conf).newInstance().serializeStream(bout) + outStrm.writeObject(outMsg) + outStrm.writeObject(outMsg) + bout.flush() + val bin = new ByteArrayInputStream(bout.toByteArray) + val inStrm = new IntVertexBroadcastMsgSerializer(conf).newInstance().deserializeStream(bin) + val inMsg1: VertexBroadcastMsg[Int] = inStrm.readObject() + val inMsg2: VertexBroadcastMsg[Int] = inStrm.readObject() + assert(outMsg.vid === inMsg1.vid) + assert(outMsg.vid === inMsg2.vid) + assert(outMsg.data === inMsg1.data) + assert(outMsg.data === inMsg2.data) + + intercept[EOFException] { + inStrm.readObject() + } + } + + test("LongVertexBroadcastMsgSerializer") { + val conf = new SparkConf(false) + val outMsg = new VertexBroadcastMsg[Long](3, 4, 5) + val bout = new ByteArrayOutputStream + val outStrm = new LongVertexBroadcastMsgSerializer(conf).newInstance().serializeStream(bout) + outStrm.writeObject(outMsg) + outStrm.writeObject(outMsg) + bout.flush() + val bin = new ByteArrayInputStream(bout.toByteArray) + val inStrm = new LongVertexBroadcastMsgSerializer(conf).newInstance().deserializeStream(bin) + val inMsg1: VertexBroadcastMsg[Long] = inStrm.readObject() + val inMsg2: VertexBroadcastMsg[Long] = inStrm.readObject() + assert(outMsg.vid === inMsg1.vid) + assert(outMsg.vid === inMsg2.vid) + assert(outMsg.data === inMsg1.data) + assert(outMsg.data === inMsg2.data) + + intercept[EOFException] { + inStrm.readObject() + } + } + + test("DoubleVertexBroadcastMsgSerializer") { + val conf = new SparkConf(false) + val outMsg = new VertexBroadcastMsg[Double](3, 4, 5.0) + val bout = new ByteArrayOutputStream + val outStrm = new DoubleVertexBroadcastMsgSerializer(conf).newInstance().serializeStream(bout) + outStrm.writeObject(outMsg) + outStrm.writeObject(outMsg) + bout.flush() + val bin = new ByteArrayInputStream(bout.toByteArray) + val inStrm = new DoubleVertexBroadcastMsgSerializer(conf).newInstance().deserializeStream(bin) + val inMsg1: VertexBroadcastMsg[Double] = inStrm.readObject() + val inMsg2: VertexBroadcastMsg[Double] = inStrm.readObject() + assert(outMsg.vid === inMsg1.vid) + assert(outMsg.vid === inMsg2.vid) + assert(outMsg.data === inMsg1.data) + assert(outMsg.data === inMsg2.data) + + intercept[EOFException] { + inStrm.readObject() + } + } + + test("IntAggMsgSerializer") { + val conf = new SparkConf(false) + val outMsg = (4: VertexID, 5) + val bout = new ByteArrayOutputStream + val outStrm = new IntAggMsgSerializer(conf).newInstance().serializeStream(bout) + outStrm.writeObject(outMsg) + outStrm.writeObject(outMsg) + bout.flush() + val bin = new ByteArrayInputStream(bout.toByteArray) + val inStrm = new IntAggMsgSerializer(conf).newInstance().deserializeStream(bin) + val inMsg1: (VertexID, Int) = inStrm.readObject() + val inMsg2: (VertexID, Int) = inStrm.readObject() + assert(outMsg === inMsg1) + assert(outMsg === inMsg2) + + intercept[EOFException] { + inStrm.readObject() + } + } + + test("LongAggMsgSerializer") { + val conf = new SparkConf(false) + val outMsg = (4: VertexID, 1L << 32) + val bout = new ByteArrayOutputStream + val outStrm = new LongAggMsgSerializer(conf).newInstance().serializeStream(bout) + outStrm.writeObject(outMsg) + outStrm.writeObject(outMsg) + bout.flush() + val bin = new ByteArrayInputStream(bout.toByteArray) + val inStrm = new LongAggMsgSerializer(conf).newInstance().deserializeStream(bin) + val inMsg1: (VertexID, Long) = inStrm.readObject() + val inMsg2: (VertexID, Long) = inStrm.readObject() + assert(outMsg === inMsg1) + assert(outMsg === inMsg2) + + intercept[EOFException] { + inStrm.readObject() + } + } + + test("DoubleAggMsgSerializer") { + val conf = new SparkConf(false) + val outMsg = (4: VertexID, 5.0) + val bout = new ByteArrayOutputStream + val outStrm = new DoubleAggMsgSerializer(conf).newInstance().serializeStream(bout) + outStrm.writeObject(outMsg) + outStrm.writeObject(outMsg) + bout.flush() + val bin = new ByteArrayInputStream(bout.toByteArray) + val inStrm = new DoubleAggMsgSerializer(conf).newInstance().deserializeStream(bin) + val inMsg1: (VertexID, Double) = inStrm.readObject() + val inMsg2: (VertexID, Double) = inStrm.readObject() + assert(outMsg === inMsg1) + assert(outMsg === inMsg2) + + intercept[EOFException] { + inStrm.readObject() + } + } + + test("TestShuffleVertexBroadcastMsg") { + withSpark { sc => + val bmsgs = sc.parallelize(0 until 100, 10).map { pid => + new VertexBroadcastMsg[Int](pid, pid, pid) + } + bmsgs.partitionBy(new HashPartitioner(3)).collect() + } + } + + test("variable long encoding") { + def testVarLongEncoding(v: Long, optimizePositive: Boolean) { + val bout = new ByteArrayOutputStream + val stream = new ShuffleSerializationStream(bout) { + def writeObject[T](t: T): SerializationStream = { + writeVarLong(t.asInstanceOf[Long], optimizePositive = optimizePositive) + this + } + } + stream.writeObject(v) + + val bin = new ByteArrayInputStream(bout.toByteArray) + val dstream = new ShuffleDeserializationStream(bin) { + def readObject[T](): T = { + readVarLong(optimizePositive).asInstanceOf[T] + } + } + val read = dstream.readObject[Long]() + assert(read === v) + } + + // Test all variable encoding code path (each branch uses 7 bits, i.e. 1L << 7 difference) + val d = Random.nextLong() % 128 + Seq[Long](0, 1L << 0 + d, 1L << 7 + d, 1L << 14 + d, 1L << 21 + d, 1L << 28 + d, 1L << 35 + d, + 1L << 42 + d, 1L << 49 + d, 1L << 56 + d, 1L << 63 + d).foreach { number => + testVarLongEncoding(number, optimizePositive = false) + testVarLongEncoding(number, optimizePositive = true) + testVarLongEncoding(-number, optimizePositive = false) + testVarLongEncoding(-number, optimizePositive = true) + } + } +} diff --git a/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala new file mode 100644 index 0000000000..573b708e89 --- /dev/null +++ b/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala @@ -0,0 +1,85 @@ +package org.apache.spark.graphx + +import org.apache.spark.SparkContext +import org.apache.spark.graphx.Graph._ +import org.apache.spark.graphx.impl.EdgePartition +import org.apache.spark.rdd._ +import org.scalatest.FunSuite + +class VertexRDDSuite extends FunSuite with LocalSparkContext { + + def vertices(sc: SparkContext, n: Int) = { + VertexRDD(sc.parallelize((0 to n).map(x => (x.toLong, x)), 5)) + } + + test("filter") { + withSpark { sc => + val n = 100 + val verts = vertices(sc, n) + val evens = verts.filter(q => ((q._2 % 2) == 0)) + assert(evens.count === (0 to n).filter(_ % 2 == 0).size) + } + } + + test("mapValues") { + withSpark { sc => + val n = 100 + val verts = vertices(sc, n) + val negatives = verts.mapValues(x => -x).cache() // Allow joining b with a derived RDD of b + assert(negatives.count === n + 1) + } + } + + test("diff") { + withSpark { sc => + val n = 100 + val verts = vertices(sc, n) + val flipEvens = verts.mapValues(x => if (x % 2 == 0) -x else x) + // diff should keep only the changed vertices + assert(verts.diff(flipEvens).map(_._2).collect().toSet === (2 to n by 2).map(-_).toSet) + // diff should keep the vertex values from `other` + assert(flipEvens.diff(verts).map(_._2).collect().toSet === (2 to n by 2).toSet) + } + } + + test("leftJoin") { + withSpark { sc => + val n = 100 + val verts = vertices(sc, n) + val evens = verts.filter(q => ((q._2 % 2) == 0)) + // leftJoin with another VertexRDD + assert(verts.leftJoin(evens) { (id, a, bOpt) => a - bOpt.getOrElse(0) }.collect.toSet === + (0 to n by 2).map(x => (x.toLong, 0)).toSet ++ (1 to n by 2).map(x => (x.toLong, x)).toSet) + // leftJoin with an RDD + val evensRDD = evens.map(identity) + assert(verts.leftJoin(evensRDD) { (id, a, bOpt) => a - bOpt.getOrElse(0) }.collect.toSet === + (0 to n by 2).map(x => (x.toLong, 0)).toSet ++ (1 to n by 2).map(x => (x.toLong, x)).toSet) + } + } + + test("innerJoin") { + withSpark { sc => + val n = 100 + val verts = vertices(sc, n) + val evens = verts.filter(q => ((q._2 % 2) == 0)) + // innerJoin with another VertexRDD + assert(verts.innerJoin(evens) { (id, a, b) => a - b }.collect.toSet === + (0 to n by 2).map(x => (x.toLong, 0)).toSet) + // innerJoin with an RDD + val evensRDD = evens.map(identity) + assert(verts.innerJoin(evensRDD) { (id, a, b) => a - b }.collect.toSet === + (0 to n by 2).map(x => (x.toLong, 0)).toSet) } + } + + test("aggregateUsingIndex") { + withSpark { sc => + val n = 100 + val verts = vertices(sc, n) + val messageTargets = (0 to n) ++ (0 to n by 2) + val messages = sc.parallelize(messageTargets.map(x => (x.toLong, 1))) + assert(verts.aggregateUsingIndex[Int](messages, _ + _).collect.toSet === + (0 to n).map(x => (x.toLong, if (x % 2 == 0) 2 else 1)).toSet) + } + } + +} diff --git a/graphx/src/test/scala/org/apache/spark/graphx/algorithms/ConnectedComponentsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/ConnectedComponentsSuite.scala new file mode 100644 index 0000000000..5e2ecfcde9 --- /dev/null +++ b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/ConnectedComponentsSuite.scala @@ -0,0 +1,83 @@ +package org.apache.spark.graphx.algorithms + +import org.scalatest.FunSuite + +import org.apache.spark.SparkContext +import org.apache.spark.SparkContext._ +import org.apache.spark.graphx._ +import org.apache.spark.graphx.util.GraphGenerators +import org.apache.spark.rdd._ + + +class ConnectedComponentsSuite extends FunSuite with LocalSparkContext { + + test("Grid Connected Components") { + withSpark { sc => + val gridGraph = GraphGenerators.gridGraph(sc, 10, 10).cache() + val ccGraph = ConnectedComponents.run(gridGraph).cache() + val maxCCid = ccGraph.vertices.map { case (vid, ccId) => ccId }.sum + assert(maxCCid === 0) + } + } // end of Grid connected components + + + test("Reverse Grid Connected Components") { + withSpark { sc => + val gridGraph = GraphGenerators.gridGraph(sc, 10, 10).reverse.cache() + val ccGraph = ConnectedComponents.run(gridGraph).cache() + val maxCCid = ccGraph.vertices.map { case (vid, ccId) => ccId }.sum + assert(maxCCid === 0) + } + } // end of Grid connected components + + + test("Chain Connected Components") { + withSpark { sc => + val chain1 = (0 until 9).map(x => (x, x+1) ) + val chain2 = (10 until 20).map(x => (x, x+1) ) + val rawEdges = sc.parallelize(chain1 ++ chain2, 3).map { case (s,d) => (s.toLong, d.toLong) } + val twoChains = Graph.fromEdgeTuples(rawEdges, 1.0).cache() + val ccGraph = ConnectedComponents.run(twoChains).cache() + val vertices = ccGraph.vertices.collect() + for ( (id, cc) <- vertices ) { + if(id < 10) { assert(cc === 0) } + else { assert(cc === 10) } + } + val ccMap = vertices.toMap + for (id <- 0 until 20) { + if (id < 10) { + assert(ccMap(id) === 0) + } else { + assert(ccMap(id) === 10) + } + } + } + } // end of chain connected components + + test("Reverse Chain Connected Components") { + withSpark { sc => + val chain1 = (0 until 9).map(x => (x, x+1) ) + val chain2 = (10 until 20).map(x => (x, x+1) ) + val rawEdges = sc.parallelize(chain1 ++ chain2, 3).map { case (s,d) => (s.toLong, d.toLong) } + val twoChains = Graph.fromEdgeTuples(rawEdges, true).reverse.cache() + val ccGraph = ConnectedComponents.run(twoChains).cache() + val vertices = ccGraph.vertices.collect + for ( (id, cc) <- vertices ) { + if (id < 10) { + assert(cc === 0) + } else { + assert(cc === 10) + } + } + val ccMap = vertices.toMap + for ( id <- 0 until 20 ) { + if (id < 10) { + assert(ccMap(id) === 0) + } else { + assert(ccMap(id) === 10) + } + } + } + } // end of reverse chain connected components + +} diff --git a/graphx/src/test/scala/org/apache/spark/graphx/algorithms/PageRankSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/PageRankSuite.scala new file mode 100644 index 0000000000..e365b1e230 --- /dev/null +++ b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/PageRankSuite.scala @@ -0,0 +1,126 @@ +package org.apache.spark.graphx.algorithms + +import org.scalatest.FunSuite + +import org.apache.spark.graphx._ +import org.apache.spark.SparkContext +import org.apache.spark.SparkContext._ +import org.apache.spark.rdd._ + +import org.apache.spark.graphx.util.GraphGenerators + + +object GridPageRank { + def apply(nRows: Int, nCols: Int, nIter: Int, resetProb: Double) = { + val inNbrs = Array.fill(nRows * nCols)(collection.mutable.MutableList.empty[Int]) + val outDegree = Array.fill(nRows * nCols)(0) + // Convert row column address into vertex ids (row major order) + def sub2ind(r: Int, c: Int): Int = r * nCols + c + // Make the grid graph + for (r <- 0 until nRows; c <- 0 until nCols) { + val ind = sub2ind(r,c) + if (r+1 < nRows) { + outDegree(ind) += 1 + inNbrs(sub2ind(r+1,c)) += ind + } + if (c+1 < nCols) { + outDegree(ind) += 1 + inNbrs(sub2ind(r,c+1)) += ind + } + } + // compute the pagerank + var pr = Array.fill(nRows * nCols)(resetProb) + for (iter <- 0 until nIter) { + val oldPr = pr + pr = new Array[Double](nRows * nCols) + for (ind <- 0 until (nRows * nCols)) { + pr(ind) = resetProb + (1.0 - resetProb) * + inNbrs(ind).map( nbr => oldPr(nbr) / outDegree(nbr)).sum + } + } + (0L until (nRows * nCols)).zip(pr) + } + +} + + +class PageRankSuite extends FunSuite with LocalSparkContext { + + def compareRanks(a: VertexRDD[Double], b: VertexRDD[Double]): Double = { + a.leftJoin(b) { case (id, a, bOpt) => (a - bOpt.getOrElse(0.0)) * (a - bOpt.getOrElse(0.0)) } + .map { case (id, error) => error }.sum + } + + test("Star PageRank") { + withSpark { sc => + val nVertices = 100 + val starGraph = GraphGenerators.starGraph(sc, nVertices).cache() + val resetProb = 0.15 + val errorTol = 1.0e-5 + + val staticRanks1 = PageRank.run(starGraph, numIter = 1, resetProb).vertices.cache() + val staticRanks2 = PageRank.run(starGraph, numIter = 2, resetProb).vertices.cache() + + // Static PageRank should only take 2 iterations to converge + val notMatching = staticRanks1.innerZipJoin(staticRanks2) { (vid, pr1, pr2) => + if (pr1 != pr2) 1 else 0 + }.map { case (vid, test) => test }.sum + assert(notMatching === 0) + + val staticErrors = staticRanks2.map { case (vid, pr) => + val correct = (vid > 0 && pr == resetProb) || + (vid == 0 && math.abs(pr - (resetProb + (1.0 - resetProb) * (resetProb * (nVertices - 1)) )) < 1.0E-5) + if (!correct) 1 else 0 + } + assert(staticErrors.sum === 0) + + val dynamicRanks = PageRank.runUntillConvergence(starGraph, 0, resetProb).vertices.cache() + val standaloneRanks = PageRank.runStandalone(starGraph, 0, resetProb).cache() + assert(compareRanks(staticRanks2, dynamicRanks) < errorTol) + assert(compareRanks(staticRanks2, standaloneRanks) < errorTol) + } + } // end of test Star PageRank + + + + test("Grid PageRank") { + withSpark { sc => + val rows = 10 + val cols = 10 + val resetProb = 0.15 + val tol = 0.0001 + val numIter = 50 + val errorTol = 1.0e-5 + val gridGraph = GraphGenerators.gridGraph(sc, rows, cols).cache() + + val staticRanks = PageRank.run(gridGraph, numIter, resetProb).vertices.cache() + val dynamicRanks = PageRank.runUntillConvergence(gridGraph, tol, resetProb).vertices.cache() + val standaloneRanks = PageRank.runStandalone(gridGraph, tol, resetProb).cache() + val referenceRanks = VertexRDD(sc.parallelize(GridPageRank(rows, cols, numIter, resetProb))) + + assert(compareRanks(staticRanks, referenceRanks) < errorTol) + assert(compareRanks(dynamicRanks, referenceRanks) < errorTol) + assert(compareRanks(standaloneRanks, referenceRanks) < errorTol) + } + } // end of Grid PageRank + + + test("Chain PageRank") { + withSpark { sc => + val chain1 = (0 until 9).map(x => (x, x+1) ) + val rawEdges = sc.parallelize(chain1, 1).map { case (s,d) => (s.toLong, d.toLong) } + val chain = Graph.fromEdgeTuples(rawEdges, 1.0).cache() + val resetProb = 0.15 + val tol = 0.0001 + val numIter = 10 + val errorTol = 1.0e-5 + + val staticRanks = PageRank.run(chain, numIter, resetProb).vertices.cache() + val dynamicRanks = PageRank.runUntillConvergence(chain, tol, resetProb).vertices.cache() + val standaloneRanks = PageRank.runStandalone(chain, tol, resetProb).cache() + + assert(compareRanks(staticRanks, dynamicRanks) < errorTol) + assert(compareRanks(dynamicRanks, standaloneRanks) < errorTol) + } + } +} diff --git a/graphx/src/test/scala/org/apache/spark/graphx/algorithms/SVDPlusPlusSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/SVDPlusPlusSuite.scala new file mode 100644 index 0000000000..06604198d7 --- /dev/null +++ b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/SVDPlusPlusSuite.scala @@ -0,0 +1,30 @@ +package org.apache.spark.graphx.algorithms + +import org.scalatest.FunSuite + +import org.apache.spark.SparkContext +import org.apache.spark.SparkContext._ +import org.apache.spark.graphx._ +import org.apache.spark.graphx.util.GraphGenerators +import org.apache.spark.rdd._ + + +class SVDPlusPlusSuite extends FunSuite with LocalSparkContext { + + test("Test SVD++ with mean square error on training set") { + withSpark { sc => + val svdppErr = 8.0 + val edges = sc.textFile("mllib/data/als/test.data").map { line => + val fields = line.split(",") + Edge(fields(0).toLong * 2, fields(1).toLong * 2 + 1, fields(2).toDouble) + } + val conf = new SVDPlusPlusConf(10, 2, 0.0, 5.0, 0.007, 0.007, 0.005, 0.015) // 2 iterations + var (graph, u) = SVDPlusPlus.run(edges, conf) + val err = graph.vertices.collect.map{ case (vid, vd) => + if (vid % 2 == 1) vd._4 else 0.0 + }.reduce(_ + _) / graph.triplets.collect.size + assert(err <= svdppErr) + } + } + +} diff --git a/graphx/src/test/scala/org/apache/spark/graphx/algorithms/StronglyConnectedComponentsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/StronglyConnectedComponentsSuite.scala new file mode 100644 index 0000000000..696b80944e --- /dev/null +++ b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/StronglyConnectedComponentsSuite.scala @@ -0,0 +1,57 @@ +package org.apache.spark.graphx.algorithms + +import org.scalatest.FunSuite + +import org.apache.spark.SparkContext +import org.apache.spark.SparkContext._ +import org.apache.spark.graphx._ +import org.apache.spark.graphx.util.GraphGenerators +import org.apache.spark.rdd._ + + +class StronglyConnectedComponentsSuite extends FunSuite with LocalSparkContext { + + test("Island Strongly Connected Components") { + withSpark { sc => + val vertices = sc.parallelize((1L to 5L).map(x => (x, -1))) + val edges = sc.parallelize(Seq.empty[Edge[Int]]) + val graph = Graph(vertices, edges) + val sccGraph = StronglyConnectedComponents.run(graph, 5) + for ((id, scc) <- sccGraph.vertices.collect) { + assert(id == scc) + } + } + } + + test("Cycle Strongly Connected Components") { + withSpark { sc => + val rawEdges = sc.parallelize((0L to 6L).map(x => (x, (x + 1) % 7))) + val graph = Graph.fromEdgeTuples(rawEdges, -1) + val sccGraph = StronglyConnectedComponents.run(graph, 20) + for ((id, scc) <- sccGraph.vertices.collect) { + assert(0L == scc) + } + } + } + + test("2 Cycle Strongly Connected Components") { + withSpark { sc => + val edges = + Array(0L -> 1L, 1L -> 2L, 2L -> 0L) ++ + Array(3L -> 4L, 4L -> 5L, 5L -> 3L) ++ + Array(6L -> 0L, 5L -> 7L) + val rawEdges = sc.parallelize(edges) + val graph = Graph.fromEdgeTuples(rawEdges, -1) + val sccGraph = StronglyConnectedComponents.run(graph, 20) + for ((id, scc) <- sccGraph.vertices.collect) { + if (id < 3) + assert(0L == scc) + else if (id < 6) + assert(3L == scc) + else + assert(id == scc) + } + } + } + +} diff --git a/graphx/src/test/scala/org/apache/spark/graphx/algorithms/TriangleCountSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/TriangleCountSuite.scala new file mode 100644 index 0000000000..0e59912754 --- /dev/null +++ b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/TriangleCountSuite.scala @@ -0,0 +1,73 @@ +package org.apache.spark.graphx.algorithms + +import org.scalatest.FunSuite + +import org.apache.spark.SparkContext +import org.apache.spark.SparkContext._ +import org.apache.spark.graphx._ +import org.apache.spark.graphx.util.GraphGenerators +import org.apache.spark.rdd._ + + +class TriangleCountSuite extends FunSuite with LocalSparkContext { + + test("Count a single triangle") { + withSpark { sc => + val rawEdges = sc.parallelize(Array( 0L->1L, 1L->2L, 2L->0L ), 2) + val graph = Graph.fromEdgeTuples(rawEdges, true).cache() + val triangleCount = TriangleCount.run(graph) + val verts = triangleCount.vertices + verts.collect.foreach { case (vid, count) => assert(count === 1) } + } + } + + test("Count two triangles") { + withSpark { sc => + val triangles = Array(0L -> 1L, 1L -> 2L, 2L -> 0L) ++ + Array(0L -> -1L, -1L -> -2L, -2L -> 0L) + val rawEdges = sc.parallelize(triangles, 2) + val graph = Graph.fromEdgeTuples(rawEdges, true).cache() + val triangleCount = TriangleCount.run(graph) + val verts = triangleCount.vertices + verts.collect().foreach { case (vid, count) => + if (vid == 0) { + assert(count === 2) + } else { + assert(count === 1) + } + } + } + } + + test("Count two triangles with bi-directed edges") { + withSpark { sc => + val triangles = + Array(0L -> 1L, 1L -> 2L, 2L -> 0L) ++ + Array(0L -> -1L, -1L -> -2L, -2L -> 0L) + val revTriangles = triangles.map { case (a,b) => (b,a) } + val rawEdges = sc.parallelize(triangles ++ revTriangles, 2) + val graph = Graph.fromEdgeTuples(rawEdges, true).cache() + val triangleCount = TriangleCount.run(graph) + val verts = triangleCount.vertices + verts.collect().foreach { case (vid, count) => + if (vid == 0) { + assert(count === 4) + } else { + assert(count === 2) + } + } + } + } + + test("Count a single triangle with duplicate edges") { + withSpark { sc => + val rawEdges = sc.parallelize(Array(0L -> 1L, 1L -> 2L, 2L -> 0L) ++ + Array(0L -> 1L, 1L -> 2L, 2L -> 0L), 2) + val graph = Graph.fromEdgeTuples(rawEdges, true, uniqueEdges = Some(RandomVertexCut)).cache() + val triangleCount = TriangleCount.run(graph) + val verts = triangleCount.vertices + verts.collect.foreach { case (vid, count) => assert(count === 1) } + } + } + +} diff --git a/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala new file mode 100644 index 0000000000..eb82436f09 --- /dev/null +++ b/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala @@ -0,0 +1,76 @@ +package org.apache.spark.graphx.impl + +import scala.reflect.ClassTag +import scala.util.Random + +import org.scalatest.FunSuite + +import org.apache.spark.graphx._ + +class EdgePartitionSuite extends FunSuite { + + test("reverse") { + val edges = List(Edge(0, 1, 0), Edge(1, 2, 0), Edge(2, 0, 0)) + val reversedEdges = List(Edge(0, 2, 0), Edge(1, 0, 0), Edge(2, 1, 0)) + val builder = new EdgePartitionBuilder[Int] + for (e <- edges) { + builder.add(e.srcId, e.dstId, e.attr) + } + val edgePartition = builder.toEdgePartition + assert(edgePartition.reverse.iterator.map(_.copy()).toList === reversedEdges) + assert(edgePartition.reverse.reverse.iterator.map(_.copy()).toList === edges) + } + + test("map") { + val edges = List(Edge(0, 1, 0), Edge(1, 2, 0), Edge(2, 0, 0)) + val builder = new EdgePartitionBuilder[Int] + for (e <- edges) { + builder.add(e.srcId, e.dstId, e.attr) + } + val edgePartition = builder.toEdgePartition + assert(edgePartition.map(e => e.srcId + e.dstId).iterator.map(_.copy()).toList === + edges.map(e => e.copy(attr = e.srcId + e.dstId))) + } + + test("groupEdges") { + val edges = List( + Edge(0, 1, 1), Edge(1, 2, 2), Edge(2, 0, 4), Edge(0, 1, 8), Edge(1, 2, 16), Edge(2, 0, 32)) + val groupedEdges = List(Edge(0, 1, 9), Edge(1, 2, 18), Edge(2, 0, 36)) + val builder = new EdgePartitionBuilder[Int] + for (e <- edges) { + builder.add(e.srcId, e.dstId, e.attr) + } + val edgePartition = builder.toEdgePartition + assert(edgePartition.groupEdges(_ + _).iterator.map(_.copy()).toList === groupedEdges) + } + + test("indexIterator") { + val edgesFrom0 = List(Edge(0, 1, 0)) + val edgesFrom1 = List(Edge(1, 0, 0), Edge(1, 2, 0)) + val sortedEdges = edgesFrom0 ++ edgesFrom1 + val builder = new EdgePartitionBuilder[Int] + for (e <- Random.shuffle(sortedEdges)) { + builder.add(e.srcId, e.dstId, e.attr) + } + + val edgePartition = builder.toEdgePartition + assert(edgePartition.iterator.map(_.copy()).toList === sortedEdges) + assert(edgePartition.indexIterator(_ == 0).map(_.copy()).toList === edgesFrom0) + assert(edgePartition.indexIterator(_ == 1).map(_.copy()).toList === edgesFrom1) + } + + test("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) + val b = makeEdgePartition(bList) + + assert(a.innerJoin(b) { (src, dst, a, b) => a }.iterator.map(_.copy()).toList === + List(Edge(0, 1, 0), Edge(1, 0, 0), Edge(5, 5, 0))) + } +} diff --git a/graphx/src/test/scala/org/apache/spark/graphx/impl/VertexPartitionSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/impl/VertexPartitionSuite.scala new file mode 100644 index 0000000000..d37d64e8c8 --- /dev/null +++ b/graphx/src/test/scala/org/apache/spark/graphx/impl/VertexPartitionSuite.scala @@ -0,0 +1,113 @@ +package org.apache.spark.graphx.impl + +import org.apache.spark.graphx._ +import org.scalatest.FunSuite + +class VertexPartitionSuite extends FunSuite { + + test("isDefined, filter") { + val vp = VertexPartition(Iterator((0L, 1), (1L, 1))).filter { (vid, attr) => vid == 0 } + assert(vp.isDefined(0)) + assert(!vp.isDefined(1)) + assert(!vp.isDefined(2)) + assert(!vp.isDefined(-1)) + } + + test("isActive, numActives, replaceActives") { + val vp = VertexPartition(Iterator((0L, 1), (1L, 1))) + .filter { (vid, attr) => vid == 0 } + .replaceActives(Iterator(0, 2, 0)) + assert(vp.isActive(0)) + assert(!vp.isActive(1)) + assert(vp.isActive(2)) + assert(!vp.isActive(-1)) + assert(vp.numActives == Some(2)) + } + + test("map") { + val vp = VertexPartition(Iterator((0L, 1), (1L, 1))).map { (vid, attr) => 2 } + assert(vp(0) === 2) + } + + test("diff") { + val vp = VertexPartition(Iterator((0L, 1), (1L, 1), (2L, 1))) + val vp2 = vp.filter { (vid, attr) => vid <= 1 } + val vp3a = vp.map { (vid, attr) => 2 } + val vp3b = VertexPartition(vp3a.iterator) + // diff with same index + val diff1 = vp2.diff(vp3a) + assert(diff1(0) === 2) + assert(diff1(1) === 2) + assert(diff1(2) === 2) + assert(!diff1.isDefined(2)) + // diff with different indexes + val diff2 = vp2.diff(vp3b) + assert(diff2(0) === 2) + assert(diff2(1) === 2) + assert(diff2(2) === 2) + assert(!diff2.isDefined(2)) + } + + test("leftJoin") { + val vp = VertexPartition(Iterator((0L, 1), (1L, 1), (2L, 1))) + val vp2a = vp.filter { (vid, attr) => vid <= 1 }.map { (vid, attr) => 2 } + val vp2b = VertexPartition(vp2a.iterator) + // leftJoin with same index + val join1 = vp.leftJoin(vp2a) { (vid, a, bOpt) => bOpt.getOrElse(a) } + assert(join1.iterator.toSet === Set((0L, 2), (1L, 2), (2L, 1))) + // leftJoin with different indexes + val join2 = vp.leftJoin(vp2b) { (vid, a, bOpt) => bOpt.getOrElse(a) } + assert(join2.iterator.toSet === Set((0L, 2), (1L, 2), (2L, 1))) + // leftJoin an iterator + val join3 = vp.leftJoin(vp2a.iterator) { (vid, a, bOpt) => bOpt.getOrElse(a) } + assert(join3.iterator.toSet === Set((0L, 2), (1L, 2), (2L, 1))) + } + + test("innerJoin") { + val vp = VertexPartition(Iterator((0L, 1), (1L, 1), (2L, 1))) + val vp2a = vp.filter { (vid, attr) => vid <= 1 }.map { (vid, attr) => 2 } + val vp2b = VertexPartition(vp2a.iterator) + // innerJoin with same index + val join1 = vp.innerJoin(vp2a) { (vid, a, b) => b } + assert(join1.iterator.toSet === Set((0L, 2), (1L, 2))) + // innerJoin with different indexes + val join2 = vp.innerJoin(vp2b) { (vid, a, b) => b } + assert(join2.iterator.toSet === Set((0L, 2), (1L, 2))) + // innerJoin an iterator + val join3 = vp.innerJoin(vp2a.iterator) { (vid, a, b) => b } + assert(join3.iterator.toSet === Set((0L, 2), (1L, 2))) + } + + test("createUsingIndex") { + val vp = VertexPartition(Iterator((0L, 1), (1L, 1), (2L, 1))) + val elems = List((0L, 2), (2L, 2), (3L, 2)) + val vp2 = vp.createUsingIndex(elems.iterator) + assert(vp2.iterator.toSet === Set((0L, 2), (2L, 2))) + assert(vp.index === vp2.index) + } + + test("innerJoinKeepLeft") { + val vp = VertexPartition(Iterator((0L, 1), (1L, 1), (2L, 1))) + val elems = List((0L, 2), (2L, 2), (3L, 2)) + val vp2 = vp.innerJoinKeepLeft(elems.iterator) + assert(vp2.iterator.toSet === Set((0L, 2), (2L, 2))) + assert(vp2(1) === 1) + } + + test("aggregateUsingIndex") { + val vp = VertexPartition(Iterator((0L, 1), (1L, 1), (2L, 1))) + val messages = List((0L, "a"), (2L, "b"), (0L, "c"), (3L, "d")) + val vp2 = vp.aggregateUsingIndex[String](messages.iterator, _ + _) + assert(vp2.iterator.toSet === Set((0L, "ac"), (2L, "b"))) + } + + test("reindex") { + val vp = VertexPartition(Iterator((0L, 1), (1L, 1), (2L, 1))) + val vp2 = vp.filter { (vid, attr) => vid <= 1 } + val vp3 = vp2.reindex() + assert(vp2.iterator.toSet === vp3.iterator.toSet) + assert(vp2(2) === 1) + assert(vp3.index.getPos(2) === -1) + } + +} diff --git a/graphx/src/test/scala/org/apache/spark/graphx/util/BytecodeUtilsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/util/BytecodeUtilsSuite.scala new file mode 100644 index 0000000000..11db339750 --- /dev/null +++ b/graphx/src/test/scala/org/apache/spark/graphx/util/BytecodeUtilsSuite.scala @@ -0,0 +1,93 @@ +package org.apache.spark.graphx.util + +import org.scalatest.FunSuite + + +class BytecodeUtilsSuite extends FunSuite { + + import BytecodeUtilsSuite.TestClass + + test("closure invokes a method") { + val c1 = {e: TestClass => println(e.foo); println(e.bar); println(e.baz); } + assert(BytecodeUtils.invokedMethod(c1, classOf[TestClass], "foo")) + assert(BytecodeUtils.invokedMethod(c1, classOf[TestClass], "bar")) + assert(BytecodeUtils.invokedMethod(c1, classOf[TestClass], "baz")) + + val c2 = {e: TestClass => println(e.foo); println(e.bar); } + assert(BytecodeUtils.invokedMethod(c2, classOf[TestClass], "foo")) + assert(BytecodeUtils.invokedMethod(c2, classOf[TestClass], "bar")) + assert(!BytecodeUtils.invokedMethod(c2, classOf[TestClass], "baz")) + + val c3 = {e: TestClass => println(e.foo); } + assert(BytecodeUtils.invokedMethod(c3, classOf[TestClass], "foo")) + assert(!BytecodeUtils.invokedMethod(c3, classOf[TestClass], "bar")) + assert(!BytecodeUtils.invokedMethod(c3, classOf[TestClass], "baz")) + } + + test("closure inside a closure invokes a method") { + val c1 = {e: TestClass => println(e.foo); println(e.bar); println(e.baz); } + val c2 = {e: TestClass => c1(e); println(e.foo); } + assert(BytecodeUtils.invokedMethod(c2, classOf[TestClass], "foo")) + assert(BytecodeUtils.invokedMethod(c2, classOf[TestClass], "bar")) + assert(BytecodeUtils.invokedMethod(c2, classOf[TestClass], "baz")) + } + + test("closure inside a closure inside a closure invokes a method") { + val c1 = {e: TestClass => println(e.baz); } + val c2 = {e: TestClass => c1(e); println(e.foo); } + val c3 = {e: TestClass => c2(e) } + assert(BytecodeUtils.invokedMethod(c3, classOf[TestClass], "foo")) + assert(!BytecodeUtils.invokedMethod(c3, classOf[TestClass], "bar")) + assert(BytecodeUtils.invokedMethod(c3, classOf[TestClass], "baz")) + } + + test("closure calling a function that invokes a method") { + def zoo(e: TestClass) { + println(e.baz) + } + val c1 = {e: TestClass => zoo(e)} + assert(!BytecodeUtils.invokedMethod(c1, classOf[TestClass], "foo")) + assert(!BytecodeUtils.invokedMethod(c1, classOf[TestClass], "bar")) + assert(BytecodeUtils.invokedMethod(c1, classOf[TestClass], "baz")) + } + + test("closure calling a function that invokes a method which uses another closure") { + val c2 = {e: TestClass => println(e.baz)} + def zoo(e: TestClass) { + c2(e) + } + val c1 = {e: TestClass => zoo(e)} + assert(!BytecodeUtils.invokedMethod(c1, classOf[TestClass], "foo")) + assert(!BytecodeUtils.invokedMethod(c1, classOf[TestClass], "bar")) + assert(BytecodeUtils.invokedMethod(c1, classOf[TestClass], "baz")) + } + + test("nested closure") { + val c2 = {e: TestClass => println(e.baz)} + def zoo(e: TestClass, c: TestClass => Unit) { + c(e) + } + val c1 = {e: TestClass => zoo(e, c2)} + assert(!BytecodeUtils.invokedMethod(c1, classOf[TestClass], "foo")) + assert(!BytecodeUtils.invokedMethod(c1, classOf[TestClass], "bar")) + assert(BytecodeUtils.invokedMethod(c1, classOf[TestClass], "baz")) + } + + // The following doesn't work yet, because the byte code doesn't contain any information + // about what exactly "c" is. +// test("invoke interface") { +// val c1 = {e: TestClass => c(e)} +// assert(!BytecodeUtils.invokedMethod(c1, classOf[TestClass], "foo")) +// assert(!BytecodeUtils.invokedMethod(c1, classOf[TestClass], "bar")) +// assert(BytecodeUtils.invokedMethod(c1, classOf[TestClass], "baz")) +// } + + private val c = {e: TestClass => println(e.baz)} +} + + +object BytecodeUtilsSuite { + class TestClass(val foo: Int, val bar: Long) { + def baz: Boolean = false + } +} -- cgit v1.2.3 From b7c92dded33e61976dea10beef88ab52e2009b42 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Thu, 9 Jan 2014 20:44:28 -0800 Subject: Add implicit algorithm methods for Graph; remove standalone PageRank --- .../scala/org/apache/spark/graphx/Analytics.scala | 2 +- .../spark/graphx/algorithms/Algorithms.scala | 56 ++++++++++++++++++++++ .../graphx/algorithms/ConnectedComponents.scala | 6 ++- .../apache/spark/graphx/algorithms/PageRank.scala | 55 ++------------------- .../algorithms/StronglyConnectedComponents.scala | 8 ++-- .../apache/spark/graphx/algorithms/package.scala | 8 ++++ .../algorithms/ConnectedComponentsSuite.scala | 8 ++-- .../spark/graphx/algorithms/PageRankSuite.scala | 27 ++++------- .../StronglyConnectedComponentsSuite.scala | 6 +-- .../graphx/algorithms/TriangleCountSuite.scala | 8 ++-- 10 files changed, 99 insertions(+), 85 deletions(-) create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/algorithms/Algorithms.scala create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/algorithms/package.scala (limited to 'graphx') diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Analytics.scala b/graphx/src/main/scala/org/apache/spark/graphx/Analytics.scala index 0cafc3fdf9..def6d69190 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Analytics.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Analytics.scala @@ -83,7 +83,7 @@ object Analytics extends Logging { println("GRAPHX: Number of edges " + graph.edges.count) //val pr = Analytics.pagerank(graph, numIter) - val pr = PageRank.runStandalone(graph, tol) + val pr = graph.pageRank(tol).vertices println("GRAPHX: Total rank: " + pr.map(_._2).reduce(_+_)) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/Algorithms.scala b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/Algorithms.scala new file mode 100644 index 0000000000..4af7af545c --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/Algorithms.scala @@ -0,0 +1,56 @@ +package org.apache.spark.graphx.algorithms + +import scala.reflect.ClassTag + +import org.apache.spark.graphx._ + +class Algorithms[VD: ClassTag, ED: ClassTag](self: Graph[VD, ED]) { + /** + * Run a dynamic version of PageRank returning a graph with vertex attributes containing the + * PageRank and edge attributes containing the normalized edge weight. + * + * @see [[org.apache.spark.graphx.algorithms.PageRank]], method `runUntilConvergence`. + */ + def pageRank(tol: Double, resetProb: Double = 0.15): Graph[Double, Double] = { + PageRank.runUntilConvergence(self, tol, resetProb) + } + + /** + * Run PageRank for a fixed number of iterations returning a graph with vertex attributes + * containing the PageRank and edge attributes the normalized edge weight. + * + * @see [[org.apache.spark.graphx.algorithms.PageRank]], method `run`. + */ + def staticPageRank(numIter: Int, resetProb: Double = 0.15): Graph[Double, Double] = { + PageRank.run(self, numIter, resetProb) + } + + /** + * Compute the connected component membership of each vertex and return a graph with the vertex + * value containing the lowest vertex id in the connected component containing that vertex. + * + * @see [[org.apache.spark.graphx.algorithms.ConnectedComponents]] + */ + def connectedComponents(): Graph[VertexID, ED] = { + ConnectedComponents.run(self) + } + + /** + * Compute the number of triangles passing through each vertex. + * + * @see [[org.apache.spark.graphx.algorithms.TriangleCount]] + */ + def triangleCount(): Graph[Int, ED] = { + TriangleCount.run(self) + } + + /** + * Compute the strongly connected component (SCC) of each vertex and return a graph with the + * vertex value containing the lowest vertex id in the SCC containing that vertex. + * + * @see [[org.apache.spark.graphx.algorithms.StronglyConnectedComponents]] + */ + def stronglyConnectedComponents(numIter: Int): Graph[VertexID, ED] = { + StronglyConnectedComponents.run(self, numIter) + } +} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/ConnectedComponents.scala b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/ConnectedComponents.scala index a0dd36da60..137a81f4d5 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/ConnectedComponents.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/ConnectedComponents.scala @@ -1,11 +1,13 @@ package org.apache.spark.graphx.algorithms +import scala.reflect.ClassTag + import org.apache.spark.graphx._ object ConnectedComponents { /** - * Compute the connected component membership of each vertex and return an RDD with the vertex + * Compute the connected component membership of each vertex and return a graph with the vertex * value containing the lowest vertex id in the connected component containing that vertex. * * @tparam VD the vertex attribute type (discarded in the computation) @@ -16,7 +18,7 @@ object ConnectedComponents { * @return a graph with vertex attributes containing the smallest vertex in each * connected component */ - def run[VD: Manifest, ED: Manifest](graph: Graph[VD, ED]): Graph[VertexID, ED] = { + def run[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]): Graph[VertexID, ED] = { val ccGraph = graph.mapVertices { case (vid, _) => vid } def sendMessage(edge: EdgeTriplet[VertexID, ED]) = { diff --git a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/PageRank.scala b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/PageRank.scala index 0292b7316d..b423104eda 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/PageRank.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/PageRank.scala @@ -1,5 +1,7 @@ package org.apache.spark.graphx.algorithms +import scala.reflect.ClassTag + import org.apache.spark.Logging import org.apache.spark.graphx._ @@ -42,7 +44,7 @@ object PageRank extends Logging { * containing the normalized weight. * */ - def run[VD: Manifest, ED: Manifest]( + def run[VD: ClassTag, ED: ClassTag]( graph: Graph[VD, ED], numIter: Int, resetProb: Double = 0.15): Graph[Double, Double] = { @@ -109,7 +111,7 @@ object PageRank extends Logging { * @return the graph containing with each vertex containing the PageRank and each edge * containing the normalized weight. */ - def runUntillConvergence[VD: Manifest, ED: Manifest]( + def runUntilConvergence[VD: ClassTag, ED: ClassTag]( graph: Graph[VD, ED], tol: Double, resetProb: Double = 0.15): Graph[Double, Double] = { // Initialize the pagerankGraph with each edge attribute @@ -153,53 +155,4 @@ object PageRank extends Logging { .mapVertices((vid, attr) => attr._1) } // end of deltaPageRank - def runStandalone[VD: Manifest, ED: Manifest]( - graph: Graph[VD, ED], tol: Double, resetProb: Double = 0.15): VertexRDD[Double] = { - - // Initialize the ranks - var ranks: VertexRDD[Double] = graph.vertices.mapValues((vid, attr) => resetProb).cache() - - // Initialize the delta graph where each vertex stores its delta and each edge knows its weight - var deltaGraph: Graph[Double, Double] = - graph.outerJoinVertices(graph.outDegrees)((vid, vdata, deg) => deg.getOrElse(0)) - .mapTriplets(e => 1.0 / e.srcAttr) - .mapVertices((vid, degree) => resetProb).cache() - var numDeltas: Long = ranks.count() - - var prevDeltas: Option[VertexRDD[Double]] = None - - var i = 0 - val weight = (1.0 - resetProb) - while (numDeltas > 0) { - // Compute new deltas. Only deltas that existed in the last round (i.e., were greater than - // `tol`) get to send messages; those that were less than `tol` would send messages less than - // `tol` as well. - val deltas = deltaGraph - .mapReduceTriplets[Double]( - et => Iterator((et.dstId, et.srcAttr * et.attr * weight)), - _ + _, - prevDeltas.map((_, EdgeDirection.Out))) - .filter { case (vid, delta) => delta > tol } - .cache() - prevDeltas = Some(deltas) - numDeltas = deltas.count() - logInfo("Standalone PageRank: iter %d has %d deltas".format(i, numDeltas)) - - // Update deltaGraph with the deltas - deltaGraph = deltaGraph.outerJoinVertices(deltas) { (vid, old, newOpt) => - newOpt.getOrElse(old) - }.cache() - - // Update ranks - ranks = ranks.leftZipJoin(deltas) { (vid, oldRank, deltaOpt) => - oldRank + deltaOpt.getOrElse(0.0) - } - ranks.foreach(x => {}) // force the iteration for ease of debugging - - i += 1 - } - - ranks - } - } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/StronglyConnectedComponents.scala b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/StronglyConnectedComponents.scala index f64fc3ef0f..49ec91aedd 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/StronglyConnectedComponents.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/StronglyConnectedComponents.scala @@ -1,12 +1,14 @@ package org.apache.spark.graphx.algorithms +import scala.reflect.ClassTag + import org.apache.spark.graphx._ object StronglyConnectedComponents { /** - * Compute the strongly connected component (SCC) of each vertex and return an RDD with the vertex - * value containing the lowest vertex id in the SCC containing that vertex. + * Compute the strongly connected component (SCC) of each vertex and return a graph with the + * vertex value containing the lowest vertex id in the SCC containing that vertex. * * @tparam VD the vertex attribute type (discarded in the computation) * @tparam ED the edge attribute type (preserved in the computation) @@ -15,7 +17,7 @@ object StronglyConnectedComponents { * * @return a graph with vertex attributes containing the smallest vertex id in each SCC */ - def run[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], numIter: Int): Graph[VertexID, ED] = { + def run[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED], numIter: Int): Graph[VertexID, ED] = { // the graph we update with final SCC ids, and the graph we return at the end var sccGraph = graph.mapVertices { case (vid, _) => vid } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/package.scala b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/package.scala new file mode 100644 index 0000000000..fbabf1257c --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/package.scala @@ -0,0 +1,8 @@ +package org.apache.spark.graphx + +import scala.reflect.ClassTag + +package object algorithms { + implicit def graphToAlgorithms[VD: ClassTag, ED: ClassTag]( + graph: Graph[VD, ED]): Algorithms[VD, ED] = new Algorithms(graph) +} diff --git a/graphx/src/test/scala/org/apache/spark/graphx/algorithms/ConnectedComponentsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/ConnectedComponentsSuite.scala index 5e2ecfcde9..209191ef07 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/algorithms/ConnectedComponentsSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/ConnectedComponentsSuite.scala @@ -14,7 +14,7 @@ class ConnectedComponentsSuite extends FunSuite with LocalSparkContext { test("Grid Connected Components") { withSpark { sc => val gridGraph = GraphGenerators.gridGraph(sc, 10, 10).cache() - val ccGraph = ConnectedComponents.run(gridGraph).cache() + val ccGraph = gridGraph.connectedComponents().cache() val maxCCid = ccGraph.vertices.map { case (vid, ccId) => ccId }.sum assert(maxCCid === 0) } @@ -24,7 +24,7 @@ class ConnectedComponentsSuite extends FunSuite with LocalSparkContext { test("Reverse Grid Connected Components") { withSpark { sc => val gridGraph = GraphGenerators.gridGraph(sc, 10, 10).reverse.cache() - val ccGraph = ConnectedComponents.run(gridGraph).cache() + val ccGraph = gridGraph.connectedComponents().cache() val maxCCid = ccGraph.vertices.map { case (vid, ccId) => ccId }.sum assert(maxCCid === 0) } @@ -37,7 +37,7 @@ class ConnectedComponentsSuite extends FunSuite with LocalSparkContext { val chain2 = (10 until 20).map(x => (x, x+1) ) val rawEdges = sc.parallelize(chain1 ++ chain2, 3).map { case (s,d) => (s.toLong, d.toLong) } val twoChains = Graph.fromEdgeTuples(rawEdges, 1.0).cache() - val ccGraph = ConnectedComponents.run(twoChains).cache() + val ccGraph = twoChains.connectedComponents().cache() val vertices = ccGraph.vertices.collect() for ( (id, cc) <- vertices ) { if(id < 10) { assert(cc === 0) } @@ -60,7 +60,7 @@ class ConnectedComponentsSuite extends FunSuite with LocalSparkContext { val chain2 = (10 until 20).map(x => (x, x+1) ) val rawEdges = sc.parallelize(chain1 ++ chain2, 3).map { case (s,d) => (s.toLong, d.toLong) } val twoChains = Graph.fromEdgeTuples(rawEdges, true).reverse.cache() - val ccGraph = ConnectedComponents.run(twoChains).cache() + val ccGraph = twoChains.connectedComponents().cache() val vertices = ccGraph.vertices.collect for ( (id, cc) <- vertices ) { if (id < 10) { diff --git a/graphx/src/test/scala/org/apache/spark/graphx/algorithms/PageRankSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/PageRankSuite.scala index e365b1e230..cd857bd3a1 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/algorithms/PageRankSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/PageRankSuite.scala @@ -2,13 +2,12 @@ package org.apache.spark.graphx.algorithms import org.scalatest.FunSuite -import org.apache.spark.graphx._ import org.apache.spark.SparkContext import org.apache.spark.SparkContext._ -import org.apache.spark.rdd._ - +import org.apache.spark.graphx._ +import org.apache.spark.graphx.algorithms._ import org.apache.spark.graphx.util.GraphGenerators - +import org.apache.spark.rdd._ object GridPageRank { def apply(nRows: Int, nCols: Int, nIter: Int, resetProb: Double) = { @@ -58,8 +57,8 @@ class PageRankSuite extends FunSuite with LocalSparkContext { val resetProb = 0.15 val errorTol = 1.0e-5 - val staticRanks1 = PageRank.run(starGraph, numIter = 1, resetProb).vertices.cache() - val staticRanks2 = PageRank.run(starGraph, numIter = 2, resetProb).vertices.cache() + val staticRanks1 = starGraph.staticPageRank(numIter = 1, resetProb).vertices.cache() + val staticRanks2 = starGraph.staticPageRank(numIter = 2, resetProb).vertices.cache() // Static PageRank should only take 2 iterations to converge val notMatching = staticRanks1.innerZipJoin(staticRanks2) { (vid, pr1, pr2) => @@ -74,10 +73,8 @@ class PageRankSuite extends FunSuite with LocalSparkContext { } assert(staticErrors.sum === 0) - val dynamicRanks = PageRank.runUntillConvergence(starGraph, 0, resetProb).vertices.cache() - val standaloneRanks = PageRank.runStandalone(starGraph, 0, resetProb).cache() + val dynamicRanks = starGraph.pageRank(0, resetProb).vertices.cache() assert(compareRanks(staticRanks2, dynamicRanks) < errorTol) - assert(compareRanks(staticRanks2, standaloneRanks) < errorTol) } } // end of test Star PageRank @@ -93,14 +90,12 @@ class PageRankSuite extends FunSuite with LocalSparkContext { val errorTol = 1.0e-5 val gridGraph = GraphGenerators.gridGraph(sc, rows, cols).cache() - val staticRanks = PageRank.run(gridGraph, numIter, resetProb).vertices.cache() - val dynamicRanks = PageRank.runUntillConvergence(gridGraph, tol, resetProb).vertices.cache() - val standaloneRanks = PageRank.runStandalone(gridGraph, tol, resetProb).cache() + val staticRanks = gridGraph.staticPageRank(numIter, resetProb).vertices.cache() + val dynamicRanks = gridGraph.pageRank(tol, resetProb).vertices.cache() val referenceRanks = VertexRDD(sc.parallelize(GridPageRank(rows, cols, numIter, resetProb))) assert(compareRanks(staticRanks, referenceRanks) < errorTol) assert(compareRanks(dynamicRanks, referenceRanks) < errorTol) - assert(compareRanks(standaloneRanks, referenceRanks) < errorTol) } } // end of Grid PageRank @@ -115,12 +110,10 @@ class PageRankSuite extends FunSuite with LocalSparkContext { val numIter = 10 val errorTol = 1.0e-5 - val staticRanks = PageRank.run(chain, numIter, resetProb).vertices.cache() - val dynamicRanks = PageRank.runUntillConvergence(chain, tol, resetProb).vertices.cache() - val standaloneRanks = PageRank.runStandalone(chain, tol, resetProb).cache() + val staticRanks = chain.staticPageRank(numIter, resetProb).vertices.cache() + val dynamicRanks = chain.pageRank(tol, resetProb).vertices.cache() assert(compareRanks(staticRanks, dynamicRanks) < errorTol) - assert(compareRanks(dynamicRanks, standaloneRanks) < errorTol) } } } diff --git a/graphx/src/test/scala/org/apache/spark/graphx/algorithms/StronglyConnectedComponentsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/StronglyConnectedComponentsSuite.scala index 696b80944e..fee7d20161 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/algorithms/StronglyConnectedComponentsSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/StronglyConnectedComponentsSuite.scala @@ -16,7 +16,7 @@ class StronglyConnectedComponentsSuite extends FunSuite with LocalSparkContext { val vertices = sc.parallelize((1L to 5L).map(x => (x, -1))) val edges = sc.parallelize(Seq.empty[Edge[Int]]) val graph = Graph(vertices, edges) - val sccGraph = StronglyConnectedComponents.run(graph, 5) + val sccGraph = graph.stronglyConnectedComponents(5) for ((id, scc) <- sccGraph.vertices.collect) { assert(id == scc) } @@ -27,7 +27,7 @@ class StronglyConnectedComponentsSuite extends FunSuite with LocalSparkContext { withSpark { sc => val rawEdges = sc.parallelize((0L to 6L).map(x => (x, (x + 1) % 7))) val graph = Graph.fromEdgeTuples(rawEdges, -1) - val sccGraph = StronglyConnectedComponents.run(graph, 20) + val sccGraph = graph.stronglyConnectedComponents(20) for ((id, scc) <- sccGraph.vertices.collect) { assert(0L == scc) } @@ -42,7 +42,7 @@ class StronglyConnectedComponentsSuite extends FunSuite with LocalSparkContext { Array(6L -> 0L, 5L -> 7L) val rawEdges = sc.parallelize(edges) val graph = Graph.fromEdgeTuples(rawEdges, -1) - val sccGraph = StronglyConnectedComponents.run(graph, 20) + val sccGraph = graph.stronglyConnectedComponents(20) for ((id, scc) <- sccGraph.vertices.collect) { if (id < 3) assert(0L == scc) diff --git a/graphx/src/test/scala/org/apache/spark/graphx/algorithms/TriangleCountSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/TriangleCountSuite.scala index 0e59912754..b85b289da6 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/algorithms/TriangleCountSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/TriangleCountSuite.scala @@ -15,7 +15,7 @@ class TriangleCountSuite extends FunSuite with LocalSparkContext { withSpark { sc => val rawEdges = sc.parallelize(Array( 0L->1L, 1L->2L, 2L->0L ), 2) val graph = Graph.fromEdgeTuples(rawEdges, true).cache() - val triangleCount = TriangleCount.run(graph) + val triangleCount = graph.triangleCount() val verts = triangleCount.vertices verts.collect.foreach { case (vid, count) => assert(count === 1) } } @@ -27,7 +27,7 @@ class TriangleCountSuite extends FunSuite with LocalSparkContext { Array(0L -> -1L, -1L -> -2L, -2L -> 0L) val rawEdges = sc.parallelize(triangles, 2) val graph = Graph.fromEdgeTuples(rawEdges, true).cache() - val triangleCount = TriangleCount.run(graph) + val triangleCount = graph.triangleCount() val verts = triangleCount.vertices verts.collect().foreach { case (vid, count) => if (vid == 0) { @@ -47,7 +47,7 @@ class TriangleCountSuite extends FunSuite with LocalSparkContext { val revTriangles = triangles.map { case (a,b) => (b,a) } val rawEdges = sc.parallelize(triangles ++ revTriangles, 2) val graph = Graph.fromEdgeTuples(rawEdges, true).cache() - val triangleCount = TriangleCount.run(graph) + val triangleCount = graph.triangleCount() val verts = triangleCount.vertices verts.collect().foreach { case (vid, count) => if (vid == 0) { @@ -64,7 +64,7 @@ class TriangleCountSuite extends FunSuite with LocalSparkContext { val rawEdges = sc.parallelize(Array(0L -> 1L, 1L -> 2L, 2L -> 0L) ++ Array(0L -> 1L, 1L -> 2L, 2L -> 0L), 2) val graph = Graph.fromEdgeTuples(rawEdges, true, uniqueEdges = Some(RandomVertexCut)).cache() - val triangleCount = TriangleCount.run(graph) + val triangleCount = graph.triangleCount() val verts = triangleCount.vertices verts.collect.foreach { case (vid, count) => assert(count === 1) } } -- cgit v1.2.3 From 8ae108f6c48528f3bb7498d586eb51a70c043764 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Thu, 9 Jan 2014 23:25:35 -0800 Subject: Unpersist previous iterations in Pregel --- .../scala/org/apache/spark/graphx/Analytics.scala | 2 +- .../main/scala/org/apache/spark/graphx/EdgeRDD.scala | 5 +++++ .../main/scala/org/apache/spark/graphx/Pregel.scala | 19 +++++++++++++++---- .../scala/org/apache/spark/graphx/VertexRDD.scala | 5 +++++ .../org/apache/spark/graphx/algorithms/PageRank.scala | 1 + .../spark/graphx/impl/ReplicatedVertexView.scala | 16 ++++++++++++++-- 6 files changed, 41 insertions(+), 7 deletions(-) (limited to 'graphx') diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Analytics.scala b/graphx/src/main/scala/org/apache/spark/graphx/Analytics.scala index def6d69190..2c4c885a04 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Analytics.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Analytics.scala @@ -83,7 +83,7 @@ object Analytics extends Logging { println("GRAPHX: Number of edges " + graph.edges.count) //val pr = Analytics.pagerank(graph, numIter) - val pr = graph.pageRank(tol).vertices + val pr = graph.pageRank(tol).vertices.cache() println("GRAPHX: Total rank: " + pr.map(_._2).reduce(_+_)) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala b/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala index e4ef460e6f..7fd6580626 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala @@ -44,6 +44,11 @@ class EdgeRDD[@specialized ED: ClassTag]( /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */ override def cache(): EdgeRDD[ED] = persist() + override def unpersist(blocking: Boolean = true): EdgeRDD[ED] = { + partitionsRDD.unpersist(blocking) + this + } + def mapEdgePartitions[ED2: ClassTag](f: (PartitionID, EdgePartition[ED]) => EdgePartition[ED2]) : EdgeRDD[ED2] = { // iter => iter.map { case (pid, ep) => (pid, f(ep)) } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala b/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala index 8ddb788135..ed8733a806 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala @@ -93,25 +93,36 @@ object Pregel { mergeMsg: (A, A) => A) : Graph[VD, ED] = { - var g = graph.mapVertices( (vid, vdata) => vprog(vid, vdata, initialMsg) ) + var g = graph.mapVertices( (vid, vdata) => vprog(vid, vdata, initialMsg) ).cache() // compute the messages - var messages = g.mapReduceTriplets(sendMsg, mergeMsg).cache() + var messages = g.mapReduceTriplets(sendMsg, mergeMsg) var activeMessages = messages.count() // Loop + var prevG: Graph[VD, ED] = null var i = 0 while (activeMessages > 0 && i < maxIterations) { // Receive the messages. Vertices that didn't get any messages do not appear in newVerts. val newVerts = g.vertices.innerJoin(messages)(vprog).cache() // Update the graph with the new vertices. + prevG = g g = g.outerJoinVertices(newVerts) { (vid, old, newOpt) => newOpt.getOrElse(old) } + g.vertices.cache() val oldMessages = messages // Send new messages. Vertices that didn't get any messages don't appear in newVerts, so don't - // get to send messages. + // get to send messages. We must cache messages so it can be materialized on the next line, + // allowing us to uncache the previous iteration. messages = g.mapReduceTriplets(sendMsg, mergeMsg, Some((newVerts, EdgeDirection.Out))).cache() + // Materializes messages, newVerts, and g.rvv (which materializes g.vertices). Hides + // oldMessages (depended on by newVerts), newVerts (depended on by messages), prevG.vertices + // (depended on by newVerts and g.vertices), and prevG.rvv (depended on by oldMessages and + // g.rvv). activeMessages = messages.count() - // after counting we can unpersist the old messages + // Unpersist hidden RDDs oldMessages.unpersist(blocking=false) + newVerts.unpersist(blocking=false) + prevG.vertices.unpersist(blocking=false) + prevG.asInstanceOf[org.apache.spark.graphx.impl.GraphImpl[VD, ED]].replicatedVertexView.unpersist(blocking=false) // count the iteration i += 1 } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala index cfee9b089f..971e2615d4 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala @@ -98,6 +98,11 @@ class VertexRDD[@specialized VD: ClassTag]( /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */ override def cache(): VertexRDD[VD] = persist() + override def unpersist(blocking: Boolean = true): VertexRDD[VD] = { + partitionsRDD.unpersist(blocking) + this + } + /** Return the number of vertices in this set. */ override def count(): Long = { partitionsRDD.map(_.size).reduce(_ + _) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/PageRank.scala b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/PageRank.scala index b423104eda..179d310554 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/PageRank.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/PageRank.scala @@ -125,6 +125,7 @@ object PageRank extends Logging { .mapTriplets( e => 1.0 / e.srcAttr ) // Set the vertex attributes to (initalPR, delta = 0) .mapVertices( (id, attr) => (0.0, 0.0) ) + .cache() // Display statistics about pagerank logInfo(pagerankGraph.statistics.toString) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala index 63180bc3af..0e2f5a9dd9 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala @@ -14,9 +14,11 @@ import org.apache.spark.graphx._ * specified, `updatedVerts` are treated as incremental updates to the previous view. Otherwise, a * fresh view is created. * - * The view is always cached (i.e., once it is created, it remains materialized). This avoids + * The view is always cached (i.e., once it is evaluated, it remains materialized). This avoids * constructing it twice if the user calls graph.triplets followed by graph.mapReduceTriplets, for - * example. + * example. However, it means iterative algorithms must manually call `Graph.unpersist` on previous + * iterations' graphs for best GC performance. See the implementation of + * [[org.apache.spark.graphx.Pregel]] for an example. */ private[impl] class ReplicatedVertexView[VD: ClassTag]( @@ -51,6 +53,16 @@ class ReplicatedVertexView[VD: ClassTag]( private lazy val dstAttrOnly: RDD[(PartitionID, VertexPartition[VD])] = create(false, true) private lazy val noAttrs: RDD[(PartitionID, VertexPartition[VD])] = create(false, false) + def unpersist(blocking: Boolean = true): ReplicatedVertexView[VD] = { + bothAttrs.unpersist(blocking) + srcAttrOnly.unpersist(blocking) + dstAttrOnly.unpersist(blocking) + noAttrs.unpersist(blocking) + // Don't unpersist localVertexIDMap because a future ReplicatedVertexView may be using it + // without modification + this + } + def get(includeSrc: Boolean, includeDst: Boolean): RDD[(PartitionID, VertexPartition[VD])] = { (includeSrc, includeDst) match { case (true, true) => bothAttrs -- cgit v1.2.3 From 2578332f97d489bf0e238b2dbe1a3e1d0258a910 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Thu, 9 Jan 2014 23:34:35 -0800 Subject: Add Graph.unpersistVertices() --- graphx/src/main/scala/org/apache/spark/graphx/Graph.scala | 6 ++++++ graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala | 14 ++++++-------- .../scala/org/apache/spark/graphx/impl/GraphImpl.scala | 6 ++++++ 3 files changed, 18 insertions(+), 8 deletions(-) (limited to 'graphx') 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 2b7c0a2583..6f2d19d0da 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala @@ -93,6 +93,12 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] { */ def cache(): Graph[VD, ED] + /** + * Uncache only the vertices of this graph, leaving the edges alone. This is useful because most + * graph operations modify the vertices but reuse the edges. + */ + def unpersistVertices(blocking: Boolean = true): Graph[VD, ED] + /** * Repartition the edges in the graph according to partitionStrategy. */ diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala b/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala index ed8733a806..0af230ed29 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala @@ -106,23 +106,21 @@ object Pregel { // Update the graph with the new vertices. prevG = g g = g.outerJoinVertices(newVerts) { (vid, old, newOpt) => newOpt.getOrElse(old) } - g.vertices.cache() + g.cache() val oldMessages = messages // Send new messages. Vertices that didn't get any messages don't appear in newVerts, so don't // get to send messages. We must cache messages so it can be materialized on the next line, // allowing us to uncache the previous iteration. messages = g.mapReduceTriplets(sendMsg, mergeMsg, Some((newVerts, EdgeDirection.Out))).cache() - // Materializes messages, newVerts, and g.rvv (which materializes g.vertices). Hides - // oldMessages (depended on by newVerts), newVerts (depended on by messages), prevG.vertices - // (depended on by newVerts and g.vertices), and prevG.rvv (depended on by oldMessages and - // g.rvv). + // The call to count() materializes `messages`, `newVerts`, and the vertices of `g`. This + // hides oldMessages (depended on by newVerts), newVerts (depended on by messages), and the + // vertices of prevG (depended on by newVerts, oldMessages, and the vertices of g). activeMessages = messages.count() - // Unpersist hidden RDDs + // Unpersist the RDDs hidden by newly-materialized RDDs oldMessages.unpersist(blocking=false) newVerts.unpersist(blocking=false) - prevG.vertices.unpersist(blocking=false) - prevG.asInstanceOf[org.apache.spark.graphx.impl.GraphImpl[VD, ED]].replicatedVertexView.unpersist(blocking=false) + prevG.unpersistVertices(blocking=false) // count the iteration i += 1 } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala index be9f188150..2dd1324d4f 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala @@ -65,6 +65,12 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( override def cache(): Graph[VD, ED] = persist(StorageLevel.MEMORY_ONLY) + override def unpersistVertices(blocking: Boolean = true): Graph[VD, ED] = { + vertices.unpersist(blocking) + replicatedVertexView.unpersist(blocking) + this + } + override def partitionBy(partitionStrategy: PartitionStrategy): Graph[VD, ED] = { val numPartitions = edges.partitions.size val edTag = classTag[ED] -- cgit v1.2.3 From 8b6b8ac87f6ffb92b3395344bf2696d5c7fb3798 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 10 Jan 2014 00:34:08 -0800 Subject: Unpersist previous iterations in GraphLab --- .../main/scala/org/apache/spark/graphx/GraphLab.scala | 16 ++++++++++------ 1 file changed, 10 insertions(+), 6 deletions(-) (limited to 'graphx') diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphLab.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphLab.scala index 437288405f..94cfa7e126 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphLab.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphLab.scala @@ -102,6 +102,7 @@ object GraphLab extends Logging { // Main Loop --------------------------------------------------------------------- var i = 0 var numActive = activeGraph.numVertices + var prevActiveGraph: Graph[(Boolean, VD), ED] = null while (i < numIter && numActive > 0) { // Gather @@ -109,22 +110,25 @@ object GraphLab extends Logging { activeGraph.aggregateNeighbors(gather, mergeFunc, gatherDirection) // Apply - activeGraph = activeGraph.outerJoinVertices(gathered)(apply).cache() - - + val applied = activeGraph.outerJoinVertices(gathered)(apply) // Scatter is basically a gather in the opposite direction so we reverse the edge direction - // activeGraph: Graph[(Boolean, VD), ED] val scattered: RDD[(VertexID, Boolean)] = - activeGraph.aggregateNeighbors(scatter, _ || _, scatterDirection.reverse) + applied.aggregateNeighbors(scatter, _ || _, scatterDirection.reverse) + prevActiveGraph = activeGraph activeGraph = activeGraph.outerJoinVertices(scattered)(applyActive).cache() - // Calculate the number of active vertices + // Calculate the number of active vertices. The call to reduce() materializes the vertices of + // `activeGraph`, hiding the vertices of `prevActiveGraph`. numActive = activeGraph.vertices.map{ case (vid, data) => if (data._1) 1 else 0 }.reduce(_ + _) logInfo("Number active vertices: " + numActive) + + // Unpersist the RDDs hidden by newly-materialized RDDs + prevActiveGraph.unpersistVertices(blocking=false) + i += 1 } -- cgit v1.2.3 From ba511f890ee0d7f85746126c4be734538ede21ea Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 10 Jan 2014 00:35:02 -0800 Subject: Avoid recomputation by caching all multiply-used RDDs --- .../apache/spark/graphx/algorithms/PageRank.scala | 1 + .../spark/graphx/algorithms/SVDPlusPlus.scala | 4 ++ .../algorithms/StronglyConnectedComponents.scala | 9 ++--- .../org/apache/spark/graphx/impl/GraphImpl.scala | 47 +++++++++++++--------- .../org/apache/spark/graphx/GraphOpsSuite.scala | 8 ++-- .../scala/org/apache/spark/graphx/GraphSuite.scala | 8 ++-- .../org/apache/spark/graphx/PregelSuite.scala | 6 +-- .../org/apache/spark/graphx/VertexRDDSuite.scala | 12 +++--- .../algorithms/ConnectedComponentsSuite.scala | 16 ++++---- .../spark/graphx/algorithms/PageRankSuite.scala | 8 ++-- .../spark/graphx/algorithms/SVDPlusPlusSuite.scala | 1 + 11 files changed, 67 insertions(+), 53 deletions(-) (limited to 'graphx') diff --git a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/PageRank.scala b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/PageRank.scala index 179d310554..ab447d5422 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/PageRank.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/PageRank.scala @@ -61,6 +61,7 @@ object PageRank extends Logging { .mapTriplets( e => 1.0 / e.srcAttr ) // Set the vertex attributes to the initial pagerank values .mapVertices( (id, attr) => 1.0 ) + .cache() // Display statistics about pagerank logInfo(pagerankGraph.statistics.toString) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/SVDPlusPlus.scala b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/SVDPlusPlus.scala index 8fdfa3d907..2a13553d79 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/SVDPlusPlus.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/SVDPlusPlus.scala @@ -42,6 +42,7 @@ object SVDPlusPlus { } // calculate global rating mean + edges.cache() val (rs, rc) = edges.map(e => (e.attr, 1L)).reduce((a, b) => (a._1 + b._1, a._2 + b._2)) val u = rs / rc @@ -72,11 +73,13 @@ object SVDPlusPlus { for (i <- 0 until conf.maxIters) { // phase 1, calculate pu + |N(u)|^(-0.5)*sum(y) for user nodes + g.cache() var t1 = g.mapReduceTriplets(et => Iterator((et.srcId, et.dstAttr._2)), (g1: RealVector, g2: RealVector) => g1.add(g2)) g = g.outerJoinVertices(t1) { (vid: VertexID, vd: (RealVector, RealVector, Double, Double), msg: Option[RealVector]) => if (msg.isDefined) (vd._1, vd._1.add(msg.get.mapMultiply(vd._4)), vd._3, vd._4) else vd } // phase 2, update p for user nodes and q, y for item nodes + g.cache() val t2 = g.mapReduceTriplets(mapTrainF(conf, u), (g1: (RealVector, RealVector, Double), g2: (RealVector, RealVector, Double)) => (g1._1.add(g2._1), g1._2.add(g2._2), g1._3 + g2._3)) g = g.outerJoinVertices(t2) { (vid: VertexID, vd: (RealVector, RealVector, Double, Double), msg: Option[(RealVector, RealVector, Double)]) => @@ -94,6 +97,7 @@ object SVDPlusPlus { val err = (et.attr - pred) * (et.attr - pred) Iterator((et.dstId, err)) } + g.cache() val t3 = g.mapReduceTriplets(mapTestF(conf, u), (g1: Double, g2: Double) => g1 + g2) g = g.outerJoinVertices(t3) { (vid: VertexID, vd: (RealVector, RealVector, Double, Double), msg: Option[Double]) => if (msg.isDefined) (vd._1, vd._2, vd._3, msg.get) else vd diff --git a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/StronglyConnectedComponents.scala b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/StronglyConnectedComponents.scala index 49ec91aedd..864f0ec57c 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/StronglyConnectedComponents.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/StronglyConnectedComponents.scala @@ -22,7 +22,7 @@ object StronglyConnectedComponents { // the graph we update with final SCC ids, and the graph we return at the end var sccGraph = graph.mapVertices { case (vid, _) => vid } // graph we are going to work with in our iterations - var sccWorkGraph = graph.mapVertices { case (vid, _) => (vid, false) } + var sccWorkGraph = graph.mapVertices { case (vid, _) => (vid, false) }.cache() var numVertices = sccWorkGraph.numVertices var iter = 0 @@ -32,10 +32,9 @@ object StronglyConnectedComponents { numVertices = sccWorkGraph.numVertices sccWorkGraph = sccWorkGraph.outerJoinVertices(sccWorkGraph.outDegrees) { (vid, data, degreeOpt) => if (degreeOpt.isDefined) data else (vid, true) - } - sccWorkGraph = sccWorkGraph.outerJoinVertices(sccWorkGraph.inDegrees) { + }.outerJoinVertices(sccWorkGraph.inDegrees) { (vid, data, degreeOpt) => if (degreeOpt.isDefined) data else (vid, true) - } + }.cache() // get all vertices to be removed val finalVertices = sccWorkGraph.vertices @@ -47,7 +46,7 @@ object StronglyConnectedComponents { (vid, scc, opt) => opt.getOrElse(scc) } // only keep vertices that are not final - sccWorkGraph = sccWorkGraph.subgraph(vpred = (vid, data) => !data._2) + sccWorkGraph = sccWorkGraph.subgraph(vpred = (vid, data) => !data._2).cache() } while (sccWorkGraph.numVertices < numVertices) sccWorkGraph = sccWorkGraph.mapVertices{ case (vid, (color, isFinal)) => (vid, isFinal) } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala index 2dd1324d4f..987a646c0c 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala @@ -32,19 +32,6 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( @transient val replicatedVertexView: ReplicatedVertexView[VD]) extends Graph[VD, ED] with Serializable { - def this( - vertices: VertexRDD[VD], - edges: EdgeRDD[ED], - routingTable: RoutingTable) = { - this(vertices, edges, routingTable, new ReplicatedVertexView(vertices, edges, routingTable)) - } - - def this( - vertices: VertexRDD[VD], - edges: EdgeRDD[ED]) = { - this(vertices, edges, new RoutingTable(edges, vertices)) - } - /** Return a RDD that brings edges together with their source and destination vertices. */ @transient override val triplets: RDD[EdgeTriplet[VD, ED]] = { val vdTag = classTag[VD] @@ -90,7 +77,7 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( val edgePartition = builder.toEdgePartition Iterator((pid, edgePartition)) }, preservesPartitioning = true).cache()) - new GraphImpl(vertices, newEdges) + GraphImpl(vertices, newEdges) } override def statistics: Map[String, Any] = { @@ -166,7 +153,7 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( override def mapVertices[VD2: ClassTag](f: (VertexID, VD) => VD2): Graph[VD2, ED] = { if (classTag[VD] equals classTag[VD2]) { // The map preserves type, so we can use incremental replication - val newVerts = vertices.mapVertexPartitions(_.map(f)) + val newVerts = vertices.mapVertexPartitions(_.map(f)).cache() val changedVerts = vertices.asInstanceOf[VertexRDD[VD2]].diff(newVerts) val newReplicatedVertexView = new ReplicatedVertexView[VD2]( changedVerts, edges, routingTable, @@ -174,7 +161,7 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( new GraphImpl(newVerts, edges, routingTable, newReplicatedVertexView) } else { // The map does not preserve type, so we must re-replicate all vertices - new GraphImpl(vertices.mapVertexPartitions(_.map(f)), edges, routingTable) + GraphImpl(vertices.mapVertexPartitions(_.map(f)), edges, routingTable) } } @@ -336,7 +323,7 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( } else { // updateF does not preserve type, so we must re-replicate all vertices val newVerts = vertices.leftJoin(updates)(updateF) - new GraphImpl(newVerts, edges, routingTable) + GraphImpl(newVerts, edges, routingTable) } } @@ -382,7 +369,29 @@ object GraphImpl { val vertexRDD = VertexRDD(vids, vPartitioned, defaultVertexAttr) - new GraphImpl(vertexRDD, edgeRDD) + GraphImpl(vertexRDD, edgeRDD) + } + + def apply[VD: ClassTag, ED: ClassTag]( + vertices: VertexRDD[VD], + edges: EdgeRDD[ED]): GraphImpl[VD, ED] = { + // Cache RDDs that are referenced multiple times + edges.cache() + + GraphImpl(vertices, edges, new RoutingTable(edges, vertices)) + } + + def apply[VD: ClassTag, ED: ClassTag]( + vertices: VertexRDD[VD], + edges: EdgeRDD[ED], + routingTable: RoutingTable): GraphImpl[VD, ED] = { + // Cache RDDs that are referenced multiple times. `routingTable` is cached by default, so we + // don't cache it explicitly. + vertices.cache() + edges.cache() + + new GraphImpl( + vertices, edges, routingTable, new ReplicatedVertexView(vertices, edges, routingTable)) } /** @@ -413,7 +422,7 @@ object GraphImpl { val vids = collectVertexIDsFromEdges(edges, new HashPartitioner(edges.partitions.size)) // Create the VertexRDD. val vertices = VertexRDD(vids.mapValues(x => defaultVertexAttr)) - new GraphImpl(vertices, edges) + GraphImpl(vertices, edges) } /** Collects all vids mentioned in edges and partitions them by partitioner. */ diff --git a/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala index cc281fce99..cd3c0bbd30 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala @@ -53,8 +53,8 @@ class GraphOpsSuite extends FunSuite with LocalSparkContext { withSpark { sc => val chain = (0 until 100).map(x => (x, (x+1)%100) ) val rawEdges = sc.parallelize(chain, 3).map { case (s,d) => (s.toLong, d.toLong) } - val graph = Graph.fromEdgeTuples(rawEdges, 1.0) - val nbrs = graph.collectNeighborIds(EdgeDirection.Both) + val graph = Graph.fromEdgeTuples(rawEdges, 1.0).cache() + val nbrs = graph.collectNeighborIds(EdgeDirection.Both).cache() assert(nbrs.count === chain.size) assert(graph.numVertices === nbrs.count) nbrs.collect.foreach { case (vid, nbrs) => assert(nbrs.size === 2) } @@ -71,14 +71,14 @@ class GraphOpsSuite extends FunSuite with LocalSparkContext { val n = 5 val vertices = sc.parallelize((0 to n).map(x => (x:VertexID, x))) val edges = sc.parallelize((1 to n).map(x => Edge(0, x, x))) - val graph: Graph[Int, Int] = Graph(vertices, edges) + val graph: Graph[Int, Int] = Graph(vertices, edges).cache() val filteredGraph = graph.filter( graph => { val degrees: VertexRDD[Int] = graph.outDegrees graph.outerJoinVertices(degrees) {(vid, data, deg) => deg.getOrElse(0)} }, vpred = (vid: VertexID, deg:Int) => deg > 0 - ) + ).cache() val v = filteredGraph.vertices.collect().toSet assert(v === Set((0,0))) diff --git a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala index 094fa722a0..c32a6cbb81 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala @@ -175,7 +175,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { val n = 5 val vertices = sc.parallelize((0 to n).map(x => (x:VertexID, x))) val edges = sc.parallelize((1 to n).map(x => Edge(0, x, x))) - val graph: Graph[Int, Int] = Graph(vertices, edges) + val graph: Graph[Int, Int] = Graph(vertices, edges).cache() val subgraph = graph.subgraph( e => e.dstId != 4L, @@ -211,7 +211,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { test("mapReduceTriplets") { withSpark { sc => val n = 5 - val star = starGraph(sc, n).mapVertices { (_, _) => 0 } + val star = starGraph(sc, n).mapVertices { (_, _) => 0 }.cache() val starDeg = star.joinVertices(star.degrees){ (vid, oldV, deg) => deg } val neighborDegreeSums = starDeg.mapReduceTriplets( edge => Iterator((edge.srcId, edge.dstAttr), (edge.dstId, edge.srcAttr)), @@ -235,7 +235,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { // outerJoinVertices followed by mapReduceTriplets(activeSetOpt) val ringEdges = sc.parallelize((0 until n).map(x => (x: VertexID, (x+1) % n: VertexID)), 3) val ring = Graph.fromEdgeTuples(ringEdges, 0) .mapVertices((vid, attr) => vid).cache() - val changed = ring.vertices.filter { case (vid, attr) => attr % 2 == 1 }.mapValues(-_) + val changed = ring.vertices.filter { case (vid, attr) => attr % 2 == 1 }.mapValues(-_).cache() val changedGraph = ring.outerJoinVertices(changed) { (vid, old, newOpt) => newOpt.getOrElse(old) } val numOddNeighbors = changedGraph.mapReduceTriplets(et => { // Map function should only run on edges with source in the active set @@ -252,7 +252,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { test("outerJoinVertices") { withSpark { sc => val n = 5 - val reverseStar = starGraph(sc, n).reverse + val reverseStar = starGraph(sc, n).reverse.cache() // outerJoinVertices changing type val reverseStarDegrees = reverseStar.outerJoinVertices(reverseStar.outDegrees) { (vid, a, bOpt) => bOpt.getOrElse(0) } diff --git a/graphx/src/test/scala/org/apache/spark/graphx/PregelSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/PregelSuite.scala index 429622357f..1ff3d75633 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/PregelSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/PregelSuite.scala @@ -10,8 +10,8 @@ class PregelSuite extends FunSuite with LocalSparkContext { test("1 iteration") { withSpark { sc => val n = 5 - val star = - Graph.fromEdgeTuples(sc.parallelize((1 to n).map(x => (0: VertexID, x: VertexID)), 3), "v") + val starEdges = (1 to n).map(x => (0: VertexID, x: VertexID)) + val star = Graph.fromEdgeTuples(sc.parallelize(starEdges, 3), "v").cache() val result = Pregel(star, 0)( (vid, attr, msg) => attr, et => Iterator.empty, @@ -27,7 +27,7 @@ class PregelSuite extends FunSuite with LocalSparkContext { sc.parallelize((1 until n).map(x => (x: VertexID, x + 1: VertexID)), 3), 0).cache() assert(chain.vertices.collect.toSet === (1 to n).map(x => (x: VertexID, 0)).toSet) - val chainWithSeed = chain.mapVertices { (vid, attr) => if (vid == 1) 1 else 0 } + val chainWithSeed = chain.mapVertices { (vid, attr) => if (vid == 1) 1 else 0 }.cache() assert(chainWithSeed.vertices.collect.toSet === Set((1: VertexID, 1)) ++ (2 to n).map(x => (x: VertexID, 0)).toSet) val result = Pregel(chainWithSeed, 0)( diff --git a/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala index 573b708e89..d94a3aa67c 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala @@ -33,8 +33,8 @@ class VertexRDDSuite extends FunSuite with LocalSparkContext { test("diff") { withSpark { sc => val n = 100 - val verts = vertices(sc, n) - val flipEvens = verts.mapValues(x => if (x % 2 == 0) -x else x) + val verts = vertices(sc, n).cache() + val flipEvens = verts.mapValues(x => if (x % 2 == 0) -x else x).cache() // diff should keep only the changed vertices assert(verts.diff(flipEvens).map(_._2).collect().toSet === (2 to n by 2).map(-_).toSet) // diff should keep the vertex values from `other` @@ -45,8 +45,8 @@ class VertexRDDSuite extends FunSuite with LocalSparkContext { test("leftJoin") { withSpark { sc => val n = 100 - val verts = vertices(sc, n) - val evens = verts.filter(q => ((q._2 % 2) == 0)) + val verts = vertices(sc, n).cache() + val evens = verts.filter(q => ((q._2 % 2) == 0)).cache() // leftJoin with another VertexRDD assert(verts.leftJoin(evens) { (id, a, bOpt) => a - bOpt.getOrElse(0) }.collect.toSet === (0 to n by 2).map(x => (x.toLong, 0)).toSet ++ (1 to n by 2).map(x => (x.toLong, x)).toSet) @@ -60,8 +60,8 @@ class VertexRDDSuite extends FunSuite with LocalSparkContext { test("innerJoin") { withSpark { sc => val n = 100 - val verts = vertices(sc, n) - val evens = verts.filter(q => ((q._2 % 2) == 0)) + val verts = vertices(sc, n).cache() + val evens = verts.filter(q => ((q._2 % 2) == 0)).cache() // innerJoin with another VertexRDD assert(verts.innerJoin(evens) { (id, a, b) => a - b }.collect.toSet === (0 to n by 2).map(x => (x.toLong, 0)).toSet) diff --git a/graphx/src/test/scala/org/apache/spark/graphx/algorithms/ConnectedComponentsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/ConnectedComponentsSuite.scala index 209191ef07..16fc3fe5a2 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/algorithms/ConnectedComponentsSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/ConnectedComponentsSuite.scala @@ -13,8 +13,8 @@ class ConnectedComponentsSuite extends FunSuite with LocalSparkContext { test("Grid Connected Components") { withSpark { sc => - val gridGraph = GraphGenerators.gridGraph(sc, 10, 10).cache() - val ccGraph = gridGraph.connectedComponents().cache() + val gridGraph = GraphGenerators.gridGraph(sc, 10, 10) + val ccGraph = gridGraph.connectedComponents() val maxCCid = ccGraph.vertices.map { case (vid, ccId) => ccId }.sum assert(maxCCid === 0) } @@ -23,8 +23,8 @@ class ConnectedComponentsSuite extends FunSuite with LocalSparkContext { test("Reverse Grid Connected Components") { withSpark { sc => - val gridGraph = GraphGenerators.gridGraph(sc, 10, 10).reverse.cache() - val ccGraph = gridGraph.connectedComponents().cache() + val gridGraph = GraphGenerators.gridGraph(sc, 10, 10).reverse + val ccGraph = gridGraph.connectedComponents() val maxCCid = ccGraph.vertices.map { case (vid, ccId) => ccId }.sum assert(maxCCid === 0) } @@ -36,8 +36,8 @@ class ConnectedComponentsSuite extends FunSuite with LocalSparkContext { val chain1 = (0 until 9).map(x => (x, x+1) ) val chain2 = (10 until 20).map(x => (x, x+1) ) val rawEdges = sc.parallelize(chain1 ++ chain2, 3).map { case (s,d) => (s.toLong, d.toLong) } - val twoChains = Graph.fromEdgeTuples(rawEdges, 1.0).cache() - val ccGraph = twoChains.connectedComponents().cache() + val twoChains = Graph.fromEdgeTuples(rawEdges, 1.0) + val ccGraph = twoChains.connectedComponents() val vertices = ccGraph.vertices.collect() for ( (id, cc) <- vertices ) { if(id < 10) { assert(cc === 0) } @@ -59,8 +59,8 @@ class ConnectedComponentsSuite extends FunSuite with LocalSparkContext { val chain1 = (0 until 9).map(x => (x, x+1) ) val chain2 = (10 until 20).map(x => (x, x+1) ) val rawEdges = sc.parallelize(chain1 ++ chain2, 3).map { case (s,d) => (s.toLong, d.toLong) } - val twoChains = Graph.fromEdgeTuples(rawEdges, true).reverse.cache() - val ccGraph = twoChains.connectedComponents().cache() + val twoChains = Graph.fromEdgeTuples(rawEdges, true).reverse + val ccGraph = twoChains.connectedComponents() val vertices = ccGraph.vertices.collect for ( (id, cc) <- vertices ) { if (id < 10) { diff --git a/graphx/src/test/scala/org/apache/spark/graphx/algorithms/PageRankSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/PageRankSuite.scala index cd857bd3a1..de2c2d1107 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/algorithms/PageRankSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/PageRankSuite.scala @@ -57,7 +57,7 @@ class PageRankSuite extends FunSuite with LocalSparkContext { val resetProb = 0.15 val errorTol = 1.0e-5 - val staticRanks1 = starGraph.staticPageRank(numIter = 1, resetProb).vertices.cache() + val staticRanks1 = starGraph.staticPageRank(numIter = 1, resetProb).vertices val staticRanks2 = starGraph.staticPageRank(numIter = 2, resetProb).vertices.cache() // Static PageRank should only take 2 iterations to converge @@ -92,7 +92,7 @@ class PageRankSuite extends FunSuite with LocalSparkContext { val staticRanks = gridGraph.staticPageRank(numIter, resetProb).vertices.cache() val dynamicRanks = gridGraph.pageRank(tol, resetProb).vertices.cache() - val referenceRanks = VertexRDD(sc.parallelize(GridPageRank(rows, cols, numIter, resetProb))) + val referenceRanks = VertexRDD(sc.parallelize(GridPageRank(rows, cols, numIter, resetProb))).cache() assert(compareRanks(staticRanks, referenceRanks) < errorTol) assert(compareRanks(dynamicRanks, referenceRanks) < errorTol) @@ -110,8 +110,8 @@ class PageRankSuite extends FunSuite with LocalSparkContext { val numIter = 10 val errorTol = 1.0e-5 - val staticRanks = chain.staticPageRank(numIter, resetProb).vertices.cache() - val dynamicRanks = chain.pageRank(tol, resetProb).vertices.cache() + val staticRanks = chain.staticPageRank(numIter, resetProb).vertices + val dynamicRanks = chain.pageRank(tol, resetProb).vertices assert(compareRanks(staticRanks, dynamicRanks) < errorTol) } diff --git a/graphx/src/test/scala/org/apache/spark/graphx/algorithms/SVDPlusPlusSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/SVDPlusPlusSuite.scala index 06604198d7..7bd93e0e6c 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/algorithms/SVDPlusPlusSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/SVDPlusPlusSuite.scala @@ -20,6 +20,7 @@ class SVDPlusPlusSuite extends FunSuite with LocalSparkContext { } val conf = new SVDPlusPlusConf(10, 2, 0.0, 5.0, 0.007, 0.007, 0.005, 0.015) // 2 iterations var (graph, u) = SVDPlusPlus.run(edges, conf) + graph.cache() val err = graph.vertices.collect.map{ case (vid, vd) => if (vid % 2 == 1) vd._4 else 0.0 }.reduce(_ + _) / graph.triplets.collect.size -- cgit v1.2.3 From 4cc550909a693ac9954a9d7327347353d9513049 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 10 Jan 2014 00:59:59 -0800 Subject: graph -> graphx in log4j.properties --- graphx/src/test/resources/log4j.properties | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) (limited to 'graphx') diff --git a/graphx/src/test/resources/log4j.properties b/graphx/src/test/resources/log4j.properties index 896936d8c4..85e57f0c4b 100644 --- a/graphx/src/test/resources/log4j.properties +++ b/graphx/src/test/resources/log4j.properties @@ -19,7 +19,7 @@ log4j.rootCategory=INFO, file log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false -log4j.appender.file.file=graph/target/unit-tests.log +log4j.appender.file.file=graphx/target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n -- cgit v1.2.3 From 729277ebc41f77103ffbfca55942d85bc5ac4800 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 10 Jan 2014 01:53:28 -0800 Subject: Undo 8b6b8ac87f6ffb92b3395344bf2696d5c7fb3798 Getting unpersist right in GraphLab is tricky. --- graphx/src/main/scala/org/apache/spark/graphx/GraphLab.scala | 10 +++------- 1 file changed, 3 insertions(+), 7 deletions(-) (limited to 'graphx') diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphLab.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphLab.scala index 94cfa7e126..7efc69c64e 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphLab.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphLab.scala @@ -110,25 +110,21 @@ object GraphLab extends Logging { activeGraph.aggregateNeighbors(gather, mergeFunc, gatherDirection) // Apply - val applied = activeGraph.outerJoinVertices(gathered)(apply) + val applied = activeGraph.outerJoinVertices(gathered)(apply).cache() // Scatter is basically a gather in the opposite direction so we reverse the edge direction val scattered: RDD[(VertexID, Boolean)] = applied.aggregateNeighbors(scatter, _ || _, scatterDirection.reverse) prevActiveGraph = activeGraph - activeGraph = activeGraph.outerJoinVertices(scattered)(applyActive).cache() + activeGraph = applied.outerJoinVertices(scattered)(applyActive).cache() - // Calculate the number of active vertices. The call to reduce() materializes the vertices of - // `activeGraph`, hiding the vertices of `prevActiveGraph`. + // Calculate the number of active vertices. numActive = activeGraph.vertices.map{ case (vid, data) => if (data._1) 1 else 0 }.reduce(_ + _) logInfo("Number active vertices: " + numActive) - // Unpersist the RDDs hidden by newly-materialized RDDs - prevActiveGraph.unpersistVertices(blocking=false) - i += 1 } -- cgit v1.2.3 From 1b2aad918c6893c73779856e9a0c6fb934e3d65f Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 10 Jan 2014 10:34:40 -0800 Subject: Update graphx/pom.xml to mirror mllib/pom.xml --- graphx/pom.xml | 76 ++++++---------------------------------------------------- 1 file changed, 7 insertions(+), 69 deletions(-) (limited to 'graphx') diff --git a/graphx/pom.xml b/graphx/pom.xml index fd3dcaad7c..3e5faf230d 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -26,15 +26,15 @@ org.apache.spark - spark-graph_2.9.3 + spark-graphx_2.10 jar - Spark Graph + Spark Project GraphX http://spark-project.org/ org.apache.spark - spark-core_2.9.3 + spark-core_${scala.binary.version} ${project.version} provided @@ -45,18 +45,18 @@ org.scalatest - scalatest_${scala.version} + scalatest_${scala.binary.version} test org.scalacheck - scalacheck_${scala.version} + scalacheck_${scala.binary.version} test - target/scala-${scala.version}/classes - target/scala-${scala.version}/test-classes + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes org.scalatest @@ -64,66 +64,4 @@ - - - - hadoop1 - - - org.spark-project - spark-core - ${project.version} - hadoop1 - - - org.apache.hadoop - hadoop-core - provided - - - - - - org.apache.maven.plugins - maven-jar-plugin - - hadoop1 - - - - - - - hadoop2 - - - org.spark-project - spark-core - ${project.version} - hadoop2 - - - org.apache.hadoop - hadoop-core - provided - - - org.apache.hadoop - hadoop-client - provided - - - - - - org.apache.maven.plugins - maven-jar-plugin - - hadoop2 - - - - - - -- cgit v1.2.3 From bf50e8c6cdb30ad9bb13bed20f482236d7771231 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 10 Jan 2014 10:37:04 -0800 Subject: Remove commented code from Analytics --- .../scala/org/apache/spark/graphx/Analytics.scala | 430 --------------------- 1 file changed, 430 deletions(-) (limited to 'graphx') diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Analytics.scala b/graphx/src/main/scala/org/apache/spark/graphx/Analytics.scala index 2c4c885a04..1359b92c6b 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Analytics.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Analytics.scala @@ -44,11 +44,8 @@ object Analytics extends Logging { case _ => throw new IllegalArgumentException("Invalid Partition Strategy: " + v) } } -// setLogLevels(org.apache.log4j.Level.DEBUG, Seq("org.apache.spark")) - val serializer = "org.apache.spark.serializer.KryoSerializer" System.setProperty("spark.serializer", serializer) - //System.setProperty("spark.shuffle.compress", "false") System.setProperty("spark.kryo.registrator", "org.apache.spark.graphx.GraphKryoRegistrator") taskType match { @@ -82,7 +79,6 @@ object Analytics extends Logging { println("GRAPHX: Number of vertices " + graph.vertices.count) println("GRAPHX: Number of edges " + graph.edges.count) - //val pr = Analytics.pagerank(graph, numIter) val pr = graph.pageRank(tol).vertices.cache() println("GRAPHX: Total rank: " + pr.map(_._2).reduce(_+_)) @@ -159,435 +155,9 @@ object Analytics extends Logging { sc.stop() } -// -// case "shortestpath" => { -// -// var numIter = Int.MaxValue -// var isDynamic = true -// var sources: List[Int] = List.empty -// -// options.foreach{ -// case ("numIter", v) => numIter = v.toInt -// case ("dynamic", v) => isDynamic = v.toBoolean -// case ("source", v) => sources ++= List(v.toInt) -// case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) -// } -// -// -// if(!isDynamic && numIter == Int.MaxValue) { -// println("Set number of iterations!") -// sys.exit(1) -// } -// -// if(sources.isEmpty) { -// println("No sources provided!") -// sys.exit(1) -// } -// -// println("======================================") -// println("| Shortest Path |") -// println("--------------------------------------") -// println(" Using parameters:") -// println(" \tDynamic: " + isDynamic) -// println(" \tNumIter: " + numIter) -// println(" \tSources: [" + sources.mkString(", ") + "]") -// println("======================================") -// -// val sc = new SparkContext(host, "ShortestPath(" + fname + ")") -// val graph = GraphLoader.textFile(sc, fname, a => (if(a.isEmpty) 1.0F else a(0).toFloat ) ) -// //val sp = Analytics.shortestPath(graph, sources, numIter) -// // val cc = if(isDynamic) Analytics.dynamicShortestPath(graph, sources, numIter) -// // else Analytics.shortestPath(graph, sources, numIter) -// println("Longest Path: " + sp.vertices.map(_.data).reduce(math.max(_,_))) -// -// sc.stop() -// } - - - // case "als" => { - - // var numIter = 5 - // var lambda = 0.01 - // var latentK = 10 - // var usersFname = "usersFactors.tsv" - // var moviesFname = "moviesFname.tsv" - // var numVPart = 4 - // var numEPart = 4 - - // options.foreach{ - // case ("numIter", v) => numIter = v.toInt - // case ("lambda", v) => lambda = v.toDouble - // case ("latentK", v) => latentK = v.toInt - // case ("usersFname", v) => usersFname = v - // case ("moviesFname", v) => moviesFname = v - // case ("numVPart", v) => numVPart = v.toInt - // case ("numEPart", v) => numEPart = v.toInt - // case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) - // } - - // println("======================================") - // println("| Alternating Least Squares |") - // println("--------------------------------------") - // println(" Using parameters:") - // println(" \tNumIter: " + numIter) - // println(" \tLambda: " + lambda) - // println(" \tLatentK: " + latentK) - // println(" \tusersFname: " + usersFname) - // println(" \tmoviesFname: " + moviesFname) - // println("======================================") - - // val sc = new SparkContext(host, "ALS(" + fname + ")") - // val graph = GraphLoader.textFile(sc, fname, a => a(0).toDouble ) - // graph.numVPart = numVPart - // graph.numEPart = numEPart - - // val maxUser = graph.edges.map(_._1).reduce(math.max(_,_)) - // val minMovie = graph.edges.map(_._2).reduce(math.min(_,_)) - // assert(maxUser < minMovie) - - // val factors = Analytics.alternatingLeastSquares(graph, latentK, lambda, numIter).cache - // factors.filter(_._1 <= maxUser).map(r => r._1 + "\t" + r._2.mkString("\t")) - // .saveAsTextFile(usersFname) - // factors.filter(_._1 >= minMovie).map(r => r._1 + "\t" + r._2.mkString("\t")) - // .saveAsTextFile(moviesFname) - - // sc.stop() - // } - - case _ => { println("Invalid task type.") } } } - - // /** - // * Compute the shortest path to a set of markers - // */ - // def shortestPath[VD: Manifest](graph: Graph[VD, Double], sources: List[Int], numIter: Int) = { - // val sourceSet = sources.toSet - // val spGraph = graph.mapVertices { - // case Vertex(vid, _) => Vertex(vid, (if(sourceSet.contains(vid)) 0.0 else Double.MaxValue)) - // } - // GraphLab.iterateGA[Double, Double, Double](spGraph)( - // (me_id, edge) => edge.otherVertex(me_id).data + edge.data, // gather - // (a: Double, b: Double) => math.min(a, b), // merge - // (v, a: Option[Double]) => math.min(v.data, a.getOrElse(Double.MaxValue)), // apply - // numIter, - // gatherDirection = EdgeDirection.In) - // } - - // /** - // * Compute the connected component membership of each vertex - // * and return an RDD with the vertex value containing the - // * lowest vertex id in the connected component containing - // * that vertex. - // */ - // def dynamicConnectedComponents[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], - // numIter: Int = Int.MaxValue) = { - - // val vertices = graph.vertices.mapPartitions(iter => iter.map { case (vid, _) => (vid, vid) }) - // val edges = graph.edges // .mapValues(v => None) - // val ccGraph = new Graph(vertices, edges) - - // ccGraph.iterateDynamic( - // (me_id, edge) => edge.otherVertex(me_id).data, // gather - // (a: Int, b: Int) => math.min(a, b), // merge - // Integer.MAX_VALUE, - // (v, a: Int) => math.min(v.data, a), // apply - // (me_id, edge) => edge.otherVertex(me_id).data > edge.vertex(me_id).data, // scatter - // numIter, - // gatherEdges = EdgeDirection.Both, - // scatterEdges = EdgeDirection.Both).vertices - // // - // // graph_ret.vertices.collect.foreach(println) - // // graph_ret.edges.take(10).foreach(println) - // } - - - // /** - // * Compute the shortest path to a set of markers - // */ - // def dynamicShortestPath[VD: Manifest, ED: Manifest](graph: Graph[VD, Double], - // sources: List[Int], numIter: Int) = { - // val sourceSet = sources.toSet - // val vertices = graph.vertices.mapPartitions( - // iter => iter.map { - // case (vid, _) => (vid, (if(sourceSet.contains(vid)) 0.0F else Double.MaxValue) ) - // }); - - // val edges = graph.edges // .mapValues(v => None) - // val spGraph = new Graph(vertices, edges) - - // val niterations = Int.MaxValue - // spGraph.iterateDynamic( - // (me_id, edge) => edge.otherVertex(me_id).data + edge.data, // gather - // (a: Double, b: Double) => math.min(a, b), // merge - // Double.MaxValue, - // (v, a: Double) => math.min(v.data, a), // apply - // (me_id, edge) => edge.vertex(me_id).data + edge.data < edge.otherVertex(me_id).data, // scatter - // numIter, - // gatherEdges = EdgeDirection.In, - // scatterEdges = EdgeDirection.Out).vertices - // } - - - // /** - // * - // */ - // def alternatingLeastSquares[VD: ClassTag, ED: ClassTag](graph: Graph[VD, Double], - // latentK: Int, lambda: Double, numIter: Int) = { - // val vertices = graph.vertices.mapPartitions( _.map { - // case (vid, _) => (vid, Array.fill(latentK){ scala.util.Random.nextDouble() } ) - // }).cache - // val maxUser = graph.edges.map(_._1).reduce(math.max(_,_)) - // val edges = graph.edges // .mapValues(v => None) - // val alsGraph = new Graph(vertices, edges) - // alsGraph.numVPart = graph.numVPart - // alsGraph.numEPart = graph.numEPart - - // val niterations = Int.MaxValue - // alsGraph.iterateDynamic[(Array[Double], Array[Double])]( - // (me_id, edge) => { // gather - // val X = edge.otherVertex(me_id).data - // val y = edge.data - // val Xy = X.map(_ * y) - // val XtX = (for(i <- 0 until latentK; j <- i until latentK) yield(X(i) * X(j))).toArray - // (Xy, XtX) - // }, - // (a, b) => { - // // The difference between the while loop and the zip is a FACTOR OF TWO in overall - // // runtime - // var i = 0 - // while(i < a._1.length) { a._1(i) += b._1(i); i += 1 } - // i = 0 - // while(i < a._2.length) { a._2(i) += b._2(i); i += 1 } - // a - // // (a._1.zip(b._1).map{ case (q,r) => q+r }, a._2.zip(b._2).map{ case (q,r) => q+r }) - // }, - // (Array.empty[Double], Array.empty[Double]), // default value is empty - // (vertex, accum) => { // apply - // val XyArray = accum._1 - // val XtXArray = accum._2 - // if(XyArray.isEmpty) vertex.data // no neighbors - // else { - // val XtX = DenseMatrix.tabulate(latentK,latentK){ (i,j) => - // (if(i < j) XtXArray(i + (j+1)*j/2) else XtXArray(i + (j+1)*j/2)) + - // (if(i == j) lambda else 1.0F) //regularization - // } - // val Xy = DenseMatrix.create(latentK,1,XyArray) - // val w = XtX \ Xy - // w.data - // } - // }, - // (me_id, edge) => true, - // numIter, - // gatherEdges = EdgeDirection.Both, - // scatterEdges = EdgeDirection.Both, - // vertex => vertex.id < maxUser).vertices - // } - - // def main(args: Array[String]) = { - // val host = args(0) - // val taskType = args(1) - // val fname = args(2) - // val options = args.drop(3).map { arg => - // arg.dropWhile(_ == '-').split('=') match { - // case Array(opt, v) => (opt -> v) - // case _ => throw new IllegalArgumentException("Invalid argument: " + arg) - // } - // } - - // System.setProperty("spark.serializer", "spark.KryoSerializer") - // //System.setProperty("spark.shuffle.compress", "false") - // System.setProperty("spark.kryo.registrator", "spark.graphx.GraphKryoRegistrator") - - // taskType match { - // case "pagerank" => { - - // var numIter = Int.MaxValue - // var isDynamic = false - // var tol:Double = 0.001 - // var outFname = "" - // var numVPart = 4 - // var numEPart = 4 - - // options.foreach{ - // case ("numIter", v) => numIter = v.toInt - // case ("dynamic", v) => isDynamic = v.toBoolean - // case ("tol", v) => tol = v.toDouble - // case ("output", v) => outFname = v - // case ("numVPart", v) => numVPart = v.toInt - // case ("numEPart", v) => numEPart = v.toInt - // case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) - // } - - // if(!isDynamic && numIter == Int.MaxValue) { - // println("Set number of iterations!") - // sys.exit(1) - // } - // println("======================================") - // println("| PageRank |") - // println("--------------------------------------") - // println(" Using parameters:") - // println(" \tDynamic: " + isDynamic) - // if(isDynamic) println(" \t |-> Tolerance: " + tol) - // println(" \tNumIter: " + numIter) - // println("======================================") - - // val sc = new SparkContext(host, "PageRank(" + fname + ")") - - // val graph = GraphLoader.textFile(sc, fname, a => 1.0).withPartitioner(numVPart, numEPart).cache() - - // val startTime = System.currentTimeMillis - // logInfo("GRAPHX: starting tasks") - // logInfo("GRAPHX: Number of vertices " + graph.vertices.count) - // logInfo("GRAPHX: Number of edges " + graph.edges.count) - - // val pr = Analytics.pagerank(graph, numIter) - // // val pr = if(isDynamic) Analytics.dynamicPagerank(graph, tol, numIter) - // // else Analytics.pagerank(graph, numIter) - // logInfo("GRAPHX: Total rank: " + pr.vertices.map{ case Vertex(id,r) => r }.reduce(_+_) ) - // if (!outFname.isEmpty) { - // println("Saving pageranks of pages to " + outFname) - // pr.vertices.map{case Vertex(id, r) => id + "\t" + r}.saveAsTextFile(outFname) - // } - // logInfo("GRAPHX: Runtime: " + ((System.currentTimeMillis - startTime)/1000.0) + " seconds") - // sc.stop() - // } - - // case "cc" => { - - // var numIter = Int.MaxValue - // var isDynamic = false - - // options.foreach{ - // case ("numIter", v) => numIter = v.toInt - // case ("dynamic", v) => isDynamic = v.toBoolean - // case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) - // } - - // if(!isDynamic && numIter == Int.MaxValue) { - // println("Set number of iterations!") - // sys.exit(1) - // } - // println("======================================") - // println("| Connected Components |") - // println("--------------------------------------") - // println(" Using parameters:") - // println(" \tDynamic: " + isDynamic) - // println(" \tNumIter: " + numIter) - // println("======================================") - - // val sc = new SparkContext(host, "ConnectedComponents(" + fname + ")") - // val graph = GraphLoader.textFile(sc, fname, a => 1.0) - // val cc = Analytics.connectedComponents(graph, numIter) - // // val cc = if(isDynamic) Analytics.dynamicConnectedComponents(graph, numIter) - // // else Analytics.connectedComponents(graph, numIter) - // println("Components: " + cc.vertices.map(_.data).distinct()) - - // sc.stop() - // } - - // case "shortestpath" => { - - // var numIter = Int.MaxValue - // var isDynamic = true - // var sources: List[Int] = List.empty - - // options.foreach{ - // case ("numIter", v) => numIter = v.toInt - // case ("dynamic", v) => isDynamic = v.toBoolean - // case ("source", v) => sources ++= List(v.toInt) - // case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) - // } - - - // if(!isDynamic && numIter == Int.MaxValue) { - // println("Set number of iterations!") - // sys.exit(1) - // } - - // if(sources.isEmpty) { - // println("No sources provided!") - // sys.exit(1) - // } - - // println("======================================") - // println("| Shortest Path |") - // println("--------------------------------------") - // println(" Using parameters:") - // println(" \tDynamic: " + isDynamic) - // println(" \tNumIter: " + numIter) - // println(" \tSources: [" + sources.mkString(", ") + "]") - // println("======================================") - - // val sc = new SparkContext(host, "ShortestPath(" + fname + ")") - // val graph = GraphLoader.textFile(sc, fname, a => (if(a.isEmpty) 1.0 else a(0).toDouble ) ) - // val sp = Analytics.shortestPath(graph, sources, numIter) - // // val cc = if(isDynamic) Analytics.dynamicShortestPath(graph, sources, numIter) - // // else Analytics.shortestPath(graph, sources, numIter) - // println("Longest Path: " + sp.vertices.map(_.data).reduce(math.max(_,_))) - - // sc.stop() - // } - - - // case "als" => { - - // var numIter = 5 - // var lambda = 0.01 - // var latentK = 10 - // var usersFname = "usersFactors.tsv" - // var moviesFname = "moviesFname.tsv" - // var numVPart = 4 - // var numEPart = 4 - - // options.foreach{ - // case ("numIter", v) => numIter = v.toInt - // case ("lambda", v) => lambda = v.toDouble - // case ("latentK", v) => latentK = v.toInt - // case ("usersFname", v) => usersFname = v - // case ("moviesFname", v) => moviesFname = v - // case ("numVPart", v) => numVPart = v.toInt - // case ("numEPart", v) => numEPart = v.toInt - // case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) - // } - - // println("======================================") - // println("| Alternating Least Squares |") - // println("--------------------------------------") - // println(" Using parameters:") - // println(" \tNumIter: " + numIter) - // println(" \tLambda: " + lambda) - // println(" \tLatentK: " + latentK) - // println(" \tusersFname: " + usersFname) - // println(" \tmoviesFname: " + moviesFname) - // println("======================================") - - // val sc = new SparkContext(host, "ALS(" + fname + ")") - // val graph = GraphLoader.textFile(sc, fname, a => a(0).toDouble ) - // graph.numVPart = numVPart - // graph.numEPart = numEPart - - // val maxUser = graph.edges.map(_._1).reduce(math.max(_,_)) - // val minMovie = graph.edges.map(_._2).reduce(math.min(_,_)) - // assert(maxUser < minMovie) - - // val factors = Analytics.alternatingLeastSquares(graph, latentK, lambda, numIter).cache - // factors.filter(_._1 <= maxUser).map(r => r._1 + "\t" + r._2.mkString("\t")) - // .saveAsTextFile(usersFname) - // factors.filter(_._1 >= minMovie).map(r => r._1 + "\t" + r._2.mkString("\t")) - // .saveAsTextFile(moviesFname) - - // sc.stop() - // } - - - // case _ => { - // println("Invalid task type.") - // } - // } - // } - } -- cgit v1.2.3 From cfc10c74a33cfd0997f53cb37053fd69193ee790 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 10 Jan 2014 10:43:23 -0800 Subject: Remove EdgeTriplet.{src,dst}Stale, which were unused --- graphx/src/main/scala/org/apache/spark/graphx/EdgeTriplet.scala | 3 --- 1 file changed, 3 deletions(-) (limited to 'graphx') 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 b0565b7e0e..7b209ed6c7 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/EdgeTriplet.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeTriplet.scala @@ -28,9 +28,6 @@ class EdgeTriplet[VD, ED] extends Edge[ED] { */ var dstAttr: VD = _ //nullValue[VD] - var srcStale: Boolean = false - var dstStale: Boolean = false - /** * Set the edge properties of this triplet. */ -- cgit v1.2.3 From 4c114a7556c30b186f27a3f7faa9dd7a20b223e1 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 10 Jan 2014 11:37:54 -0800 Subject: Start cleaning up Scaladocs in Graph and EdgeRDD --- .../scala/org/apache/spark/graphx/EdgeRDD.scala | 5 +- .../main/scala/org/apache/spark/graphx/Graph.scala | 57 +++++++++------------- 2 files changed, 27 insertions(+), 35 deletions(-) (limited to 'graphx') 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 7fd6580626..d1d9703159 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala @@ -7,7 +7,10 @@ import org.apache.spark.graphx.impl.EdgePartition import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel - +/** + * `EdgeRDD[ED]` extends `RDD[Edge[ED]]` by storing the edges in columnar format on each partition + * for performance. It is constructed using [[org.apache.spark.graphx.impl.EdgePartitionBuilder]]. + */ class EdgeRDD[@specialized ED: ClassTag]( val partitionsRDD: RDD[(PartitionID, EdgePartition[ED])]) extends RDD[Edge[ED]](partitionsRDD.context, List(new OneToOneDependency(partitionsRDD))) { 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 6f2d19d0da..aeeaddb320 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala @@ -15,12 +15,9 @@ import org.apache.spark.storage.StorageLevel * RDDs, the graph is a functional data-structure in which mutating * operations return new graphs. * - * @see GraphOps for additional graph member functions. - * - * @note The majority of the graph operations are implemented in - * `GraphOps`. All the convenience operations are defined in the - * `GraphOps` class which may be shared across multiple graph - * implementations. + * @note [[org.apache.spark.graphx.GraphOps]] contains additional convenience operations. + * [[org.apache.spark.graphx.algorithms.Algorithms]] contains graph algorithms; to access these, + * import `org.apache.spark.graphx.algorithms._`. * * @tparam VD the vertex attribute type * @tparam ED the edge attribute type @@ -32,9 +29,6 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] { * * @note vertex ids are unique. * @return An RDD containing the vertices in this graph - * - * @see Vertex for the vertex type. - * */ val vertices: VertexRDD[VD] @@ -45,8 +39,8 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] { * * @return An RDD containing the edges in this graph * - * @see Edge for the edge type. - * @see edgesWithVertices to get an RDD which contains all the edges + * @see [[org.apache.spark.graphx.Edge]] for the edge type. + * @see `triplets` to get an RDD which contains all the edges * along with their vertex data. * */ @@ -64,11 +58,10 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] { * {{{ * type Color = Int * val graph: Graph[Color, Int] = Graph.textFile("hdfs://file.tsv") - * val numInvalid = graph.edgesWithVertices() - * .map(e => if (e.src.data == e.dst.data) 1 else 0).sum + * val numInvalid = graph.triplets.map(e => if (e.src.data == e.dst.data) 1 else 0).sum * }}} * - * @see edges() If only the edge data and adjacent vertex ids are + * @see `edges` if only the edge data and adjacent vertex ids are * required. * */ @@ -88,19 +81,19 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] { * Return a graph that is cached when first created. This is used to * pin a graph in memory enabling multiple queries to reuse the same * construction process. - * - * @see RDD.cache() for a more detailed explanation of caching. */ def cache(): Graph[VD, ED] /** - * Uncache only the vertices of this graph, leaving the edges alone. This is useful because most - * graph operations modify the vertices but reuse the edges. + * Uncache only the vertices of this graph, leaving the edges alone. This is useful in iterative + * algorithms that modify the vertex attributes but reuse the edges. This method can be used to + * uncache the vertex attributes of previous iterations once they are no longer needed, improving + * GC performance. */ def unpersistVertices(blocking: Boolean = true): Graph[VD, ED] /** - * Repartition the edges in the graph according to partitionStrategy. + * Repartition the edges in the graph according to `partitionStrategy`. */ def partitionBy(partitionStrategy: PartitionStrategy): Graph[VD, ED] @@ -113,8 +106,7 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] { * Construct a new graph where each vertex value has been * transformed by the map function. * - * @note This graph is not changed and that the new graph has the - * same structure. As a consequence the underlying index structures + * @note The new graph has the same structure. As a consequence the underlying index structures * can be reused. * * @param map the function from a vertex object to a new vertex value. @@ -126,8 +118,7 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] { * {{{ * val rawGraph: Graph[(), ()] = Graph.textFile("hdfs://file") * val root = 42 - * var bfsGraph = rawGraph - * .mapVertices[Int]((vid, data) => if (vid == root) 0 else Math.MaxValue) + * var bfsGraph = rawGraph.mapVertices[Int]((vid, data) => if (vid == root) 0 else Math.MaxValue) * }}} * */ @@ -137,7 +128,7 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] { * Construct a new graph where the value of each edge is * transformed by the map operation. This function is not passed * the vertex value for the vertices adjacent to the edge. If - * vertex values are desired use the mapTriplets function. + * vertex values are desired, use the `mapTriplets` method. * * @note This graph is not changed and that the new graph has the * same structure. As a consequence the underlying index structures @@ -156,15 +147,13 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] { } /** - * Construct a new graph transforming the value of each edge using - * the user defined iterator transform. The iterator transform is - * given an iterator over edge triplets within a logical partition - * and should yield a new iterator over the new values of each edge - * in the order in which they are provided to the iterator transform - * If adjacent vertex values are not required, consider using the - * mapEdges function instead. + * Construct a new graph transforming the value of each edge using the user defined iterator + * transform. The iterator transform is given an iterator over edges within a logical partition + * as well as the partition's ID, and it should return a new iterator over the new values of each + * edge. The new iterator's elements must correspond one-to-one with the old iterator's + * elements. If adjacent vertex values are desired, use the `mapTriplets` method. * - * @note This that this does not change the structure of the + * @note This does not change the structure of the * graph or modify the values of this graph. As a consequence * the underlying index structures can be reused. * @@ -183,9 +172,9 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] { * transformed by the map operation. This function passes vertex * values for the adjacent vertices to the map function. If * adjacent vertex values are not required, consider using the - * mapEdges function instead. + * `mapEdges` method instead. * - * @note This that this does not change the structure of the + * @note This does not change the structure of the * graph or modify the values of this graph. As a consequence * the underlying index structures can be reused. * -- cgit v1.2.3 From 5fcd2a61b4f10d039336e2ca3716e99c0d96ded2 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 10 Jan 2014 12:17:04 -0800 Subject: Finish cleaning up Graph docs --- .../main/scala/org/apache/spark/graphx/Graph.scala | 180 ++++++++++----------- 1 file changed, 82 insertions(+), 98 deletions(-) (limited to 'graphx') 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 aeeaddb320..a188de5537 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala @@ -15,8 +15,8 @@ import org.apache.spark.storage.StorageLevel * RDDs, the graph is a functional data-structure in which mutating * operations return new graphs. * - * @note [[org.apache.spark.graphx.GraphOps]] contains additional convenience operations. - * [[org.apache.spark.graphx.algorithms.Algorithms]] contains graph algorithms; to access these, + * @note [[GraphOps]] contains additional convenience operations. + * [[algorithms.Algorithms]] contains graph algorithms; to access these, * import `org.apache.spark.graphx.algorithms._`. * * @tparam VD the vertex attribute type @@ -25,32 +25,31 @@ import org.apache.spark.storage.StorageLevel abstract class Graph[VD: ClassTag, ED: ClassTag] { /** - * Get the vertices and their data. + * An RDD containing the vertices and their associated attributes. * * @note vertex ids are unique. - * @return An RDD containing the vertices in this graph + * @return an RDD containing the vertices in this graph */ val vertices: VertexRDD[VD] /** - * Get the Edges and their data as an RDD. The entries in the RDD - * contain just the source id and target id along with the edge - * data. + * An RDD containing the edges and their associated attributes. The entries in the RDD contain + * just the source id and target id along with the edge data. * - * @return An RDD containing the edges in this graph + * @return an RDD containing the edges in this graph * - * @see [[org.apache.spark.graphx.Edge]] for the edge type. - * @see `triplets` to get an RDD which contains all the edges + * @see [[Edge]] for the edge type. + * @see [[triplets]] to get an RDD which contains all the edges * along with their vertex data. * */ val edges: EdgeRDD[ED] /** - * Get the edges with the vertex data associated with the adjacent - * pair of vertices. + * An RDD containing the edge triplets, which are edges along with the vertex data associated with + * the adjacent vertices. * - * @return An RDD containing edge triplets. + * @return an RDD containing edge triplets * * @example This operation might be used to evaluate a graph * coloring where we would like to check that both vertices are a @@ -68,7 +67,7 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] { val triplets: RDD[EdgeTriplet[VD, ED]] /** - * Cache the vertices and edges associated with this graph. + * Caches the vertices and edges associated with this graph at the specified storage level. * * @param newLevel the level at which to cache the graph. @@ -78,14 +77,14 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] { def persist(newLevel: StorageLevel = StorageLevel.MEMORY_ONLY): Graph[VD, ED] /** - * Return a graph that is cached when first created. This is used to + * Caches the vertices and edges associated with this graph. This is used to * pin a graph in memory enabling multiple queries to reuse the same * construction process. */ def cache(): Graph[VD, ED] /** - * Uncache only the vertices of this graph, leaving the edges alone. This is useful in iterative + * Uncaches only the vertices of this graph, leaving the edges alone. This is useful in iterative * algorithms that modify the vertex attributes but reuse the edges. This method can be used to * uncache the vertex attributes of previous iterations once they are no longer needed, improving * GC performance. @@ -93,23 +92,22 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] { def unpersistVertices(blocking: Boolean = true): Graph[VD, ED] /** - * Repartition the edges in the graph according to `partitionStrategy`. + * Repartitions the edges in the graph according to `partitionStrategy`. */ def partitionBy(partitionStrategy: PartitionStrategy): Graph[VD, ED] /** - * Compute statistics describing the graph representation. + * Computes statistics describing the graph representation. */ def statistics: Map[String, Any] /** - * Construct a new graph where each vertex value has been - * transformed by the map function. + * Transforms each vertex attribute in the graph using the map function. * * @note The new graph has the same structure. As a consequence the underlying index structures * can be reused. * - * @param map the function from a vertex object to a new vertex value. + * @param map the function from a vertex object to a new vertex value * * @tparam VD2 the new vertex data type * @@ -125,10 +123,9 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] { def mapVertices[VD2: ClassTag](map: (VertexID, VD) => VD2): Graph[VD2, ED] /** - * Construct a new graph where the value of each edge is - * transformed by the map operation. This function is not passed - * the vertex value for the vertices adjacent to the edge. If - * vertex values are desired, use the `mapTriplets` method. + * Transforms each edge attribute in the graph using the map function. The map function is not + * passed the vertex value for the vertices adjacent to the edge. If vertex values are desired, + * use [[mapTriplets]]. * * @note This graph is not changed and that the new graph has the * same structure. As a consequence the underlying index structures @@ -147,19 +144,19 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] { } /** - * Construct a new graph transforming the value of each edge using the user defined iterator - * transform. The iterator transform is given an iterator over edges within a logical partition - * as well as the partition's ID, and it should return a new iterator over the new values of each - * edge. The new iterator's elements must correspond one-to-one with the old iterator's - * elements. If adjacent vertex values are desired, use the `mapTriplets` method. + * Transforms each edge attribute using the map function, passing it a whole partition at a + * time. The map function is given an iterator over edges within a logical partition as well as + * the partition's ID, and it should return a new iterator over the new values of each edge. The + * new iterator's elements must correspond one-to-one with the old iterator's elements. If + * adjacent vertex values are desired, use [[mapTriplets]]. * * @note This does not change the structure of the * graph or modify the values of this graph. As a consequence * the underlying index structures can be reused. * - * @param map the function which takes a partition id and an iterator - * over all the edges in the partition and must return an iterator over - * the new values for each edge in the order of the input iterator. + * @param map a function that takes a partition id and an iterator + * over all the edges in the partition, and must return an iterator over + * the new values for each edge in the order of the input iterator * * @tparam ED2 the new edge data type * @@ -168,11 +165,8 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] { map: (PartitionID, Iterator[Edge[ED]]) => Iterator[ED2]): Graph[VD, ED2] /** - * Construct a new graph where the value of each edge is - * transformed by the map operation. This function passes vertex - * values for the adjacent vertices to the map function. If - * adjacent vertex values are not required, consider using the - * `mapEdges` method instead. + * Transforms each edge attribute using the map function, passing it the adjacent vertex attributes + * as well. If adjacent vertex values are not required, consider using [[mapEdges]] instead. * * @note This does not change the structure of the * graph or modify the values of this graph. As a consequence @@ -196,21 +190,17 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] { } /** - * Construct a new graph transforming the value of each edge using - * the user defined iterator transform. The iterator transform is - * given an iterator over edge triplets within a logical partition - * and should yield a new iterator over the new values of each edge - * in the order in which they are provided to the iterator transform - * If adjacent vertex values are not required, consider using the - * mapEdges function instead. - * - * @note This that this does not change the structure of the + * Transforms each edge attribute a partition at a time using the map function, passing it the + * adjacent vertex attributes as well. The map function is given an iterator over edge triplets + * within a logical partition and should yield a new iterator over the new values of each edge in + * the order in which they are provided. If adjacent vertex values are not required, consider + * using [[mapEdges]] instead. + * + * @note This does not change the structure of the * graph or modify the values of this graph. As a consequence * the underlying index structures can be reused. * - * @param map the function which takes a partition id and an iterator - * over all the edges in the partition and must return an iterator over - * the new values for each edge in the order of the input iterator. + * @param map the iterator transform * * @tparam ED2 the new edge data type * @@ -220,86 +210,82 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] { Graph[VD, ED2] /** - * Construct a new graph with all the edges reversed. If this graph - * contains an edge from a to b then the returned graph contains an - * edge from b to a. + * Reverses all edges in the graph. If this graph contains an edge from a to b then the returned + * graph contains an edge from b to a. */ def reverse: Graph[VD, ED] /** - * This function takes a vertex and edge predicate and constructs - * the subgraph that consists of vertices and edges that satisfy the - * predict. The resulting graph contains the vertices and edges - * that satisfy: + * Restricts the graph to only the vertices and edges satisfying the predicates. The resulting + * subgraph satisifies * * {{{ * V' = {v : for all v in V where vpred(v)} * E' = {(u,v): for all (u,v) in E where epred((u,v)) && vpred(u) && vpred(v)} * }}} * - * @param epred the edge predicate which takes a triplet and + * @param epred the edge predicate, which takes a triplet and * evaluates to true if the edge is to remain in the subgraph. Note - * that only edges in which both vertices satisfy the vertex + * that only edges where both vertices satisfy the vertex * predicate are considered. * - * @param vpred the vertex predicate which takes a vertex object and + * @param vpred the vertex predicate, which takes a vertex object and * evaluates to true if the vertex is to be included in the subgraph * * @return the subgraph containing only the vertices and edges that - * satisfy the predicates. + * satisfy the predicates */ def subgraph(epred: EdgeTriplet[VD,ED] => Boolean = (x => true), vpred: (VertexID, VD) => Boolean = ((v,d) => true) ): Graph[VD, ED] /** - * Subgraph of this graph with only vertices and edges from the other graph. + * Restricts the graph to only the vertices and edges that are also in `other`, but keeps the + * attributes from this graph. * @param other the graph to project this graph onto - * @return a graph with vertices and edges that exists in both the current graph and other, - * with vertex and edge data from the current graph. + * @return a graph with vertices and edges that exist in both the current graph and `other`, + * with vertex and edge data from the current graph */ def mask[VD2: ClassTag, ED2: ClassTag](other: Graph[VD2, ED2]): Graph[VD, ED] /** - * This function merges multiple edges between two vertices into a single Edge. For correct - * results, the graph must have been partitioned using partitionBy. + * Merges multiple edges between two vertices into a single edge. For correct results, the graph + * must have been partitioned using [[partitionBy]]. * * @tparam ED2 the type of the resulting edge data after grouping. * - * @param f the user supplied commutative associative function to merge edge attributes for + * @param f the user-supplied commutative associative function to merge edge attributes for * duplicate edges. * - * @return Graph[VD,ED2] The resulting graph with a single Edge for each source, dest vertex pair. + * @return The resulting graph with a single edge for each (source, dest) vertex pair. */ def groupEdges(merge: (ED, ED) => ED): Graph[VD,ED] /** - * The mapReduceTriplets function is used to compute statistics - * about the neighboring edges and vertices of each vertex. The - * user supplied `mapFunc` function is invoked on each edge of the - * graph generating 0 or more "messages" to be "sent" to either - * vertex in the edge. The `reduceFunc` is then used to combine the - * output of the map phase destined to each vertex. + * Computes statistics about the neighboring edges and vertices of each vertex. The user supplied + * `mapFunc` function is invoked on each edge of the graph, generating 0 or more "messages" to be + * "sent" to either vertex in the edge. The `reduceFunc` is then used to combine the output of + * the map phase destined to each vertex. * * @tparam A the type of "message" to be sent to each vertex * * @param mapFunc the user defined map function which returns 0 or - * more messages to neighboring vertices. + * more messages to neighboring vertices * * @param reduceFunc the user defined reduce function which should * be commutative and assosciative and is used to combine the output - * of the map phase. + * of the map phase * - * @param activeSet optionally, a set of "active" vertices and a direction of edges to consider + * @param activeSetOpt optionally, a set of "active" vertices and a direction of edges to consider * when running `mapFunc`. For example, if the direction is Out, `mapFunc` will only be run on - * edges originating from vertices in the active set. `activeSet` must have the same index as the - * graph's vertices. + * edges originating from vertices in the active set. The active set must have the same index as + * the graph's vertices. * - * @example We can use this function to compute the inDegree of each + * @example We can use this function to compute the in-degree of each * vertex * {{{ * val rawGraph: Graph[(),()] = Graph.textFile("twittergraph") * val inDeg: RDD[(VertexID, Int)] = - * mapReduceTriplets[Int](et => Array((et.dst.id, 1)), _ + _) + * mapReduceTriplets[Int](et => Iterator((et.dst.id, 1)), _ + _) * }}} * * @note By expressing computation at the edge level we achieve @@ -316,10 +302,9 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] { : VertexRDD[A] /** - * Join the vertices with an RDD and then apply a function from the - * the vertex and RDD entry to a new vertex value and type. The - * input table should contain at most one entry for each vertex. If - * no entry is provided the map function is invoked passing none. + * Joins the vertices with entries in the `table` RDD and merges the results using `mapFunc`. The + * input table should contain at most one entry for each vertex. If no entry in `table` is + * provided for a particular vertex in the graph, the map function receives `None`. * * @tparam U the type of entry in the table of updates * @tparam VD2 the new vertex value type @@ -331,12 +316,11 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] { * values. The map function is invoked for all vertices, even those * that do not have a corresponding entry in the table. * - * @example This function is used to update the vertices with new - * values based on external data. For example we could add the out - * degree to each vertex record + * @example This function is used to update the vertices with new values based on external data. + * For example we could add the out-degree to each vertex record: * * {{{ - * val rawGraph: Graph[(),()] = Graph.textFile("webgraph") + * val rawGraph: Graph[_, _] = Graph.textFile("webgraph") * val outDeg: RDD[(VertexID, Int)] = rawGraph.outDegrees() * val graph = rawGraph.outerJoinVertices(outDeg) { * (vid, data, optDeg) => optDeg.getOrElse(0) @@ -364,13 +348,13 @@ object Graph { /** * Construct a graph from a collection of edges encoded as vertex id pairs. * - * @param rawEdges a collection of edges in (src,dst) form. + * @param rawEdges a collection of edges in (src, dst) form * @param uniqueEdges if multiple identical edges are found they are combined and the edge * attribute is set to the sum. Otherwise duplicate edges are treated as separate. To enable - * uniqueEdges, a [[PartitionStrategy]] must be provided. + * `uniqueEdges`, a [[PartitionStrategy]] must be provided. * * @return a graph with edge attributes containing either the count of duplicate edges or 1 - * (if `uniqueEdges=None`) and vertex attributes containing the total degree of each vertex. + * (if `uniqueEdges` is `None`) and vertex attributes containing the total degree of each vertex. */ def fromEdgeTuples[VD: ClassTag]( rawEdges: RDD[(VertexID, VertexID)], @@ -400,10 +384,10 @@ object Graph { } /** - * Construct a graph from a collection attributed vertices and - * edges. Duplicate vertices are picked arbitrarily and + * Construct a graph from a collection of vertices and + * edges with attributes. Duplicate vertices are picked arbitrarily and * vertices found in the edge collection but not in the input - * vertices are the default attribute. + * vertices are assigned the default attribute. * * @tparam VD the vertex attribute type * @tparam ED the edge attribute type @@ -412,7 +396,7 @@ object Graph { * @param defaultVertexAttr the default vertex attribute to use for * vertices that are mentioned in edges but not in vertices * @param partitionStrategy the partition strategy to use when - * partitioning the edges. + * partitioning the edges */ def apply[VD: ClassTag, ED: ClassTag]( vertices: RDD[(VertexID, VD)], @@ -422,10 +406,10 @@ object Graph { } /** - * The implicit graphToGraphOPs function extracts the GraphOps member from a graph. + * Implicitly extracts the [[GraphOps]] member from a graph. * * To improve modularity the Graph type only contains a small set of basic operations. All the - * convenience operations are defined in the GraphOps class which may be shared across multiple + * convenience operations are defined in the [[GraphOps]] class which may be shared across multiple * graph implementations. */ implicit def graphToGraphOps[VD: ClassTag, ED: ClassTag](g: Graph[VD, ED]) = g.ops -- cgit v1.2.3 From c39ec3017f0c8dcf5546ab8b9153bfe237b2aa68 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 10 Jan 2014 12:17:17 -0800 Subject: Remove some commented code --- graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala | 1 - graphx/src/main/scala/org/apache/spark/graphx/EdgeTriplet.scala | 4 ---- 2 files changed, 5 deletions(-) (limited to 'graphx') 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 d1d9703159..61228e9628 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala @@ -54,7 +54,6 @@ class EdgeRDD[@specialized ED: ClassTag]( def mapEdgePartitions[ED2: ClassTag](f: (PartitionID, EdgePartition[ED]) => EdgePartition[ED2]) : EdgeRDD[ED2] = { -// iter => iter.map { case (pid, ep) => (pid, f(ep)) } new EdgeRDD[ED2](partitionsRDD.mapPartitions({ iter => val (pid, ep) = iter.next() Iterator(Tuple2(pid, f(pid, ep))) 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 7b209ed6c7..5e2528925f 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/EdgeTriplet.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeTriplet.scala @@ -14,10 +14,6 @@ import org.apache.spark.graphx.impl.VertexPartition * that is not a trait. */ class EdgeTriplet[VD, ED] extends Edge[ED] { -// class EdgeTriplet[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) VD: ClassTag, -// @specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED: ClassTag] extends Edge[ED] { - - /** * The source vertex attribute */ -- cgit v1.2.3 From eee9bc0958cd3b5efa312d4690f93ff259ca4b39 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 10 Jan 2014 12:36:15 -0800 Subject: Remove commented-out perf files --- .../org/apache/spark/graphx/perf/BagelTest.scala | 76 ---------------------- .../org/apache/spark/graphx/perf/SparkTest.scala | 75 --------------------- 2 files changed, 151 deletions(-) delete mode 100644 graphx/src/main/scala/org/apache/spark/graphx/perf/BagelTest.scala delete mode 100644 graphx/src/main/scala/org/apache/spark/graphx/perf/SparkTest.scala (limited to 'graphx') diff --git a/graphx/src/main/scala/org/apache/spark/graphx/perf/BagelTest.scala b/graphx/src/main/scala/org/apache/spark/graphx/perf/BagelTest.scala deleted file mode 100644 index 81332e0800..0000000000 --- a/graphx/src/main/scala/org/apache/spark/graphx/perf/BagelTest.scala +++ /dev/null @@ -1,76 +0,0 @@ -///// This file creates circular dependencies between examples bagle and graph - -// package org.apache.spark.graphx.perf - -// import org.apache.spark._ -// import org.apache.spark.SparkContext._ -// import org.apache.spark.bagel.Bagel - -// import org.apache.spark.examples.bagel -// //import org.apache.spark.bagel.examples._ -// import org.apache.spark.graphx._ - - -// object BagelTest { - -// def main(args: Array[String]) { -// val host = args(0) -// val taskType = args(1) -// val fname = args(2) -// val options = args.drop(3).map { arg => -// arg.dropWhile(_ == '-').split('=') match { -// case Array(opt, v) => (opt -> v) -// case _ => throw new IllegalArgumentException("Invalid argument: " + arg) -// } -// } - -// System.setProperty("spark.serializer", "org.apache.spark.serializer.KryoSerializer") -// //System.setProperty("spark.shuffle.compress", "false") -// System.setProperty("spark.kryo.registrator", "org.apache.spark.bagel.examples.PRKryoRegistrator") - -// var numIter = Int.MaxValue -// var isDynamic = false -// var tol:Float = 0.001F -// var outFname = "" -// var numVPart = 4 -// var numEPart = 4 - -// options.foreach{ -// case ("numIter", v) => numIter = v.toInt -// case ("dynamic", v) => isDynamic = v.toBoolean -// case ("tol", v) => tol = v.toFloat -// case ("output", v) => outFname = v -// case ("numVPart", v) => numVPart = v.toInt -// case ("numEPart", v) => numEPart = v.toInt -// case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) -// } - -// val sc = new SparkContext(host, "PageRank(" + fname + ")") -// val g = GraphLoader.textFile(sc, fname, a => 1.0F).withPartitioner(numVPart, numEPart).cache() -// val startTime = System.currentTimeMillis - -// val numVertices = g.vertices.count() - -// val vertices = g.collectNeighborIds(EdgeDirection.Out).map { case (vid, neighbors) => -// (vid.toString, new PRVertex(1.0, neighbors.map(_.toString))) -// } - -// // Do the computation -// val epsilon = 0.01 / numVertices -// val messages = sc.parallelize(Array[(String, PRMessage)]()) -// val utils = new PageRankUtils -// val result = -// Bagel.run( -// sc, vertices, messages, combiner = new PRCombiner(), -// numPartitions = numVPart)( -// utils.computeWithCombiner(numVertices, epsilon, numIter)) - -// println("Total rank: " + result.map{ case (id, r) => r.value }.reduce(_+_) ) -// if (!outFname.isEmpty) { -// println("Saving pageranks of pages to " + outFname) -// result.map{ case (id, r) => id + "\t" + r.value }.saveAsTextFile(outFname) -// } -// println("Runtime: " + ((System.currentTimeMillis - startTime)/1000.0) + " seconds") -// sc.stop() -// } -// } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/perf/SparkTest.scala b/graphx/src/main/scala/org/apache/spark/graphx/perf/SparkTest.scala deleted file mode 100644 index 24262640ab..0000000000 --- a/graphx/src/main/scala/org/apache/spark/graphx/perf/SparkTest.scala +++ /dev/null @@ -1,75 +0,0 @@ -///// This file creates circular dependencies between examples bagle and graph - - -// package org.apache.spark.graphx.perf - -// import org.apache.spark._ -// import org.apache.spark.SparkContext._ -// import org.apache.spark.bagel.Bagel -// import org.apache.spark.bagel.examples._ -// import org.apache.spark.graphx._ - - -// object SparkTest { - -// def main(args: Array[String]) { -// val host = args(0) -// val taskType = args(1) -// val fname = args(2) -// val options = args.drop(3).map { arg => -// arg.dropWhile(_ == '-').split('=') match { -// case Array(opt, v) => (opt -> v) -// case _ => throw new IllegalArgumentException("Invalid argument: " + arg) -// } -// } - -// System.setProperty("spark.serializer", "org.apache.spark.KryoSerializer") -// //System.setProperty("spark.shuffle.compress", "false") -// System.setProperty("spark.kryo.registrator", "spark.bagel.examples.PRKryoRegistrator") - -// var numIter = Int.MaxValue -// var isDynamic = false -// var tol:Float = 0.001F -// var outFname = "" -// var numVPart = 4 -// var numEPart = 4 - -// options.foreach{ -// case ("numIter", v) => numIter = v.toInt -// case ("dynamic", v) => isDynamic = v.toBoolean -// case ("tol", v) => tol = v.toFloat -// case ("output", v) => outFname = v -// case ("numVPart", v) => numVPart = v.toInt -// case ("numEPart", v) => numEPart = v.toInt -// case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) -// } - -// val sc = new SparkContext(host, "PageRank(" + fname + ")") -// val g = GraphLoader.textFile(sc, fname, a => 1.0F).withPartitioner(numVPart, numEPart).cache() -// val startTime = System.currentTimeMillis - -// val numVertices = g.vertices.count() - -// val vertices = g.collectNeighborIds(EdgeDirection.Out).map { case (vid, neighbors) => -// (vid.toString, new PRVertex(1.0, neighbors.map(_.toString))) -// } - -// // Do the computation -// val epsilon = 0.01 / numVertices -// val messages = sc.parallelize(Array[(String, PRMessage)]()) -// val utils = new PageRankUtils -// val result = -// Bagel.run( -// sc, vertices, messages, combiner = new PRCombiner(), -// numPartitions = numVPart)( -// utils.computeWithCombiner(numVertices, epsilon, numIter)) - -// println("Total rank: " + result.map{ case (id, r) => r.value }.reduce(_+_) ) -// if (!outFname.isEmpty) { -// println("Saving pageranks of pages to " + outFname) -// result.map{ case (id, r) => id + "\t" + r.value }.saveAsTextFile(outFname) -// } -// println("Runtime: " + ((System.currentTimeMillis - startTime)/1000.0) + " seconds") -// sc.stop() -// } -// } -- cgit v1.2.3 From 37611e57f6ca1768529e4038198de56f85f90665 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 10 Jan 2014 12:37:03 -0800 Subject: Improve docs for EdgeRDD, EdgeTriplet, and GraphLab --- .../scala/org/apache/spark/graphx/EdgeRDD.scala | 12 +++----- .../org/apache/spark/graphx/EdgeTriplet.scala | 13 +++------ .../scala/org/apache/spark/graphx/GraphLab.scala | 34 ++++++++++------------ 3 files changed, 24 insertions(+), 35 deletions(-) (limited to 'graphx') 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 61228e9628..05d3dbe337 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala @@ -9,7 +9,7 @@ import org.apache.spark.storage.StorageLevel /** * `EdgeRDD[ED]` extends `RDD[Edge[ED]]` by storing the edges in columnar format on each partition - * for performance. It is constructed using [[org.apache.spark.graphx.impl.EdgePartitionBuilder]]. + * for performance. It is constructed using [[impl.EdgePartitionBuilder]]. */ class EdgeRDD[@specialized ED: ClassTag]( val partitionsRDD: RDD[(PartitionID, EdgePartition[ED])]) @@ -20,9 +20,9 @@ class EdgeRDD[@specialized ED: ClassTag]( override protected def getPartitions: Array[Partition] = partitionsRDD.partitions /** - * If partitionsRDD already has a partitioner, use it. Otherwise assume that the PartitionIDs in - * partitionsRDD correspond to the actual partitions and create a new partitioner that allows - * co-partitioning with partitionsRDD. + * If `partitionsRDD` already has a partitioner, use it. Otherwise assume that the + * [[PartitionID]]s in `partitionsRDD` correspond to the actual partitions and create a new + * partitioner that allows co-partitioning with `partitionsRDD`. */ override val partitioner = partitionsRDD.partitioner.orElse(Some(Partitioner.defaultPartitioner(partitionsRDD))) @@ -33,9 +33,6 @@ class EdgeRDD[@specialized ED: ClassTag]( override def collect(): Array[Edge[ED]] = this.map(_.copy()).collect() - /** - * Caching a VertexRDD causes the index and values to be cached separately. - */ override def persist(newLevel: StorageLevel): EdgeRDD[ED] = { partitionsRDD.persist(newLevel) this @@ -76,5 +73,4 @@ class EdgeRDD[@specialized ED: ClassTag]( 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 5e2528925f..057d63a0ac 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/EdgeTriplet.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeTriplet.scala @@ -3,15 +3,10 @@ package org.apache.spark.graphx import org.apache.spark.graphx.impl.VertexPartition /** - * An edge triplet represents two vertices and edge along with their - * attributes. + * An edge triplet represents an edge along with the vertex attributes of its neighboring vertices. * * @tparam VD the type of the vertex attribute. * @tparam ED the type of the edge attribute - * - * @todo specialize edge triplet for basic types, though when I last - * tried specializing I got a warning about inherenting from a type - * that is not a trait. */ class EdgeTriplet[VD, ED] extends Edge[ED] { /** @@ -37,8 +32,8 @@ class EdgeTriplet[VD, ED] extends Edge[ED] { /** * Given one vertex in the edge return the other vertex. * - * @param vid the id one of the two vertices on the edge. - * @return the attribute for the other vertex on the edge. + * @param vid the id one of the two vertices on the edge + * @return the attribute for the other vertex on the edge */ def otherVertexAttr(vid: VertexID): VD = if (srcId == vid) dstAttr else { assert(dstId == vid); srcAttr } @@ -47,7 +42,7 @@ class EdgeTriplet[VD, ED] extends Edge[ED] { * Get the vertex object for the given vertex in the edge. * * @param vid the id of one of the two vertices on the edge - * @return the attr for the vertex with that id. + * @return the attr for the vertex with that id */ def vertexAttr(vid: VertexID): VD = if (srcId == vid) srcAttr else { assert(dstId == vid); dstAttr } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphLab.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphLab.scala index 7efc69c64e..016bab5d02 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphLab.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphLab.scala @@ -7,35 +7,33 @@ import scala.collection.JavaConversions._ import org.apache.spark.rdd.RDD /** - * This object implements the GraphLab gather-apply-scatter api. + * Implements the GraphLab gather-apply-scatter API. */ object GraphLab extends Logging { /** - * Execute the GraphLab Gather-Apply-Scatter API + * Executes the GraphLab Gather-Apply-Scatter API. * - * @todo finish documenting GraphLab Gather-Apply-Scatter API - * - * @param graph The graph on which to execute the GraphLab API - * @param gatherFunc The gather function is executed on each edge triplet - * adjacent to a vertex and returns an accumulator which + * @param graph the graph on which to execute the GraphLab API + * @param gatherFunc executed on each edge triplet + * adjacent to a vertex. Returns an accumulator which * is then merged using the merge function. - * @param mergeFunc An accumulative associative operation on the result of + * @param mergeFunc an accumulative associative operation on the result of * the gather type. - * @param applyFunc Takes a vertex and the final result of the merge operations + * @param applyFunc takes a vertex and the final result of the merge operations * on the adjacent edges and returns a new vertex value. - * @param scatterFunc Executed after the apply function the scatter function takes + * @param scatterFunc executed after the apply function. Takes * a triplet and signals whether the neighboring vertex program * must be recomputed. - * @param startVertices predicate to determine which vertices to start the computation on. - * these will be the active vertices in the first iteration. - * @param numIter The maximum number of iterations to run. - * @param gatherDirection The direction of edges to consider during the gather phase - * @param scatterDirection The direction of edges to consider during the scatter phase + * @param startVertices a predicate to determine which vertices to start the computation on. + * These will be the active vertices in the first iteration. + * @param numIter the maximum number of iterations to run + * @param gatherDirection the direction of edges to consider during the gather phase + * @param scatterDirection the direction of edges to consider during the scatter phase * - * @tparam VD The graph vertex attribute type - * @tparam ED The graph edge attribute type - * @tparam A The type accumulated during the gather phase + * @tparam VD the graph vertex attribute type + * @tparam ED the graph edge attribute type + * @tparam A the type accumulated during the gather phase * @return the resulting graph after the algorithm converges */ def apply[VD: ClassTag, ED: ClassTag, A: ClassTag] -- cgit v1.2.3 From 9454fa1f6c714a720197a6db04a6c369e425e507 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 10 Jan 2014 12:37:20 -0800 Subject: Remove duplicate method in GraphLoader and improve docs --- .../org/apache/spark/graphx/GraphLoader.scala | 63 +++++----------------- 1 file changed, 13 insertions(+), 50 deletions(-) (limited to 'graphx') 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 473cfb18cf..3c06a403ea 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala @@ -8,51 +8,20 @@ import org.apache.spark.{Logging, SparkContext} import org.apache.spark.graphx.impl.{EdgePartition, GraphImpl} import org.apache.spark.util.collection.PrimitiveVector - +/** + * Provides utilities for loading [[Graph]]s from files. + */ object GraphLoader extends Logging { /** - * Load an edge list from file initializing the Graph - * - * @tparam ED the type of the edge data of the resulting Graph - * - * @param sc the SparkContext used to construct RDDs - * @param path the path to the text file containing the edge list - * @param edgeParser a function that takes an array of strings and - * returns an ED object - * @param minEdgePartitions the number of partitions for the - * the Edge RDD + * Loads a graph from an edge list formatted file where each line contains two integers: a source + * id and a target id. Skips lines that begin with `#`. * - */ - def textFile[ED: ClassTag]( - sc: SparkContext, - path: String, - edgeParser: Array[String] => ED, - minEdgePartitions: Int = 1): - Graph[Int, ED] = { - // Parse the edge data table - val edges = sc.textFile(path, minEdgePartitions).mapPartitions( iter => - iter.filter(line => !line.isEmpty && line(0) != '#').map { line => - val lineArray = line.split("\\s+") - if(lineArray.length < 2) { - println("Invalid line: " + line) - assert(false) - } - val source = lineArray(0).trim.toLong - val target = lineArray(1).trim.toLong - val tail = lineArray.drop(2) - val edata = edgeParser(tail) - Edge(source, target, edata) - }) - val defaultVertexAttr = 1 - Graph.fromEdges(edges, defaultVertexAttr) - } - - /** - * Load a graph from an edge list formatted file with each line containing - * two integers: a source Id and a target Id. + * If desired the edges can be automatically oriented in the positive + * direction (source Id < target Id) by setting `canonicalOrientation` to + * true. * - * @example A file in the following format: + * @example Loads a file in the following format: * {{{ * # Comment Line * # Source Id <\t> Target Id @@ -62,25 +31,19 @@ object GraphLoader extends Logging { * 1 8 * }}} * - * If desired the edges can be automatically oriented in the positive - * direction (source Id < target Id) by setting `canonicalOrientation` to - * true - * * @param sc - * @param path the path to the file (e.g., /Home/data/file or hdfs://file) + * @param path the path to the file (e.g., /home/data/file or hdfs://file) * @param canonicalOrientation whether to orient edges in the positive - * direction. + * direction * @param minEdgePartitions the number of partitions for the - * the Edge RDD + * the edge RDD * @tparam ED - * @return */ def edgeListFile( sc: SparkContext, path: String, canonicalOrientation: Boolean = false, - minEdgePartitions: Int = 1): - Graph[Int, Int] = { + minEdgePartitions: Int = 1): Graph[Int, Int] = { val startTime = System.currentTimeMillis // Parse the edge data table directly into edge partitions -- cgit v1.2.3 From eb4b46f8d1f5c5593ec327ccd8f816f784cce6b1 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 10 Jan 2014 12:46:00 -0800 Subject: Improve docs for GraphOps --- .../scala/org/apache/spark/graphx/GraphOps.scala | 78 +++++++--------------- 1 file changed, 25 insertions(+), 53 deletions(-) (limited to 'graphx') 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 cacfcb1c90..69f27601ce 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala @@ -8,57 +8,47 @@ import org.apache.spark.SparkException /** - * `GraphOps` contains additional functionality (syntatic sugar) for - * the graph type and is implicitly constructed for each Graph object. - * All operations in `GraphOps` are expressed in terms of the - * efficient GraphX API. + * Contains additional functionality for [[Graph]]. All operations are expressed in terms of the + * efficient GraphX API. This class is implicitly constructed for each Graph object. * * @tparam VD the vertex attribute type * @tparam ED the edge attribute type - * */ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) { - /** - * Compute the number of edges in the graph. - */ + /** The number of edges in the graph. */ lazy val numEdges: Long = graph.edges.count() - /** - * Compute the number of vertices in the graph. - */ + /** The number of vertices in the graph. */ lazy val numVertices: Long = graph.vertices.count() /** - * Compute the in-degree of each vertex in the Graph returning an - * RDD. - * @note Vertices with no in edges are not returned in the resulting RDD. + * The in-degree of each vertex in the graph. + * @note Vertices with no in-edges are not returned in the resulting RDD. */ lazy val inDegrees: VertexRDD[Int] = degreesRDD(EdgeDirection.In) /** - * Compute the out-degree of each vertex in the Graph returning an RDD. - * @note Vertices with no out edges are not returned in the resulting RDD. + * The out-degree of each vertex in the graph. + * @note Vertices with no out-edges are not returned in the resulting RDD. */ lazy val outDegrees: VertexRDD[Int] = degreesRDD(EdgeDirection.Out) /** - * Compute the degrees of each vertex in the Graph returning an RDD. - * @note Vertices with no edges are not returned in the resulting - * RDD. + * The degree of each vertex in the graph. + * @note Vertices with no edges are not returned in the resulting RDD. */ lazy val degrees: VertexRDD[Int] = degreesRDD(EdgeDirection.Both) /** - * Compute the neighboring vertex degrees. + * Computes the neighboring vertex degrees. * - * @param edgeDirection the direction along which to collect - * neighboring vertex attributes. + * @param edgeDirection the direction along which to collect neighboring vertex attributes */ private def degreesRDD(edgeDirection: EdgeDirection): VertexRDD[Int] = { if (edgeDirection == EdgeDirection.In) { @@ -70,32 +60,20 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) { } } - /** - * This function is used to compute a statistic for the neighborhood - * of each vertex and returns a value for all vertices (including - * those without neighbors). - * - * @note Because the a default value is provided all vertices will - * have a corresponding entry in the returned RDD. + * Computes a statistic for the neighborhood of each vertex. * - * @param mapFunc the function applied to each edge adjacent to each - * vertex. The mapFunc can optionally return None in which case it - * does not contribute to the final sum. - * @param reduceFunc the function used to merge the results of each - * map operation. - * @param default the default value to use for each vertex if it has - * no neighbors or the map function repeatedly evaluates to none - * @param direction the direction of edges to consider (e.g., In, - * Out, Both). - * @tparam VD2 The returned type of the aggregation operation. + * @param mapFunc the function applied to each edge adjacent to each vertex. The mapFunc can + * optionally return `None`, in which case it does not contribute to the final sum. + * @param reduceFunc the function used to merge the results of each map operation + * @param direction the direction of edges to consider (e.g., In, Out, Both). + * @tparam A the aggregation type * - * @return A Spark.RDD containing tuples of vertex identifiers and - * their resulting value. There will be exactly one entry for ever - * vertex in the original graph. + * @return an RDD containing tuples of vertex identifiers and + * their resulting value. Vertices with no neighbors will not appear in the RDD. * * @example We can use this function to compute the average follower - * age for each user + * age for each user: * * {{{ * val graph: Graph[Int,Int] = loadGraph() @@ -107,16 +85,12 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) { * EdgeDirection.In) * .mapValues{ case (sum,followers) => sum.toDouble / followers} * }}} - * - * @todo Should this return a graph with the new vertex values? - * */ def aggregateNeighbors[A: ClassTag]( mapFunc: (VertexID, EdgeTriplet[VD, ED]) => Option[A], reduceFunc: (A, A) => A, dir: EdgeDirection) : VertexRDD[A] = { - // Define a new map function over edge triplets val mf = (et: EdgeTriplet[VD,ED]) => { // Compute the message to the dst vertex @@ -143,15 +117,14 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) { /** - * Return the Ids of the neighboring vertices. + * Collect the neighbor vertex ids for each vertex. * * @param edgeDirection the direction along which to collect * neighboring vertices * - * @return the vertex set of neighboring ids for each vertex. + * @return the set of neighboring ids for each vertex */ - def collectNeighborIds(edgeDirection: EdgeDirection) : - VertexRDD[Array[VertexID]] = { + def collectNeighborIds(edgeDirection: EdgeDirection): VertexRDD[Array[VertexID]] = { val nbrs = if (edgeDirection == EdgeDirection.Both) { graph.mapReduceTriplets[Array[VertexID]]( @@ -185,8 +158,7 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) { * @param edgeDirection the direction along which to collect * neighboring vertices * - * @return the vertex set of neighboring vertex attributes for each - * vertex. + * @return the vertex set of neighboring vertex attributes for each vertex */ def collectNeighbors(edgeDirection: EdgeDirection) : VertexRDD[ Array[(VertexID, VD)] ] = { -- cgit v1.2.3 From 7bda99778566b48355bc17f59a2e941eda70ad85 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 10 Jan 2014 13:00:28 -0800 Subject: Improve docs for PartitionStrategy --- .../apache/spark/graphx/PartitionStrategy.scala | 58 ++++++++++++---------- 1 file changed, 31 insertions(+), 27 deletions(-) (limited to 'graphx') 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 5e80a535f1..fc7635a033 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala @@ -1,16 +1,22 @@ package org.apache.spark.graphx - +/** + * Represents the way edges are assigned to edge partitions based on their source and destination + * vertex IDs. + */ sealed trait PartitionStrategy extends Serializable { def getPartition(src: VertexID, dst: VertexID, numParts: PartitionID): PartitionID } /** - * This function implements a classic 2D-Partitioning of a sparse matrix. + * Assigns edges to partitions using a 2D partitioning of the sparse edge adjacency matrix, + * guaranteeing a `2 * sqrt(numParts)` bound on vertex replication. + * * Suppose we have a graph with 11 vertices that we want to partition * over 9 machines. We can use the following sparse matrix representation: * + *
  *       __________________________________
  *  v0   | P0 *     | P1       | P2    *  |
  *  v1   |  ****    |  *       |          |
@@ -27,28 +33,23 @@ sealed trait PartitionStrategy extends Serializable {
  *  v10  |       *  |      **  |  *  *    |
  *  v11  | * <-E    |  ***     |       ** |
  *       ----------------------------------
+ * 
* - * The edge denoted by E connects v11 with v1 and is assigned to - * processor P6. To get the processor number we divide the matrix - * into sqrt(numProc) by sqrt(numProc) blocks. Notice that edges - * adjacent to v11 can only be in the first colum of - * blocks (P0, P3, P6) or the last row of blocks (P6, P7, P8). - * As a consequence we can guarantee that v11 will need to be - * replicated to at most 2 * sqrt(numProc) machines. - * - * Notice that P0 has many edges and as a consequence this - * partitioning would lead to poor work balance. To improve - * balance we first multiply each vertex id by a large prime - * to effectively shuffle the vertex locations. - * - * One of the limitations of this approach is that the number of - * machines must either be a perfect square. We partially address - * this limitation by computing the machine assignment to the next - * largest perfect square and then mapping back down to the actual - * number of machines. Unfortunately, this can also lead to work - * imbalance and so it is suggested that a perfect square is used. + * The edge denoted by `E` connects `v11` with `v1` and is assigned to processor `P6`. To get the + * processor number we divide the matrix into `sqrt(numParts)` by `sqrt(numParts)` blocks. Notice + * that edges adjacent to `v11` can only be in the first column of blocks `(P0, P3, P6)` or the last + * row of blocks `(P6, P7, P8)`. As a consequence we can guarantee that `v11` will need to be + * replicated to at most `2 * sqrt(numParts)` machines. * + * Notice that `P0` has many edges and as a consequence this partitioning would lead to poor work + * balance. To improve balance we first multiply each vertex id by a large prime to shuffle the + * vertex locations. * + * One of the limitations of this approach is that the number of machines must either be a perfect + * square. We partially address this limitation by computing the machine assignment to the next + * largest perfect square and then mapping back down to the actual number of machines. + * Unfortunately, this can also lead to work imbalance and so it is suggested that a perfect square + * is used. */ case object EdgePartition2D extends PartitionStrategy { override def getPartition(src: VertexID, dst: VertexID, numParts: PartitionID): PartitionID = { @@ -60,7 +61,10 @@ case object EdgePartition2D extends PartitionStrategy { } } - +/** + * Assigns edges to partitions using only the source vertex ID, colocating edges with the same + * source. + */ case object EdgePartition1D extends PartitionStrategy { override def getPartition(src: VertexID, dst: VertexID, numParts: PartitionID): PartitionID = { val mixingPrime: VertexID = 1125899906842597L @@ -70,8 +74,8 @@ case object EdgePartition1D extends PartitionStrategy { /** - * Assign edges to an aribtrary machine corresponding to a - * random vertex cut. + * Assigns edges to partitions by hashing the source and destination vertex IDs, resulting in a + * random vertex cut that colocates all same-direction edges between two vertices. */ case object RandomVertexCut extends PartitionStrategy { override def getPartition(src: VertexID, dst: VertexID, numParts: PartitionID): PartitionID = { @@ -81,9 +85,9 @@ case object RandomVertexCut extends PartitionStrategy { /** - * Assign edges to an arbitrary machine corresponding to a random vertex cut. This - * function ensures that edges of opposite direction between the same two vertices - * will end up on the same partition. + * Assigns edges to partitions by hashing the source and destination vertex IDs in a canonical + * direction, resulting in a random vertex cut that colocates all edges between two vertices, + * regardless of direction. */ case object CanonicalRandomVertexCut extends PartitionStrategy { override def getPartition(src: VertexID, dst: VertexID, numParts: PartitionID): PartitionID = { -- cgit v1.2.3 From 9e48af6dba5817b7ac35a5bfa5ffe8825960c4c8 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 10 Jan 2014 15:22:57 -0800 Subject: Remove unused HashUtils class --- .../org/apache/spark/graphx/util/HashUtils.scala | 21 --------------------- 1 file changed, 21 deletions(-) delete mode 100644 graphx/src/main/scala/org/apache/spark/graphx/util/HashUtils.scala (limited to 'graphx') diff --git a/graphx/src/main/scala/org/apache/spark/graphx/util/HashUtils.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/HashUtils.scala deleted file mode 100644 index 7a79d33350..0000000000 --- a/graphx/src/main/scala/org/apache/spark/graphx/util/HashUtils.scala +++ /dev/null @@ -1,21 +0,0 @@ -package org.apache.spark.graphx.util - - -object HashUtils { - - /** - * Compute a 64-bit hash value for the given string. - * See http://stackoverflow.com/questions/1660501/what-is-a-good-64bit-hash-function-in-java-for-textual-strings - */ - def hash(str: String): Long = { - var h = 1125899906842597L - val len = str.length - var i = 0 - - while (i < len) { - h = 31 * h + str(i) - i += 1 - } - h - } -} -- cgit v1.2.3 From 11dd35c28be9542d00ddb75660873525b2e22b43 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 10 Jan 2014 15:23:32 -0800 Subject: Clean up GraphGenerators --- .../apache/spark/graphx/util/GraphGenerators.scala | 75 +++------------------- 1 file changed, 9 insertions(+), 66 deletions(-) (limited to 'graphx') diff --git a/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala index 57117241ad..e0fd9b972c 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala @@ -15,9 +15,6 @@ import org.apache.spark.graphx.Graph import org.apache.spark.graphx.Edge import org.apache.spark.graphx.impl.GraphImpl -/** - * @todo cleanup and modularize code - */ object GraphGenerators { val RMATa = 0.45 @@ -25,70 +22,20 @@ object GraphGenerators { val RMATc = 0.15 val RMATd = 0.25 - def main(args: Array[String]) { - - - val serializer = "org.apache.spark.serializer.KryoSerializer" - System.setProperty("spark.serializer", serializer) - //System.setProperty("spark.shuffle.compress", "false") - System.setProperty("spark.kryo.registrator", "spark.graphx.GraphKryoRegistrator") - val host = "local[4]" - val sc = new SparkContext(host, "Lognormal graph generator") - - val lnGraph = logNormalGraph(sc, 10000) - - val rmat = rmatGraph(sc, 1000, 3000) - - //for (v <- lnGraph.vertices) { - // println(v.id + ":\t" + v.data) - //} - - val times = 100000 - //val nums = (1 to times).flatMap { n => List(sampleLogNormal(4.0, 1.3, times)) }.toList - //val avg = nums.sum / nums.length - //val sumSquares = nums.foldLeft(0.0) {(total, next) => - // (total + math.pow((next - avg), 2)) } - //val stdev = math.sqrt(sumSquares/(nums.length - 1)) - - //println("avg: " + avg + "+-" + stdev) - - - //for (i <- 1 to 1000) { - // println(sampleLogNormal(4.0, 1.3, 1000)) - //} - - sc.stop() - - } - - // Right now it just generates a bunch of edges where // the edge data is the weight (default 1) def logNormalGraph(sc: SparkContext, numVertices: Int): Graph[Int, Int] = { // based on Pregel settings val mu = 4 val sigma = 1.3 - //val vertsAndEdges = (0 until numVertices).flatMap { src => { val vertices: RDD[(VertexID, Int)] = sc.parallelize(0 until numVertices).map{ src => (src, sampleLogNormal(mu, sigma, numVertices)) } - val edges = vertices.flatMap{ v => generateRandomEdges(v._1.toInt, v._2, numVertices) } - Graph(vertices, edges, 0) - //println("Vertices:") - //for (v <- vertices) { - // println(v.id) - //} - - //println("Edges") - //for (e <- edges) { - // println(e.src, e.dst, e.data) - //} - } @@ -106,14 +53,10 @@ object GraphGenerators { /** - * Randomly samples from a log normal distribution - * whose corresponding normal distribution has the - * the given mean and standard deviation. It uses - * the formula X = exp(m+s*Z) where m, s are the - * mean, standard deviation of the lognormal distribution - * and Z~N(0, 1). In this function, - * m = e^(mu+sigma^2/2) and - * s = sqrt[(e^(sigma^2) - 1)(e^(2*mu+sigma^2))]. + * Randomly samples from a log normal distribution whose corresponding normal distribution has the + * the given mean and standard deviation. It uses the formula `X = exp(m+s*Z)` where `m`, `s` are + * the mean, standard deviation of the lognormal distribution and `Z ~ N(0, 1)`. In this function, + * `m = e^(mu+sigma^2/2)` and `s = sqrt[(e^(sigma^2) - 1)(e^(2*mu+sigma^2))]`. * * @param mu the mean of the normal distribution * @param sigma the standard deviation of the normal distribution @@ -128,8 +71,7 @@ object GraphGenerators { while (X >= maxVal) { val Z = rand.nextGaussian() - //X = math.exp((m + s*Z)) - X = math.exp((mu + sigma*Z)) + X = math.exp(mu + sigma*Z) } math.round(X.toFloat) } @@ -155,7 +97,6 @@ object GraphGenerators { } def outDegreeFromEdges[ED: ClassTag](edges: RDD[Edge[ED]]): Graph[Int, ED] = { - val vertices = edges.flatMap { edge => List((edge.srcId, 1)) } .reduceByKey(_ + _) .map{ case (vid, degree) => (vid, degree) } @@ -180,6 +121,7 @@ object GraphGenerators { * until it picks a single cell. The naming conventions in this paper match * those of the R-MAT paper. There are a power of 2 number of nodes in the graph. * The adjacency matrix looks like: + *
    *
    *          dst ->
    * (x,y) ***************  _
@@ -191,6 +133,7 @@ object GraphGenerators {
    *       |   c  |   d  |  |
    *       |      |      |  |
    *       ***************  -
+   * 
* * where this represents the subquadrant of the adj matrix currently being * subdivided. (x,y) represent the upper left hand corner of the subquadrant, @@ -198,12 +141,12 @@ object GraphGenerators { * * After choosing the next level subquadrant, we get the resulting sets * of parameters: + * {{{ * quad = a, x'=x, y'=y, T'=T/2 * quad = b, x'=x+T/2, y'=y, T'=T/2 * quad = c, x'=x, y'=y+T/2, T'=T/2 * quad = d, x'=x+T/2, y'=y+T/2, T'=T/2 - * - * @param src is the + * }}} */ @tailrec def chooseCell(x: Int, y: Int, t: Int): (Int, Int) = { -- cgit v1.2.3 From 57d7487d3da19df04de52235812fe7c8c24cc259 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 10 Jan 2014 15:42:44 -0800 Subject: Improve docs for VertexRDD --- .../scala/org/apache/spark/graphx/GraphOps.scala | 2 +- .../scala/org/apache/spark/graphx/VertexRDD.scala | 152 +++++++++------------ .../spark/graphx/impl/EdgePartitionBuilder.scala | 3 - .../spark/graphx/impl/EdgeTripletIterator.scala | 1 - 4 files changed, 62 insertions(+), 96 deletions(-) (limited to 'graphx') 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 69f27601ce..0121cb1449 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala @@ -232,7 +232,7 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) { * {{{ * graph.filter( * graph => { - * val degrees: VertexSetRDD[Int] = graph.outDegrees + * val degrees: VertexRDD[Int] = graph.outDegrees * graph.outerJoinVertices(degrees) {(vid, data, deg) => deg.getOrElse(0)} * }, * vpred = (vid: VertexID, deg:Int) => deg > 0 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 971e2615d4..3ef9d6e9cf 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala @@ -27,11 +27,10 @@ import org.apache.spark.storage.StorageLevel import org.apache.spark.graphx.impl.MsgRDDFunctions import org.apache.spark.graphx.impl.VertexPartition - /** - * A `VertexRDD[VD]` extends the `RDD[(VertexID, VD)]` by ensuring that there is - * only one entry for each vertex and by pre-indexing the entries for fast, - * efficient joins. + * A `VertexRDD[VD]` extends the `RDD[(VertexID, VD)]` by ensuring that there is only one entry for + * each vertex and by pre-indexing the entries for fast, efficient joins. Two VertexRDDs with the + * same index can be joined efficiently. * * @tparam VD the vertex attribute associated with each vertex in the set. * @@ -46,7 +45,7 @@ import org.apache.spark.graphx.impl.VertexPartition * val vset2 = VertexRDD(someData, reduceFunc) * // Finally we can use the VertexRDD to index another dataset * val otherData: RDD[(VertexID, OtherType)] = loadData(otherFile) - * val vset3 = VertexRDD(otherData, vset.index) + * val vset3 = vset2.innerJoin(otherData) { (vid, a, b) => b } * // Now we can construct very fast joins between the two sets * val vset4: VertexRDD[(SomeType, OtherType)] = vset.leftJoin(vset3) * }}} @@ -61,32 +60,18 @@ class VertexRDD[@specialized VD: ClassTag]( partitionsRDD.setName("VertexRDD") /** - * Construct a new VertexRDD that is indexed by only the keys in the RDD. - * The resulting VertexRDD will be based on a different index and can - * no longer be quickly joined with this RDD. + * 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 RDD. */ def reindex(): VertexRDD[VD] = new VertexRDD(partitionsRDD.map(_.reindex())) - /** - * The partitioner is defined by the index. - */ override val partitioner = partitionsRDD.partitioner - /** - * The actual partitions are defined by the tuples. - */ override protected def getPartitions: Array[Partition] = partitionsRDD.partitions - /** - * The preferred locations are computed based on the preferred - * locations of the tuples. - */ override protected def getPreferredLocations(s: Partition): Seq[String] = partitionsRDD.preferredLocations(s) - /** - * Caching a VertexRDD causes the index and values to be cached separately. - */ override def persist(newLevel: StorageLevel): VertexRDD[VD] = { partitionsRDD.persist(newLevel) this @@ -103,20 +88,20 @@ class VertexRDD[@specialized VD: ClassTag]( this } - /** Return the number of vertices in this set. */ + /** The number of vertices in the RDD. */ override def count(): Long = { partitionsRDD.map(_.size).reduce(_ + _) } /** - * Provide the `RDD[(VertexID, VD)]` equivalent output. + * 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 } /** - * Return a new VertexRDD by applying a function to each VertexPartition of this RDD. + * Applies a function to each [[impl.VertexPartition]] of this RDD and returns a new VertexRDD. */ def mapVertexPartitions[VD2: ClassTag](f: VertexPartition[VD] => VertexPartition[VD2]) : VertexRDD[VD2] = { @@ -126,51 +111,43 @@ class VertexRDD[@specialized VD: ClassTag]( /** - * Restrict the vertex set to the set of vertices satisfying the - * given predicate. - * - * @param pred the user defined predicate, which takes a tuple to conform to - * the RDD[(VertexID, VD)] interface + * Restricts the vertex set to the set of vertices satisfying the given predicate. This operation + * preserves the index for efficient joins with the original RDD, and it sets bits in the bitmask + * rather than allocating new memory. * - * @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. + * @param pred the user defined predicate, which takes a tuple to conform to the + * `RDD[(VertexID, VD)]` interface */ override def filter(pred: Tuple2[VertexID, VD] => Boolean): VertexRDD[VD] = this.mapVertexPartitions(_.filter(Function.untupled(pred))) /** - * Pass each vertex attribute through a map function and retain the - * original RDD's partitioning and index. + * Maps each vertex attribute, preserving the index. * * @tparam VD2 the type returned by the map function * * @param f the function applied to each value in the RDD - * @return a new VertexRDD with values obtained by applying `f` to - * each of the entries in the original VertexRDD. The resulting - * VertexRDD retains the same index. + * @return a new VertexRDD with values obtained by applying `f` to each of the entries in the + * original VertexRDD */ def mapValues[VD2: ClassTag](f: VD => VD2): VertexRDD[VD2] = this.mapVertexPartitions(_.map((vid, attr) => f(attr))) /** - * Pass each vertex attribute through a map function and retain the - * original RDD's partitioning and index. + * Maps each vertex attribute, additionally supplying the vertex ID. * * @tparam VD2 the type returned by the map function * - * @param f the function applied to each value in the RDD - * @return a new VertexRDD with values obtained by applying `f` to - * each of the entries in the original VertexRDD. The resulting - * VertexRDD retains the same index. + * @param f the function applied to each ID-value pair in the RDD + * @return a new VertexRDD with values obtained by applying `f` to each of the entries in the + * original VertexRDD. The resulting VertexRDD retains the same index. */ def mapValues[VD2: ClassTag](f: (VertexID, VD) => VD2): VertexRDD[VD2] = this.mapVertexPartitions(_.map(f)) /** - * Hides vertices that are the same between this and other. For vertices that are different, keeps - * the values from `other`. + * Hides vertices that are the same between `this` and `other`. For vertices that are different, + * keeps the values from `other`. */ def diff(other: VertexRDD[VD]): VertexRDD[VD] = { val newPartitionsRDD = partitionsRDD.zipPartitions( @@ -184,22 +161,17 @@ class VertexRDD[@specialized VD: ClassTag]( } /** - * Left join this VertexSet with another VertexSet which has the - * same Index. This function will fail if both VertexSets do not - * share the same index. The resulting vertex set contains an entry - * for each vertex in this set. If the other VertexSet is missing - * any vertex in this VertexSet then a `None` attribute is generated - * - * @tparam VD2 the attribute type of the other VertexSet - * @tparam VD3 the attribute type of the resulting VertexSet + * Left joins this RDD with another VertexRDD with the same index. This function will fail if both + * VertexRDDs do not share the same index. The resulting vertex set contains an entry for each + * vertex in `this`. If `other` is missing any vertex in this VertexRDD, `f` is passed `None`. * - * @param other the other VertexSet with which to join. - * @param f the function mapping a vertex id and its attributes in - * this and the other vertex set to a new vertex attribute. - * @return a VertexRDD containing all the vertices in this - * VertexSet with `None` attributes used for Vertices missing in the - * other VertexSet. + * @tparam VD2 the attribute type of the other VertexRDD + * @tparam VD3 the attribute type of the resulting VertexRDD * + * @param other the other VertexRDD with which to join. + * @param f the function mapping a vertex id and its attributes in this and the other vertex set + * to a new vertex attribute. + * @return a VertexRDD containing the results of `f` */ def leftZipJoin[VD2: ClassTag, VD3: ClassTag] (other: VertexRDD[VD2])(f: (VertexID, VD, Option[VD2]) => VD3): VertexRDD[VD3] = { @@ -214,29 +186,25 @@ class VertexRDD[@specialized VD: ClassTag]( } /** - * Left join this VertexRDD with an RDD containing vertex attribute - * pairs. If the other RDD is backed by a VertexRDD with the same - * index than the efficient leftZipJoin implementation is used. The - * resulting vertex set contains an entry for each vertex in this - * set. If the other VertexRDD is missing any vertex in this - * VertexRDD then a `None` attribute is generated. - * - * If there are duplicates, the vertex is picked at random. + * Left joins this VertexRDD with an RDD containing vertex attribute pairs. If the other RDD is + * backed by a VertexRDD with the same index then the efficient [[leftZipJoin]] implementation is + * used. The resulting vertex set contains an entry for each vertex in this set. If `other` is + * missing any vertex in this VertexRDD, `f` is passed `None`. If there are duplicates, the vertex + * is picked arbitrarily. * * @tparam VD2 the attribute type of the other VertexRDD * @tparam VD3 the attribute type of the resulting VertexRDD * - * @param other the other VertexRDD with which to join. - * @param f the function mapping a vertex id and its attributes in - * this and the other vertex set to a new vertex attribute. - * @return a VertexRDD containing all the vertices in this - * VertexRDD with the attribute emitted by f. + * @param other the other VertexRDD with which to join + * @param f the function mapping a vertex id and its attributes in this and the other vertex set + * to a new vertex attribute. + * @return a VertexRDD containing all the vertices in this VertexRDD with the attributes emitted + * by `f`. */ def leftJoin[VD2: ClassTag, VD3: ClassTag] (other: RDD[(VertexID, VD2)]) (f: (VertexID, VD, Option[VD2]) => VD3) - : VertexRDD[VD3] = - { + : VertexRDD[VD3] = { // Test if the other vertex is a VertexRDD to choose the optimal join strategy. // If the other set is a VertexRDD then we use the much more efficient leftZipJoin other match { @@ -255,8 +223,8 @@ class VertexRDD[@specialized VD: ClassTag]( } /** - * Same effect as leftJoin(other) { (vid, a, bOpt) => bOpt.getOrElse(a) }, but `this` and `other` - * must have the same index. + * Same effect as `leftJoin(other) { (vid, a, bOpt) => bOpt.getOrElse(a) }`, but `this` and + * `other` must have the same index. */ def innerZipJoin[U: ClassTag, VD2: ClassTag](other: VertexRDD[U]) (f: (VertexID, VD, U) => VD2): VertexRDD[VD2] = { @@ -271,8 +239,9 @@ class VertexRDD[@specialized VD: ClassTag]( } /** - * Replace vertices with corresponding vertices in `other`, and drop vertices without a - * corresponding vertex in `other`. + * Inner joins this VertexRDD with an RDD containing vertex attribute pairs. If the other RDD is + * backed by a VertexRDD with the same index then the efficient [[innerZipJoin]] implementation is + * used. */ def innerJoin[U: ClassTag, VD2: ClassTag](other: RDD[(VertexID, U)]) (f: (VertexID, VD, U) => VD2): VertexRDD[VD2] = { @@ -294,12 +263,11 @@ class VertexRDD[@specialized VD: ClassTag]( } /** - * Aggregate messages with the same ids using `reduceFunc`, returning a VertexRDD that is - * co-indexed with this one. + * Aggregates vertices in `message` that have the same ids using `reduceFunc`, returning a + * VertexRDD co-indexed with `this`. */ def aggregateUsingIndex[VD2: ClassTag]( - messages: RDD[(VertexID, VD2)], reduceFunc: (VD2, VD2) => VD2): VertexRDD[VD2] = - { + messages: RDD[(VertexID, VD2)], reduceFunc: (VD2, VD2) => VD2): VertexRDD[VD2] = { val shuffled = MsgRDDFunctions.partitionForAggregation(messages, this.partitioner.get) val parts = partitionsRDD.zipPartitions(shuffled, true) { (thisIter, msgIter) => val vertexPartition: VertexPartition[VD] = thisIter.next() @@ -312,12 +280,12 @@ class VertexRDD[@specialized VD: ClassTag]( /** - * The VertexRDD singleton is used to construct VertexRDDs + * The VertexRDD singleton is used to construct VertexRDDs. */ object VertexRDD { /** - * Construct a vertex set from an RDD of vertex-attribute pairs. + * Construct a `VertexRDD` from an RDD of vertex-attribute pairs. * Duplicate entries are removed arbitrarily. * * @tparam VD the vertex attribute type @@ -336,16 +304,15 @@ object VertexRDD { } /** - * Construct a vertex set from an RDD of vertex-attribute pairs. - * Duplicate entries are merged using mergeFunc. + * Constructs a `VertexRDD` from an RDD of vertex-attribute pairs, merging duplicates using + * `mergeFunc`. * * @tparam VD the vertex attribute type * * @param rdd the collection of vertex-attribute pairs * @param mergeFunc the associative, commutative merge function. */ - def apply[VD: ClassTag](rdd: RDD[(VertexID, VD)], mergeFunc: (VD, VD) => VD): VertexRDD[VD] = - { + 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)) @@ -356,9 +323,12 @@ object VertexRDD { new VertexRDD(vertexPartitions) } + /** + * Constructs a VertexRDD from the vertex IDs in `vids`, taking attributes from `rdd` and using + * `defaultVal` otherwise. + */ def apply[VD: ClassTag](vids: RDD[VertexID], rdd: RDD[(VertexID, VD)], defaultVal: VD) - : VertexRDD[VD] = - { + : VertexRDD[VD] = { VertexRDD(vids.map(vid => (vid, defaultVal))).leftJoin(rdd) { (vid, default, value) => value.getOrElse(default) } 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 d4f08497a2..ca64e9af66 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 @@ -6,10 +6,7 @@ import scala.util.Sorting import org.apache.spark.graphx._ import org.apache.spark.util.collection.{PrimitiveKeyOpenHashMap, PrimitiveVector} - -//private[graph] class EdgePartitionBuilder[@specialized(Long, Int, Double) ED: ClassTag](size: Int = 64) { - var edges = new PrimitiveVector[Edge[ED]](size) /** Add a new edge to the partition. */ 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 79fd962ffd..c5258360da 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 @@ -5,7 +5,6 @@ import scala.reflect.ClassTag import org.apache.spark.graphx._ import org.apache.spark.util.collection.PrimitiveKeyOpenHashMap - /** * The Iterator type returned when constructing edge triplets. This class technically could be * an anonymous class in GraphImpl.triplets, but we name it here explicitly so it is easier to -- cgit v1.2.3 From 1788729273261c8d9f6ec77b7fd15cab813338f8 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 10 Jan 2014 15:58:18 -0800 Subject: Move VertexIdToIndexMap into impl --- graphx/src/main/scala/org/apache/spark/graphx/impl/package.scala | 7 +++++++ graphx/src/main/scala/org/apache/spark/graphx/package.scala | 9 +-------- 2 files changed, 8 insertions(+), 8 deletions(-) create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/impl/package.scala (limited to 'graphx') diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/package.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/package.scala new file mode 100644 index 0000000000..a6bbf63888 --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/package.scala @@ -0,0 +1,7 @@ +package org.apache.spark.graphx + +import org.apache.spark.util.collection.OpenHashSet + +package object impl { + type VertexIdToIndexMap = OpenHashSet[VertexID] +} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/package.scala b/graphx/src/main/scala/org/apache/spark/graphx/package.scala index 96f0d91c9b..e6877d3195 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/package.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/package.scala @@ -2,9 +2,7 @@ package org.apache.spark import org.apache.spark.util.collection.OpenHashSet - package object graphx { - type VertexID = Long // TODO: Consider using Char. @@ -12,11 +10,6 @@ package object graphx { type VertexSet = OpenHashSet[VertexID] - // type VertexIdToIndexMap = it.unimi.dsi.fastutil.longs.Long2IntOpenHashMap - type VertexIdToIndexMap = OpenHashSet[VertexID] - - /** - * Return the default null-like value for a data type T. - */ + /** * Returns the default null-like value for a data type T. */ def nullValue[T] = null.asInstanceOf[T] } -- cgit v1.2.3 From 04c20e7f4f98b18187f2320ddd7dedce69017de7 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 10 Jan 2014 15:58:30 -0800 Subject: Minor cleanup to docs --- .../main/scala/org/apache/spark/graphx/Edge.scala | 2 +- .../org/apache/spark/graphx/EdgeDirection.scala | 3 +- .../apache/spark/graphx/GraphKryoRegistrator.scala | 4 ++- .../scala/org/apache/spark/graphx/Pregel.scala | 32 ++++++++++------------ .../scala/org/apache/spark/graphx/VertexRDD.scala | 2 +- 5 files changed, 21 insertions(+), 22 deletions(-) (limited to 'graphx') diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala b/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala index 29b46674f1..c6b62ebaca 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala @@ -3,7 +3,7 @@ package org.apache.spark.graphx /** * A single directed edge consisting of a source id, target id, - * and the data associated with the Edgee. + * and the data associated with the edge. * * @tparam ED type of the edge attribute */ diff --git a/graphx/src/main/scala/org/apache/spark/graphx/EdgeDirection.scala b/graphx/src/main/scala/org/apache/spark/graphx/EdgeDirection.scala index 785f941650..99910ab87b 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/EdgeDirection.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeDirection.scala @@ -2,8 +2,7 @@ package org.apache.spark.graphx /** - * The direction of directed edge relative to a vertex used to select - * the set of adjacent neighbors when running a neighborhood query. + * The direction of a directed edge relative to a vertex. */ sealed abstract class EdgeDirection { /** 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 f8aab951f0..681074ba10 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala @@ -7,7 +7,9 @@ import org.apache.spark.serializer.KryoRegistrator import org.apache.spark.util.collection.BitSet import org.apache.spark.util.BoundedPriorityQueue - +/** + * Registers GraphX classes with Kryo for improved performance. + */ class GraphKryoRegistrator extends KryoRegistrator { def registerClasses(kryo: Kryo) { 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 0af230ed29..2e6453484c 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala @@ -4,26 +4,24 @@ import scala.reflect.ClassTag /** - * This object implements a Pregel-like bulk-synchronous - * message-passing API. However, unlike the original Pregel API the - * GraphX pregel API factors the sendMessage computation over edges, - * enables the message sending computation to read both vertex - * attributes, and finally constrains messages to the graph structure. - * These changes allow for substantially more efficient distributed - * execution while also exposing greater flexibility for graph based - * computation. + * Implements a Pregel-like bulk-synchronous message-passing API. * - * @example We can use the Pregel abstraction to implement PageRank + * Unlike the original Pregel API, the GraphX Pregel API factors the sendMessage computation over + * edges, enables the message sending computation to read both vertex attributes, and constrains + * messages to the graph structure. These changes allow for substantially more efficient + * distributed execution while also exposing greater flexibility for graph-based computation. + * + * @example We can use the Pregel abstraction to implement PageRank: * {{{ * val pagerankGraph: Graph[Double, Double] = graph * // Associate the degree with each vertex - * .outerJoinVertices(graph.outDegrees){ + * .outerJoinVertices(graph.outDegrees) { * (vid, vdata, deg) => deg.getOrElse(0) * } * // Set the weight on the edges based on the degree - * .mapTriplets( e => 1.0 / e.srcAttr ) + * .mapTriplets(e => 1.0 / e.srcAttr) * // Set the vertex attributes to the initial pagerank values - * .mapVertices( (id, attr) => 1.0 ) + * .mapVertices((id, attr) => 1.0) * * def vertexProgram(id: VertexID, attr: Double, msgSum: Double): Double = * resetProb + (1.0 - resetProb) * msgSum @@ -31,7 +29,7 @@ import scala.reflect.ClassTag * Some(edge.srcAttr * edge.attr) * def messageCombiner(a: Double, b: Double): Double = a + b * val initialMessage = 0.0 - * // Execute pregel for a fixed number of iterations. + * // Execute Pregel for a fixed number of iterations. * Pregel(pagerankGraph, initialMessage, numIter)( * vertexProgram, sendMessage, messageCombiner) * }}} @@ -54,7 +52,7 @@ object Pregel { * then the vertex-program is not invoked. * * This function iterates until there are no remaining messages, or - * for maxIterations iterations. + * for `maxIterations` iterations. * * @tparam VD the vertex data type * @tparam ED the edge data type @@ -63,9 +61,9 @@ object Pregel { * @param graph the input graph. * * @param initialMsg the message each vertex will receive at the on - * the first iteration. + * the first iteration * - * @param maxIterations the maximum number of iterations to run for. + * @param maxIterations the maximum number of iterations to run for * * @param vprog the user-defined vertex program which runs on each * vertex and receives the inbound message and computes a new vertex @@ -76,7 +74,7 @@ object Pregel { * * @param sendMsg a user supplied function that is applied to out * edges of vertices that received messages in the current - * iteration. + * iteration * * @param mergeMsg a user supplied function that takes two incoming * messages of type A and merges them into a single message of type 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 3ef9d6e9cf..3ef4fec405 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala @@ -28,7 +28,7 @@ import org.apache.spark.graphx.impl.MsgRDDFunctions import org.apache.spark.graphx.impl.VertexPartition /** - * A `VertexRDD[VD]` extends the `RDD[(VertexID, VD)]` by ensuring that there is only one entry for + * `VertexRDD[VD]` extends the `RDD[(VertexID, VD)]` by ensuring that there is only one entry for * each vertex and by pre-indexing the entries for fast, efficient joins. Two VertexRDDs with the * same index can be joined efficiently. * -- cgit v1.2.3 From 85a6645d318e728454e81096ca8140b5f640e782 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 10 Jan 2014 16:08:58 -0800 Subject: Add doc for Algorithms --- .../scala/org/apache/spark/graphx/algorithms/Algorithms.scala | 10 ++++++++++ 1 file changed, 10 insertions(+) (limited to 'graphx') diff --git a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/Algorithms.scala b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/Algorithms.scala index 4af7af545c..558d8da310 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/Algorithms.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/Algorithms.scala @@ -4,6 +4,16 @@ import scala.reflect.ClassTag import org.apache.spark.graphx._ +/** + * Provides graph algorithms directly on [[org.apache.spark.graphx.Graph]] via an implicit + * conversion. + * @example + * {{{ + * import org.apache.spark.graph.algorithms._ + * val graph: Graph[_, _] = loadGraph() + * graph.connectedComponents() + * }}} + */ class Algorithms[VD: ClassTag, ED: ClassTag](self: Graph[VD, ED]) { /** * Run a dynamic version of PageRank returning a graph with vertex attributes containing the -- cgit v1.2.3 From 41d6586e8e0df94fee66a386c967b56c535e3c28 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 10 Jan 2014 18:00:54 -0800 Subject: Revert changes to Spark's (PrimitiveKey)OpenHashMap; copy PKOHM to graphx --- .../apache/spark/util/collection/OpenHashMap.scala | 51 +++---- .../util/collection/PrimitiveKeyOpenHashMap.scala | 57 +++----- .../apache/spark/graphx/impl/EdgePartition.scala | 2 +- .../spark/graphx/impl/EdgePartitionBuilder.scala | 3 +- .../spark/graphx/impl/EdgeTripletIterator.scala | 2 +- .../org/apache/spark/graphx/impl/GraphImpl.scala | 3 - .../apache/spark/graphx/impl/VertexPartition.scala | 5 +- .../util/collection/PrimitiveKeyOpenHashMap.scala | 153 +++++++++++++++++++++ 8 files changed, 192 insertions(+), 84 deletions(-) create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/util/collection/PrimitiveKeyOpenHashMap.scala (limited to 'graphx') diff --git a/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala b/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala index a7a6635dec..c26f23d500 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala @@ -28,20 +28,18 @@ import scala.reflect.ClassTag */ private[spark] class OpenHashMap[K >: Null : ClassTag, @specialized(Long, Int, Double) V: ClassTag]( - val keySet: OpenHashSet[K], var _values: Array[V]) + initialCapacity: Int) extends Iterable[(K, V)] with Serializable { - /** - * Allocate an OpenHashMap with a fixed initial capacity - */ - def this(initialCapacity: Int = 64) = - this(new OpenHashSet[K](initialCapacity), new Array[V](initialCapacity)) + def this() = this(64) - /** - * Allocate an OpenHashMap with a fixed initial capacity - */ - def this(keySet: OpenHashSet[K]) = this(keySet, new Array[V](keySet.capacity)) + protected var _keySet = new OpenHashSet[K](initialCapacity) + + // Init in constructor (instead of in declaration) to work around a Scala compiler specialization + // bug that would generate two arrays (one for Object and one for specialized T). + private var _values: Array[V] = _ + _values = new Array[V](_keySet.capacity) @transient private var _oldValues: Array[V] = null @@ -49,14 +47,14 @@ class OpenHashMap[K >: Null : ClassTag, @specialized(Long, Int, Double) V: Class private var haveNullValue = false private var nullValue: V = null.asInstanceOf[V] - override def size: Int = if (haveNullValue) keySet.size + 1 else keySet.size + override def size: Int = if (haveNullValue) _keySet.size + 1 else _keySet.size /** Get the value for a given key */ def apply(k: K): V = { if (k == null) { nullValue } else { - val pos = keySet.getPos(k) + val pos = _keySet.getPos(k) if (pos < 0) { null.asInstanceOf[V] } else { @@ -71,26 +69,9 @@ class OpenHashMap[K >: Null : ClassTag, @specialized(Long, Int, Double) V: Class haveNullValue = true nullValue = v } else { - val pos = keySet.addWithoutResize(k) & OpenHashSet.POSITION_MASK + val pos = _keySet.addWithoutResize(k) & OpenHashSet.POSITION_MASK _values(pos) = v - keySet.rehashIfNeeded(k, grow, move) - _oldValues = null - } - } - - /** Set the value for a key */ - def setMerge(k: K, v: V, mergeF: (V,V) => V) { - if (k == null) { - if(haveNullValue) { - nullValue = mergeF(nullValue, v) - } else { - haveNullValue = true - nullValue = v - } - } else { - val pos = keySet.addWithoutResize(k) & OpenHashSet.POSITION_MASK - _values(pos) = mergeF(_values(pos), v) - keySet.rehashIfNeeded(k, grow, move) + _keySet.rehashIfNeeded(k, grow, move) _oldValues = null } } @@ -111,11 +92,11 @@ class OpenHashMap[K >: Null : ClassTag, @specialized(Long, Int, Double) V: Class } nullValue } else { - val pos = keySet.addWithoutResize(k) + val pos = _keySet.addWithoutResize(k) if ((pos & OpenHashSet.NONEXISTENCE_MASK) != 0) { val newValue = defaultValue _values(pos & OpenHashSet.POSITION_MASK) = newValue - keySet.rehashIfNeeded(k, grow, move) + _keySet.rehashIfNeeded(k, grow, move) newValue } else { _values(pos) = mergeValue(_values(pos)) @@ -137,9 +118,9 @@ class OpenHashMap[K >: Null : ClassTag, @specialized(Long, Int, Double) V: Class } pos += 1 } - pos = keySet.nextPos(pos) + pos = _keySet.nextPos(pos) if (pos >= 0) { - val ret = (keySet.getValue(pos), _values(pos)) + val ret = (_keySet.getValue(pos), _values(pos)) pos += 1 ret } else { diff --git a/core/src/main/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMap.scala b/core/src/main/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMap.scala index 1dc9f744e1..2e1ef06cbc 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMap.scala @@ -29,68 +29,45 @@ import scala.reflect._ private[spark] class PrimitiveKeyOpenHashMap[@specialized(Long, Int) K: ClassTag, @specialized(Long, Int, Double) V: ClassTag]( - val keySet: OpenHashSet[K], var _values: Array[V]) + initialCapacity: Int) extends Iterable[(K, V)] with Serializable { - /** - * Allocate an OpenHashMap with a fixed initial capacity - */ - def this(initialCapacity: Int) = - this(new OpenHashSet[K](initialCapacity), new Array[V](initialCapacity)) - - /** - * Allocate an OpenHashMap with a default initial capacity, providing a true - * no-argument constructor. - */ def this() = this(64) - /** - * Allocate an OpenHashMap with a fixed initial capacity - */ - def this(keySet: OpenHashSet[K]) = this(keySet, new Array[V](keySet.capacity)) - require(classTag[K] == classTag[Long] || classTag[K] == classTag[Int]) + // Init in constructor (instead of in declaration) to work around a Scala compiler specialization + // bug that would generate two arrays (one for Object and one for specialized T). + protected var _keySet: OpenHashSet[K] = _ + private var _values: Array[V] = _ + _keySet = new OpenHashSet[K](initialCapacity) + _values = new Array[V](_keySet.capacity) + private var _oldValues: Array[V] = null - override def size = keySet.size + override def size = _keySet.size /** Get the value for a given key */ def apply(k: K): V = { - val pos = keySet.getPos(k) + val pos = _keySet.getPos(k) _values(pos) } /** Get the value for a given key, or returns elseValue if it doesn't exist. */ def getOrElse(k: K, elseValue: V): V = { - val pos = keySet.getPos(k) + val pos = _keySet.getPos(k) if (pos >= 0) _values(pos) else elseValue } /** Set the value for a key */ def update(k: K, v: V) { - val pos = keySet.addWithoutResize(k) & OpenHashSet.POSITION_MASK + val pos = _keySet.addWithoutResize(k) & OpenHashSet.POSITION_MASK _values(pos) = v - keySet.rehashIfNeeded(k, grow, move) + _keySet.rehashIfNeeded(k, grow, move) _oldValues = null } - - /** Set the value for a key */ - def setMerge(k: K, v: V, mergeF: (V, V) => V) { - val pos = keySet.addWithoutResize(k) - val ind = pos & OpenHashSet.POSITION_MASK - if ((pos & OpenHashSet.NONEXISTENCE_MASK) != 0) { // if first add - _values(ind) = v - } else { - _values(ind) = mergeF(_values(ind), v) - } - keySet.rehashIfNeeded(k, grow, move) - _oldValues = null - } - - /** * If the key doesn't exist yet in the hash map, set its value to defaultValue; otherwise, * set its value to mergeValue(oldValue). @@ -98,11 +75,11 @@ class PrimitiveKeyOpenHashMap[@specialized(Long, Int) K: ClassTag, * @return the newly updated value. */ def changeValue(k: K, defaultValue: => V, mergeValue: (V) => V): V = { - val pos = keySet.addWithoutResize(k) + val pos = _keySet.addWithoutResize(k) if ((pos & OpenHashSet.NONEXISTENCE_MASK) != 0) { val newValue = defaultValue _values(pos & OpenHashSet.POSITION_MASK) = newValue - keySet.rehashIfNeeded(k, grow, move) + _keySet.rehashIfNeeded(k, grow, move) newValue } else { _values(pos) = mergeValue(_values(pos)) @@ -116,9 +93,9 @@ class PrimitiveKeyOpenHashMap[@specialized(Long, Int) K: ClassTag, /** Get the next value we should return from next(), or null if we're finished iterating */ def computeNextPair(): (K, V) = { - pos = keySet.nextPos(pos) + pos = _keySet.nextPos(pos) if (pos >= 0) { - val ret = (keySet.getValue(pos), _values(pos)) + val ret = (_keySet.getValue(pos), _values(pos)) pos += 1 ret } else { 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 4176563d22..a03e73ee79 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 @@ -3,7 +3,7 @@ package org.apache.spark.graphx.impl import scala.reflect.ClassTag import org.apache.spark.graphx._ -import org.apache.spark.util.collection.PrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap /** * A collection of edges stored in 3 large columnar arrays (src, dst, attribute). The arrays are 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 ca64e9af66..fbc29409b5 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 @@ -4,7 +4,8 @@ import scala.reflect.ClassTag import scala.util.Sorting import org.apache.spark.graphx._ -import org.apache.spark.util.collection.{PrimitiveKeyOpenHashMap, PrimitiveVector} +import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap +import org.apache.spark.util.collection.PrimitiveVector class EdgePartitionBuilder[@specialized(Long, Int, Double) ED: ClassTag](size: Int = 64) { var edges = new PrimitiveVector[Edge[ED]](size) 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 c5258360da..bad840f1cd 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 @@ -3,7 +3,7 @@ package org.apache.spark.graphx.impl import scala.reflect.ClassTag import org.apache.spark.graphx._ -import org.apache.spark.util.collection.PrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap /** * The Iterator type returned when constructing edge triplets. This class technically could be 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 987a646c0c..c66b8c804f 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 @@ -173,9 +173,6 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( override def mapTriplets[ED2: ClassTag]( f: (PartitionID, Iterator[EdgeTriplet[VD, ED]]) => Iterator[ED2]): Graph[VD, ED2] = { - // Use an explicit manifest in PrimitiveKeyOpenHashMap init so we don't pull in the implicit - // manifest from GraphImpl (which would require serializing GraphImpl). - val vdTag = classTag[VD] val newEdgePartitions = edges.partitionsRDD.zipPartitions(replicatedVertexView.get(true, true), true) { (ePartIter, vTableReplicatedIter) => 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 7c83497ca9..f97ff75fb2 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 @@ -2,11 +2,10 @@ package org.apache.spark.graphx.impl import scala.reflect.ClassTag -import org.apache.spark.util.collection.{BitSet, PrimitiveKeyOpenHashMap} - import org.apache.spark.Logging import org.apache.spark.graphx._ - +import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap +import org.apache.spark.util.collection.BitSet private[graphx] object VertexPartition { diff --git a/graphx/src/main/scala/org/apache/spark/graphx/util/collection/PrimitiveKeyOpenHashMap.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/collection/PrimitiveKeyOpenHashMap.scala new file mode 100644 index 0000000000..1088944cd3 --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/util/collection/PrimitiveKeyOpenHashMap.scala @@ -0,0 +1,153 @@ +/* + * 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.util.collection + +import org.apache.spark.util.collection.OpenHashSet + +import scala.reflect._ + +/** + * A fast hash map implementation for primitive, non-null keys. This hash map supports + * insertions and updates, but not deletions. This map is about an order of magnitude + * faster than java.util.HashMap, while using much less space overhead. + * + * Under the hood, it uses our OpenHashSet implementation. + */ +private[spark] +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)] + with Serializable { + + /** + * Allocate an OpenHashMap with a fixed initial capacity + */ + def this(initialCapacity: Int) = + this(new OpenHashSet[K](initialCapacity), new Array[V](initialCapacity)) + + /** + * Allocate an OpenHashMap with a default initial capacity, providing a true + * no-argument constructor. + */ + def this() = this(64) + + /** + * Allocate an OpenHashMap with a fixed initial capacity + */ + def this(keySet: OpenHashSet[K]) = this(keySet, new Array[V](keySet.capacity)) + + require(classTag[K] == classTag[Long] || classTag[K] == classTag[Int]) + + private var _oldValues: Array[V] = null + + override def size = keySet.size + + /** Get the value for a given key */ + def apply(k: K): V = { + val pos = keySet.getPos(k) + _values(pos) + } + + /** Get the value for a given key, or returns elseValue if it doesn't exist. */ + def getOrElse(k: K, elseValue: V): V = { + val pos = keySet.getPos(k) + if (pos >= 0) _values(pos) else elseValue + } + + /** Set the value for a key */ + def update(k: K, v: V) { + val pos = keySet.addWithoutResize(k) & OpenHashSet.POSITION_MASK + _values(pos) = v + keySet.rehashIfNeeded(k, grow, move) + _oldValues = null + } + + + /** Set the value for a key */ + def setMerge(k: K, v: V, mergeF: (V, V) => V) { + val pos = keySet.addWithoutResize(k) + val ind = pos & OpenHashSet.POSITION_MASK + if ((pos & OpenHashSet.NONEXISTENCE_MASK) != 0) { // if first add + _values(ind) = v + } else { + _values(ind) = mergeF(_values(ind), v) + } + keySet.rehashIfNeeded(k, grow, move) + _oldValues = null + } + + + /** + * If the key doesn't exist yet in the hash map, set its value to defaultValue; otherwise, + * set its value to mergeValue(oldValue). + * + * @return the newly updated value. + */ + def changeValue(k: K, defaultValue: => V, mergeValue: (V) => V): V = { + val pos = keySet.addWithoutResize(k) + if ((pos & OpenHashSet.NONEXISTENCE_MASK) != 0) { + val newValue = defaultValue + _values(pos & OpenHashSet.POSITION_MASK) = newValue + keySet.rehashIfNeeded(k, grow, move) + newValue + } else { + _values(pos) = mergeValue(_values(pos)) + _values(pos) + } + } + + override def iterator = new Iterator[(K, V)] { + var pos = 0 + var nextPair: (K, V) = computeNextPair() + + /** Get the next value we should return from next(), or null if we're finished iterating */ + def computeNextPair(): (K, V) = { + pos = keySet.nextPos(pos) + if (pos >= 0) { + val ret = (keySet.getValue(pos), _values(pos)) + pos += 1 + ret + } else { + null + } + } + + def hasNext = nextPair != null + + def next() = { + val pair = nextPair + nextPair = computeNextPair() + pair + } + } + + // The following member variables are declared as protected instead of private for the + // specialization to work (specialized class extends the unspecialized one and needs access + // to the "private" variables). + // They also should have been val's. We use var's because there is a Scala compiler bug that + // would throw illegal access error at runtime if they are declared as val's. + protected var grow = (newCapacity: Int) => { + _oldValues = _values + _values = new Array[V](newCapacity) + } + + protected var move = (oldPos: Int, newPos: Int) => { + _values(newPos) = _oldValues(oldPos) + } +} -- cgit v1.2.3 From c4fb6a87d386cfd0d8b9d8733e75b8bd8cc0359e Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 10 Jan 2014 18:36:42 -0800 Subject: Fix scaladoc warnings --- graphx/src/main/scala/org/apache/spark/graphx/Graph.scala | 8 ++++---- graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala | 4 ++-- 2 files changed, 6 insertions(+), 6 deletions(-) (limited to 'graphx') 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 a188de5537..64d44c3e06 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala @@ -125,7 +125,7 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] { /** * Transforms each edge attribute in the graph using the map function. The map function is not * passed the vertex value for the vertices adjacent to the edge. If vertex values are desired, - * use [[mapTriplets]]. + * use `mapTriplets`. * * @note This graph is not changed and that the new graph has the * same structure. As a consequence the underlying index structures @@ -148,7 +148,7 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] { * time. The map function is given an iterator over edges within a logical partition as well as * the partition's ID, and it should return a new iterator over the new values of each edge. The * new iterator's elements must correspond one-to-one with the old iterator's elements. If - * adjacent vertex values are desired, use [[mapTriplets]]. + * adjacent vertex values are desired, use `mapTriplets`. * * @note This does not change the structure of the * graph or modify the values of this graph. As a consequence @@ -166,7 +166,7 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] { /** * Transforms each edge attribute using the map function, passing it the adjacent vertex attributes - * as well. If adjacent vertex values are not required, consider using [[mapEdges]] instead. + * as well. If adjacent vertex values are not required, consider using `mapEdges` instead. * * @note This does not change the structure of the * graph or modify the values of this graph. As a consequence @@ -194,7 +194,7 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] { * adjacent vertex attributes as well. The map function is given an iterator over edge triplets * within a logical partition and should yield a new iterator over the new values of each edge in * the order in which they are provided. If adjacent vertex values are not required, consider - * using [[mapEdges]] instead. + * using `mapEdges` instead. * * @note This does not change the structure of the * graph or modify the values of this graph. As a consequence 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 3ef4fec405..d20745d0d4 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala @@ -101,9 +101,9 @@ class VertexRDD[@specialized VD: ClassTag]( } /** - * Applies a function to each [[impl.VertexPartition]] of this RDD and returns a new VertexRDD. + * Applies a function to each `VertexPartition` of this RDD and returns a new VertexRDD. */ - def mapVertexPartitions[VD2: ClassTag](f: VertexPartition[VD] => VertexPartition[VD2]) + private[graphx] def mapVertexPartitions[VD2: ClassTag](f: VertexPartition[VD] => VertexPartition[VD2]) : VertexRDD[VD2] = { val newPartitionsRDD = partitionsRDD.mapPartitions(_.map(f), preservesPartitioning = true) new VertexRDD(newPartitionsRDD) -- cgit v1.2.3 From 2d7e8d8c48e07adea41f255e357aa750b9ddcdc6 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 10 Jan 2014 23:46:02 -0800 Subject: Add GC note to GraphLab --- graphx/src/main/scala/org/apache/spark/graphx/GraphLab.scala | 6 ++++++ 1 file changed, 6 insertions(+) (limited to 'graphx') diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphLab.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphLab.scala index 016bab5d02..2f828ad807 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphLab.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphLab.scala @@ -35,6 +35,12 @@ object GraphLab extends Logging { * @tparam ED the graph edge attribute type * @tparam A the type accumulated during the gather phase * @return the resulting graph after the algorithm converges + * + * @note Unlike [[Pregel]], this implementation of [[GraphLab]] does not unpersist RDDs from + * previous iterations. As a result, long-running iterative GraphLab programs will eventually fill + * the Spark cache. Though Spark will evict RDDs from old iterations eventually, garbage + * collection will take longer than necessary since it must examine the entire cache. This will be + * fixed in a future update. */ def apply[VD: ClassTag, ED: ClassTag, A: ClassTag] (graph: Graph[VD, ED], numIter: Int, -- cgit v1.2.3 From 34496d6a9fd18ba708b66dcc318c7568608e963f Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Sat, 11 Jan 2014 00:08:15 -0800 Subject: Move Analytics to algorithms and fix doc --- .../scala/org/apache/spark/graphx/Analytics.scala | 163 --------------------- .../apache/spark/graphx/algorithms/Analytics.scala | 157 ++++++++++++++++++++ 2 files changed, 157 insertions(+), 163 deletions(-) delete mode 100644 graphx/src/main/scala/org/apache/spark/graphx/Analytics.scala create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/algorithms/Analytics.scala (limited to 'graphx') diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Analytics.scala b/graphx/src/main/scala/org/apache/spark/graphx/Analytics.scala deleted file mode 100644 index 1359b92c6b..0000000000 --- a/graphx/src/main/scala/org/apache/spark/graphx/Analytics.scala +++ /dev/null @@ -1,163 +0,0 @@ -package org.apache.spark.graphx - -import org.apache.spark._ -import org.apache.spark.graphx.algorithms._ - - -/** - * The Analytics object contains a collection of basic graph analytics - * algorithms that operate largely on the graph structure. - * - * In addition the Analytics object contains a driver `main` which can - * be used to apply the various functions to graphs in standard - * formats. - */ -object Analytics extends Logging { - - def main(args: Array[String]) = { - val host = args(0) - val taskType = args(1) - val fname = args(2) - val options = args.drop(3).map { arg => - arg.dropWhile(_ == '-').split('=') match { - case Array(opt, v) => (opt -> v) - case _ => throw new IllegalArgumentException("Invalid argument: " + arg) - } - } - - def setLogLevels(level: org.apache.log4j.Level, loggers: TraversableOnce[String]) = { - loggers.map{ - loggerName => - val logger = org.apache.log4j.Logger.getLogger(loggerName) - val prevLevel = logger.getLevel() - logger.setLevel(level) - loggerName -> prevLevel - }.toMap - } - - def pickPartitioner(v: String): PartitionStrategy = { - v match { - case "RandomVertexCut" => RandomVertexCut - case "EdgePartition1D" => EdgePartition1D - case "EdgePartition2D" => EdgePartition2D - case "CanonicalRandomVertexCut" => CanonicalRandomVertexCut - case _ => throw new IllegalArgumentException("Invalid Partition Strategy: " + v) - } - } - val serializer = "org.apache.spark.serializer.KryoSerializer" - System.setProperty("spark.serializer", serializer) - System.setProperty("spark.kryo.registrator", "org.apache.spark.graphx.GraphKryoRegistrator") - - taskType match { - case "pagerank" => { - - var tol:Float = 0.001F - var outFname = "" - var numVPart = 4 - var numEPart = 4 - var partitionStrategy: Option[PartitionStrategy] = None - - options.foreach{ - case ("tol", v) => tol = v.toFloat - case ("output", v) => outFname = v - case ("numVPart", v) => numVPart = v.toInt - case ("numEPart", v) => numEPart = v.toInt - case ("partStrategy", v) => partitionStrategy = Some(pickPartitioner(v)) - case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) - } - - println("======================================") - println("| PageRank |") - println("======================================") - - val sc = new SparkContext(host, "PageRank(" + fname + ")") - - val unpartitionedGraph = GraphLoader.edgeListFile(sc, fname, - minEdgePartitions = numEPart).cache() - val graph = partitionStrategy.foldLeft(unpartitionedGraph)(_.partitionBy(_)) - - println("GRAPHX: Number of vertices " + graph.vertices.count) - println("GRAPHX: Number of edges " + graph.edges.count) - - val pr = graph.pageRank(tol).vertices.cache() - - println("GRAPHX: Total rank: " + pr.map(_._2).reduce(_+_)) - - if (!outFname.isEmpty) { - logWarning("Saving pageranks of pages to " + outFname) - pr.map{case (id, r) => id + "\t" + r}.saveAsTextFile(outFname) - } - - sc.stop() - } - - case "cc" => { - - var numIter = Int.MaxValue - var numVPart = 4 - var numEPart = 4 - var isDynamic = false - var partitionStrategy: Option[PartitionStrategy] = None - - options.foreach{ - case ("numIter", v) => numIter = v.toInt - case ("dynamic", v) => isDynamic = v.toBoolean - case ("numEPart", v) => numEPart = v.toInt - case ("numVPart", v) => numVPart = v.toInt - case ("partStrategy", v) => partitionStrategy = Some(pickPartitioner(v)) - case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) - } - - if(!isDynamic && numIter == Int.MaxValue) { - println("Set number of iterations!") - sys.exit(1) - } - println("======================================") - println("| Connected Components |") - println("--------------------------------------") - println(" Using parameters:") - println(" \tDynamic: " + isDynamic) - println(" \tNumIter: " + numIter) - println("======================================") - - val sc = new SparkContext(host, "ConnectedComponents(" + fname + ")") - val unpartitionedGraph = GraphLoader.edgeListFile(sc, fname, - minEdgePartitions = numEPart).cache() - val graph = partitionStrategy.foldLeft(unpartitionedGraph)(_.partitionBy(_)) - - val cc = ConnectedComponents.run(graph) - println("Components: " + cc.vertices.map{ case (vid,data) => data}.distinct()) - sc.stop() - } - - case "triangles" => { - var numVPart = 4 - var numEPart = 4 - // TriangleCount requires the graph to be partitioned - var partitionStrategy: PartitionStrategy = RandomVertexCut - - options.foreach{ - case ("numEPart", v) => numEPart = v.toInt - case ("numVPart", v) => numVPart = v.toInt - case ("partStrategy", v) => partitionStrategy = pickPartitioner(v) - case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) - } - println("======================================") - println("| Triangle Count |") - println("--------------------------------------") - val sc = new SparkContext(host, "TriangleCount(" + fname + ")") - val graph = GraphLoader.edgeListFile(sc, fname, canonicalOrientation = true, - minEdgePartitions = numEPart).partitionBy(partitionStrategy).cache() - val triangles = TriangleCount.run(graph) - println("Triangles: " + triangles.vertices.map { - case (vid,data) => data.toLong - }.reduce(_+_) / 3) - sc.stop() - } - - case _ => { - println("Invalid task type.") - } - } - } -} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/Analytics.scala b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/Analytics.scala new file mode 100644 index 0000000000..f09685402a --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/Analytics.scala @@ -0,0 +1,157 @@ +package org.apache.spark.graphx.algorithms + +import org.apache.spark._ +import org.apache.spark.graphx._ + +/** + * Driver program for running graph algorithms. + */ +object Analytics extends Logging { + + def main(args: Array[String]) = { + val host = args(0) + val taskType = args(1) + val fname = args(2) + val options = args.drop(3).map { arg => + arg.dropWhile(_ == '-').split('=') match { + case Array(opt, v) => (opt -> v) + case _ => throw new IllegalArgumentException("Invalid argument: " + arg) + } + } + + def setLogLevels(level: org.apache.log4j.Level, loggers: TraversableOnce[String]) = { + loggers.map{ + loggerName => + val logger = org.apache.log4j.Logger.getLogger(loggerName) + val prevLevel = logger.getLevel() + logger.setLevel(level) + loggerName -> prevLevel + }.toMap + } + + def pickPartitioner(v: String): PartitionStrategy = { + v match { + case "RandomVertexCut" => RandomVertexCut + case "EdgePartition1D" => EdgePartition1D + case "EdgePartition2D" => EdgePartition2D + case "CanonicalRandomVertexCut" => CanonicalRandomVertexCut + case _ => throw new IllegalArgumentException("Invalid Partition Strategy: " + v) + } + } + val serializer = "org.apache.spark.serializer.KryoSerializer" + System.setProperty("spark.serializer", serializer) + System.setProperty("spark.kryo.registrator", "org.apache.spark.graphx.GraphKryoRegistrator") + + taskType match { + case "pagerank" => { + + var tol:Float = 0.001F + var outFname = "" + var numVPart = 4 + var numEPart = 4 + var partitionStrategy: Option[PartitionStrategy] = None + + options.foreach{ + case ("tol", v) => tol = v.toFloat + case ("output", v) => outFname = v + case ("numVPart", v) => numVPart = v.toInt + case ("numEPart", v) => numEPart = v.toInt + case ("partStrategy", v) => partitionStrategy = Some(pickPartitioner(v)) + case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) + } + + println("======================================") + println("| PageRank |") + println("======================================") + + val sc = new SparkContext(host, "PageRank(" + fname + ")") + + val unpartitionedGraph = GraphLoader.edgeListFile(sc, fname, + minEdgePartitions = numEPart).cache() + val graph = partitionStrategy.foldLeft(unpartitionedGraph)(_.partitionBy(_)) + + println("GRAPHX: Number of vertices " + graph.vertices.count) + println("GRAPHX: Number of edges " + graph.edges.count) + + val pr = graph.pageRank(tol).vertices.cache() + + println("GRAPHX: Total rank: " + pr.map(_._2).reduce(_+_)) + + if (!outFname.isEmpty) { + logWarning("Saving pageranks of pages to " + outFname) + pr.map{case (id, r) => id + "\t" + r}.saveAsTextFile(outFname) + } + + sc.stop() + } + + case "cc" => { + + var numIter = Int.MaxValue + var numVPart = 4 + var numEPart = 4 + var isDynamic = false + var partitionStrategy: Option[PartitionStrategy] = None + + options.foreach{ + case ("numIter", v) => numIter = v.toInt + case ("dynamic", v) => isDynamic = v.toBoolean + case ("numEPart", v) => numEPart = v.toInt + case ("numVPart", v) => numVPart = v.toInt + case ("partStrategy", v) => partitionStrategy = Some(pickPartitioner(v)) + case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) + } + + if(!isDynamic && numIter == Int.MaxValue) { + println("Set number of iterations!") + sys.exit(1) + } + println("======================================") + println("| Connected Components |") + println("--------------------------------------") + println(" Using parameters:") + println(" \tDynamic: " + isDynamic) + println(" \tNumIter: " + numIter) + println("======================================") + + val sc = new SparkContext(host, "ConnectedComponents(" + fname + ")") + val unpartitionedGraph = GraphLoader.edgeListFile(sc, fname, + minEdgePartitions = numEPart).cache() + val graph = partitionStrategy.foldLeft(unpartitionedGraph)(_.partitionBy(_)) + + val cc = ConnectedComponents.run(graph) + println("Components: " + cc.vertices.map{ case (vid,data) => data}.distinct()) + sc.stop() + } + + case "triangles" => { + var numVPart = 4 + var numEPart = 4 + // TriangleCount requires the graph to be partitioned + var partitionStrategy: PartitionStrategy = RandomVertexCut + + options.foreach{ + case ("numEPart", v) => numEPart = v.toInt + case ("numVPart", v) => numVPart = v.toInt + case ("partStrategy", v) => partitionStrategy = pickPartitioner(v) + case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) + } + println("======================================") + println("| Triangle Count |") + println("--------------------------------------") + val sc = new SparkContext(host, "TriangleCount(" + fname + ")") + val graph = GraphLoader.edgeListFile(sc, fname, canonicalOrientation = true, + minEdgePartitions = numEPart).partitionBy(partitionStrategy).cache() + val triangles = TriangleCount.run(graph) + println("Triangles: " + triangles.vertices.map { + case (vid,data) => data.toLong + }.reduce(_+_) / 3) + sc.stop() + } + + case _ => { + println("Invalid task type.") + } + } + } +} -- cgit v1.2.3 From 4f7ddf40fc353ebcfd0f5dec7ea09aa90c657d32 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Sat, 11 Jan 2014 00:13:49 -0800 Subject: Optimize Edge.lexicographicOrdering --- graphx/src/main/scala/org/apache/spark/graphx/Edge.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) (limited to 'graphx') diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala b/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala index c6b62ebaca..b3a2a21862 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala @@ -45,6 +45,6 @@ case class Edge[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED] object Edge { def lexicographicOrdering[ED] = new Ordering[Edge[ED]] { override def compare(a: Edge[ED], b: Edge[ED]): Int = - Ordering[(VertexID, VertexID)].compare((a.srcId, a.dstId), (b.srcId, b.dstId)) + (if (a.srcId != b.srcId) a.srcId - b.srcId else a.dstId - b.dstId).toInt } } -- cgit v1.2.3 From feaa07802203b79f454454445c0a12a2784ccfeb Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Sat, 11 Jan 2014 00:28:18 -0800 Subject: algorithms -> lib --- .../main/scala/org/apache/spark/graphx/Graph.scala | 4 +- .../spark/graphx/algorithms/Algorithms.scala | 66 --------- .../apache/spark/graphx/algorithms/Analytics.scala | 157 -------------------- .../graphx/algorithms/ConnectedComponents.scala | 39 ----- .../apache/spark/graphx/algorithms/PageRank.scala | 160 --------------------- .../spark/graphx/algorithms/SVDPlusPlus.scala | 107 -------------- .../algorithms/StronglyConnectedComponents.scala | 88 ------------ .../spark/graphx/algorithms/TriangleCount.scala | 78 ---------- .../apache/spark/graphx/algorithms/package.scala | 8 -- .../org/apache/spark/graphx/lib/Algorithms.scala | 66 +++++++++ .../org/apache/spark/graphx/lib/Analytics.scala | 157 ++++++++++++++++++++ .../spark/graphx/lib/ConnectedComponents.scala | 39 +++++ .../org/apache/spark/graphx/lib/PageRank.scala | 160 +++++++++++++++++++++ .../org/apache/spark/graphx/lib/SVDPlusPlus.scala | 107 ++++++++++++++ .../graphx/lib/StronglyConnectedComponents.scala | 88 ++++++++++++ .../apache/spark/graphx/lib/TriangleCount.scala | 78 ++++++++++ .../org/apache/spark/graphx/lib/package.scala | 8 ++ .../algorithms/ConnectedComponentsSuite.scala | 83 ----------- .../spark/graphx/algorithms/PageRankSuite.scala | 119 --------------- .../spark/graphx/algorithms/SVDPlusPlusSuite.scala | 31 ---- .../StronglyConnectedComponentsSuite.scala | 57 -------- .../graphx/algorithms/TriangleCountSuite.scala | 73 ---------- .../graphx/lib/ConnectedComponentsSuite.scala | 83 +++++++++++ .../apache/spark/graphx/lib/PageRankSuite.scala | 119 +++++++++++++++ .../apache/spark/graphx/lib/SVDPlusPlusSuite.scala | 31 ++++ .../lib/StronglyConnectedComponentsSuite.scala | 57 ++++++++ .../spark/graphx/lib/TriangleCountSuite.scala | 73 ++++++++++ 27 files changed, 1068 insertions(+), 1068 deletions(-) delete mode 100644 graphx/src/main/scala/org/apache/spark/graphx/algorithms/Algorithms.scala delete mode 100644 graphx/src/main/scala/org/apache/spark/graphx/algorithms/Analytics.scala delete mode 100644 graphx/src/main/scala/org/apache/spark/graphx/algorithms/ConnectedComponents.scala delete mode 100644 graphx/src/main/scala/org/apache/spark/graphx/algorithms/PageRank.scala delete mode 100644 graphx/src/main/scala/org/apache/spark/graphx/algorithms/SVDPlusPlus.scala delete mode 100644 graphx/src/main/scala/org/apache/spark/graphx/algorithms/StronglyConnectedComponents.scala delete mode 100644 graphx/src/main/scala/org/apache/spark/graphx/algorithms/TriangleCount.scala delete mode 100644 graphx/src/main/scala/org/apache/spark/graphx/algorithms/package.scala create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/lib/Algorithms.scala create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/lib/Analytics.scala create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/lib/ConnectedComponents.scala create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/lib/StronglyConnectedComponents.scala create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/lib/TriangleCount.scala create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/lib/package.scala delete mode 100644 graphx/src/test/scala/org/apache/spark/graphx/algorithms/ConnectedComponentsSuite.scala delete mode 100644 graphx/src/test/scala/org/apache/spark/graphx/algorithms/PageRankSuite.scala delete mode 100644 graphx/src/test/scala/org/apache/spark/graphx/algorithms/SVDPlusPlusSuite.scala delete mode 100644 graphx/src/test/scala/org/apache/spark/graphx/algorithms/StronglyConnectedComponentsSuite.scala delete mode 100644 graphx/src/test/scala/org/apache/spark/graphx/algorithms/TriangleCountSuite.scala create mode 100644 graphx/src/test/scala/org/apache/spark/graphx/lib/ConnectedComponentsSuite.scala create mode 100644 graphx/src/test/scala/org/apache/spark/graphx/lib/PageRankSuite.scala create mode 100644 graphx/src/test/scala/org/apache/spark/graphx/lib/SVDPlusPlusSuite.scala create mode 100644 graphx/src/test/scala/org/apache/spark/graphx/lib/StronglyConnectedComponentsSuite.scala create mode 100644 graphx/src/test/scala/org/apache/spark/graphx/lib/TriangleCountSuite.scala (limited to 'graphx') 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 64d44c3e06..56513cac20 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala @@ -16,8 +16,8 @@ import org.apache.spark.storage.StorageLevel * operations return new graphs. * * @note [[GraphOps]] contains additional convenience operations. - * [[algorithms.Algorithms]] contains graph algorithms; to access these, - * import `org.apache.spark.graphx.algorithms._`. + * [[lib.Algorithms]] contains graph algorithms; to access these, + * import `org.apache.spark.graphx.lib._`. * * @tparam VD the vertex attribute type * @tparam ED the edge attribute type diff --git a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/Algorithms.scala b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/Algorithms.scala deleted file mode 100644 index 558d8da310..0000000000 --- a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/Algorithms.scala +++ /dev/null @@ -1,66 +0,0 @@ -package org.apache.spark.graphx.algorithms - -import scala.reflect.ClassTag - -import org.apache.spark.graphx._ - -/** - * Provides graph algorithms directly on [[org.apache.spark.graphx.Graph]] via an implicit - * conversion. - * @example - * {{{ - * import org.apache.spark.graph.algorithms._ - * val graph: Graph[_, _] = loadGraph() - * graph.connectedComponents() - * }}} - */ -class Algorithms[VD: ClassTag, ED: ClassTag](self: Graph[VD, ED]) { - /** - * Run a dynamic version of PageRank returning a graph with vertex attributes containing the - * PageRank and edge attributes containing the normalized edge weight. - * - * @see [[org.apache.spark.graphx.algorithms.PageRank]], method `runUntilConvergence`. - */ - def pageRank(tol: Double, resetProb: Double = 0.15): Graph[Double, Double] = { - PageRank.runUntilConvergence(self, tol, resetProb) - } - - /** - * Run PageRank for a fixed number of iterations returning a graph with vertex attributes - * containing the PageRank and edge attributes the normalized edge weight. - * - * @see [[org.apache.spark.graphx.algorithms.PageRank]], method `run`. - */ - def staticPageRank(numIter: Int, resetProb: Double = 0.15): Graph[Double, Double] = { - PageRank.run(self, numIter, resetProb) - } - - /** - * Compute the connected component membership of each vertex and return a graph with the vertex - * value containing the lowest vertex id in the connected component containing that vertex. - * - * @see [[org.apache.spark.graphx.algorithms.ConnectedComponents]] - */ - def connectedComponents(): Graph[VertexID, ED] = { - ConnectedComponents.run(self) - } - - /** - * Compute the number of triangles passing through each vertex. - * - * @see [[org.apache.spark.graphx.algorithms.TriangleCount]] - */ - def triangleCount(): Graph[Int, ED] = { - TriangleCount.run(self) - } - - /** - * Compute the strongly connected component (SCC) of each vertex and return a graph with the - * vertex value containing the lowest vertex id in the SCC containing that vertex. - * - * @see [[org.apache.spark.graphx.algorithms.StronglyConnectedComponents]] - */ - def stronglyConnectedComponents(numIter: Int): Graph[VertexID, ED] = { - StronglyConnectedComponents.run(self, numIter) - } -} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/Analytics.scala b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/Analytics.scala deleted file mode 100644 index f09685402a..0000000000 --- a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/Analytics.scala +++ /dev/null @@ -1,157 +0,0 @@ -package org.apache.spark.graphx.algorithms - -import org.apache.spark._ -import org.apache.spark.graphx._ - -/** - * Driver program for running graph algorithms. - */ -object Analytics extends Logging { - - def main(args: Array[String]) = { - val host = args(0) - val taskType = args(1) - val fname = args(2) - val options = args.drop(3).map { arg => - arg.dropWhile(_ == '-').split('=') match { - case Array(opt, v) => (opt -> v) - case _ => throw new IllegalArgumentException("Invalid argument: " + arg) - } - } - - def setLogLevels(level: org.apache.log4j.Level, loggers: TraversableOnce[String]) = { - loggers.map{ - loggerName => - val logger = org.apache.log4j.Logger.getLogger(loggerName) - val prevLevel = logger.getLevel() - logger.setLevel(level) - loggerName -> prevLevel - }.toMap - } - - def pickPartitioner(v: String): PartitionStrategy = { - v match { - case "RandomVertexCut" => RandomVertexCut - case "EdgePartition1D" => EdgePartition1D - case "EdgePartition2D" => EdgePartition2D - case "CanonicalRandomVertexCut" => CanonicalRandomVertexCut - case _ => throw new IllegalArgumentException("Invalid Partition Strategy: " + v) - } - } - val serializer = "org.apache.spark.serializer.KryoSerializer" - System.setProperty("spark.serializer", serializer) - System.setProperty("spark.kryo.registrator", "org.apache.spark.graphx.GraphKryoRegistrator") - - taskType match { - case "pagerank" => { - - var tol:Float = 0.001F - var outFname = "" - var numVPart = 4 - var numEPart = 4 - var partitionStrategy: Option[PartitionStrategy] = None - - options.foreach{ - case ("tol", v) => tol = v.toFloat - case ("output", v) => outFname = v - case ("numVPart", v) => numVPart = v.toInt - case ("numEPart", v) => numEPart = v.toInt - case ("partStrategy", v) => partitionStrategy = Some(pickPartitioner(v)) - case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) - } - - println("======================================") - println("| PageRank |") - println("======================================") - - val sc = new SparkContext(host, "PageRank(" + fname + ")") - - val unpartitionedGraph = GraphLoader.edgeListFile(sc, fname, - minEdgePartitions = numEPart).cache() - val graph = partitionStrategy.foldLeft(unpartitionedGraph)(_.partitionBy(_)) - - println("GRAPHX: Number of vertices " + graph.vertices.count) - println("GRAPHX: Number of edges " + graph.edges.count) - - val pr = graph.pageRank(tol).vertices.cache() - - println("GRAPHX: Total rank: " + pr.map(_._2).reduce(_+_)) - - if (!outFname.isEmpty) { - logWarning("Saving pageranks of pages to " + outFname) - pr.map{case (id, r) => id + "\t" + r}.saveAsTextFile(outFname) - } - - sc.stop() - } - - case "cc" => { - - var numIter = Int.MaxValue - var numVPart = 4 - var numEPart = 4 - var isDynamic = false - var partitionStrategy: Option[PartitionStrategy] = None - - options.foreach{ - case ("numIter", v) => numIter = v.toInt - case ("dynamic", v) => isDynamic = v.toBoolean - case ("numEPart", v) => numEPart = v.toInt - case ("numVPart", v) => numVPart = v.toInt - case ("partStrategy", v) => partitionStrategy = Some(pickPartitioner(v)) - case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) - } - - if(!isDynamic && numIter == Int.MaxValue) { - println("Set number of iterations!") - sys.exit(1) - } - println("======================================") - println("| Connected Components |") - println("--------------------------------------") - println(" Using parameters:") - println(" \tDynamic: " + isDynamic) - println(" \tNumIter: " + numIter) - println("======================================") - - val sc = new SparkContext(host, "ConnectedComponents(" + fname + ")") - val unpartitionedGraph = GraphLoader.edgeListFile(sc, fname, - minEdgePartitions = numEPart).cache() - val graph = partitionStrategy.foldLeft(unpartitionedGraph)(_.partitionBy(_)) - - val cc = ConnectedComponents.run(graph) - println("Components: " + cc.vertices.map{ case (vid,data) => data}.distinct()) - sc.stop() - } - - case "triangles" => { - var numVPart = 4 - var numEPart = 4 - // TriangleCount requires the graph to be partitioned - var partitionStrategy: PartitionStrategy = RandomVertexCut - - options.foreach{ - case ("numEPart", v) => numEPart = v.toInt - case ("numVPart", v) => numVPart = v.toInt - case ("partStrategy", v) => partitionStrategy = pickPartitioner(v) - case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) - } - println("======================================") - println("| Triangle Count |") - println("--------------------------------------") - val sc = new SparkContext(host, "TriangleCount(" + fname + ")") - val graph = GraphLoader.edgeListFile(sc, fname, canonicalOrientation = true, - minEdgePartitions = numEPart).partitionBy(partitionStrategy).cache() - val triangles = TriangleCount.run(graph) - println("Triangles: " + triangles.vertices.map { - case (vid,data) => data.toLong - }.reduce(_+_) / 3) - sc.stop() - } - - case _ => { - println("Invalid task type.") - } - } - } -} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/ConnectedComponents.scala b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/ConnectedComponents.scala deleted file mode 100644 index 137a81f4d5..0000000000 --- a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/ConnectedComponents.scala +++ /dev/null @@ -1,39 +0,0 @@ -package org.apache.spark.graphx.algorithms - -import scala.reflect.ClassTag - -import org.apache.spark.graphx._ - - -object ConnectedComponents { - /** - * Compute the connected component membership of each vertex and return a graph with the vertex - * value containing the lowest vertex id in the connected component containing that vertex. - * - * @tparam VD the vertex attribute type (discarded in the computation) - * @tparam ED the edge attribute type (preserved in the computation) - * - * @param graph the graph for which to compute the connected components - * - * @return a graph with vertex attributes containing the smallest vertex in each - * connected component - */ - def run[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]): Graph[VertexID, ED] = { - val ccGraph = graph.mapVertices { case (vid, _) => vid } - - def sendMessage(edge: EdgeTriplet[VertexID, ED]) = { - if (edge.srcAttr < edge.dstAttr) { - Iterator((edge.dstId, edge.srcAttr)) - } else if (edge.srcAttr > edge.dstAttr) { - Iterator((edge.srcId, edge.dstAttr)) - } else { - Iterator.empty - } - } - val initialMessage = Long.MaxValue - Pregel(ccGraph, initialMessage)( - vprog = (id, attr, msg) => math.min(attr, msg), - sendMsg = sendMessage, - mergeMsg = (a, b) => math.min(a, b)) - } // end of connectedComponents -} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/PageRank.scala b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/PageRank.scala deleted file mode 100644 index ab447d5422..0000000000 --- a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/PageRank.scala +++ /dev/null @@ -1,160 +0,0 @@ -package org.apache.spark.graphx.algorithms - -import scala.reflect.ClassTag - -import org.apache.spark.Logging -import org.apache.spark.graphx._ - - -object PageRank extends Logging { - - /** - * Run PageRank for a fixed number of iterations returning a graph - * with vertex attributes containing the PageRank and edge - * attributes the normalized edge weight. - * - * The following PageRank fixed point is computed for each vertex. - * - * {{{ - * var PR = Array.fill(n)( 1.0 ) - * val oldPR = Array.fill(n)( 1.0 ) - * for( iter <- 0 until numIter ) { - * swap(oldPR, PR) - * for( i <- 0 until n ) { - * PR[i] = alpha + (1 - alpha) * inNbrs[i].map(j => oldPR[j] / outDeg[j]).sum - * } - * } - * }}} - * - * where `alpha` is the random reset probability (typically 0.15), - * `inNbrs[i]` is the set of neighbors whick link to `i` and - * `outDeg[j]` is the out degree of vertex `j`. - * - * Note that this is not the "normalized" PageRank and as a consequence pages that have no - * inlinks will have a PageRank of alpha. - * - * @tparam VD the original vertex attribute (not used) - * @tparam ED the original edge attribute (not used) - * - * @param graph the graph on which to compute PageRank - * @param numIter the number of iterations of PageRank to run - * @param resetProb the random reset probability (alpha) - * - * @return the graph containing with each vertex containing the PageRank and each edge - * containing the normalized weight. - * - */ - def run[VD: ClassTag, ED: ClassTag]( - graph: Graph[VD, ED], numIter: Int, resetProb: Double = 0.15): Graph[Double, Double] = - { - - /** - * Initialize the pagerankGraph with each edge attribute having - * weight 1/outDegree and each vertex with attribute 1.0. - */ - val pagerankGraph: Graph[Double, Double] = graph - // Associate the degree with each vertex - .outerJoinVertices(graph.outDegrees){ - (vid, vdata, deg) => deg.getOrElse(0) - } - // Set the weight on the edges based on the degree - .mapTriplets( e => 1.0 / e.srcAttr ) - // Set the vertex attributes to the initial pagerank values - .mapVertices( (id, attr) => 1.0 ) - .cache() - - // Display statistics about pagerank - logInfo(pagerankGraph.statistics.toString) - - // Define the three functions needed to implement PageRank in the GraphX - // version of Pregel - def vertexProgram(id: VertexID, attr: Double, msgSum: Double): Double = - resetProb + (1.0 - resetProb) * msgSum - def sendMessage(edge: EdgeTriplet[Double, Double]) = - Iterator((edge.dstId, edge.srcAttr * edge.attr)) - def messageCombiner(a: Double, b: Double): Double = a + b - // The initial message received by all vertices in PageRank - val initialMessage = 0.0 - - // Execute pregel for a fixed number of iterations. - Pregel(pagerankGraph, initialMessage, numIter)( - vertexProgram, sendMessage, messageCombiner) - } - - /** - * Run a dynamic version of PageRank returning a graph with vertex attributes containing the - * PageRank and edge attributes containing the normalized edge weight. - * - * {{{ - * var PR = Array.fill(n)( 1.0 ) - * val oldPR = Array.fill(n)( 0.0 ) - * while( max(abs(PR - oldPr)) > tol ) { - * swap(oldPR, PR) - * for( i <- 0 until n if abs(PR[i] - oldPR[i]) > tol ) { - * PR[i] = alpha + (1 - \alpha) * inNbrs[i].map(j => oldPR[j] / outDeg[j]).sum - * } - * } - * }}} - * - * where `alpha` is the random reset probability (typically 0.15), `inNbrs[i]` is the set of - * neighbors whick link to `i` and `outDeg[j]` is the out degree of vertex `j`. - * - * Note that this is not the "normalized" PageRank and as a consequence pages that have no - * inlinks will have a PageRank of alpha. - * - * @tparam VD the original vertex attribute (not used) - * @tparam ED the original edge attribute (not used) - * - * @param graph the graph on which to compute PageRank - * @param tol the tolerance allowed at convergence (smaller => more * accurate). - * @param resetProb the random reset probability (alpha) - * - * @return the graph containing with each vertex containing the PageRank and each edge - * containing the normalized weight. - */ - def runUntilConvergence[VD: ClassTag, ED: ClassTag]( - graph: Graph[VD, ED], tol: Double, resetProb: Double = 0.15): Graph[Double, Double] = - { - // Initialize the pagerankGraph with each edge attribute - // having weight 1/outDegree and each vertex with attribute 1.0. - val pagerankGraph: Graph[(Double, Double), Double] = graph - // Associate the degree with each vertex - .outerJoinVertices(graph.outDegrees) { - (vid, vdata, deg) => deg.getOrElse(0) - } - // Set the weight on the edges based on the degree - .mapTriplets( e => 1.0 / e.srcAttr ) - // Set the vertex attributes to (initalPR, delta = 0) - .mapVertices( (id, attr) => (0.0, 0.0) ) - .cache() - - // Display statistics about pagerank - logInfo(pagerankGraph.statistics.toString) - - // Define the three functions needed to implement PageRank in the GraphX - // version of Pregel - def vertexProgram(id: VertexID, attr: (Double, Double), msgSum: Double): (Double, Double) = { - val (oldPR, lastDelta) = attr - val newPR = oldPR + (1.0 - resetProb) * msgSum - (newPR, newPR - oldPR) - } - - def sendMessage(edge: EdgeTriplet[(Double, Double), Double]) = { - if (edge.srcAttr._2 > tol) { - Iterator((edge.dstId, edge.srcAttr._2 * edge.attr)) - } else { - Iterator.empty - } - } - - def messageCombiner(a: Double, b: Double): Double = a + b - - // The initial message received by all vertices in PageRank - val initialMessage = resetProb / (1.0 - resetProb) - - // Execute a dynamic version of Pregel. - Pregel(pagerankGraph, initialMessage)(vertexProgram, sendMessage, messageCombiner) - .mapVertices((vid, attr) => attr._1) - } // end of deltaPageRank - -} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/SVDPlusPlus.scala b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/SVDPlusPlus.scala deleted file mode 100644 index 2a13553d79..0000000000 --- a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/SVDPlusPlus.scala +++ /dev/null @@ -1,107 +0,0 @@ -package org.apache.spark.graphx.algorithms - -import org.apache.spark.rdd._ -import org.apache.spark.graphx._ -import scala.util.Random -import org.apache.commons.math.linear._ - -class SVDPlusPlusConf( // SVDPlusPlus parameters - var rank: Int, - var maxIters: Int, - var minVal: Double, - var maxVal: Double, - var gamma1: Double, - var gamma2: Double, - var gamma6: Double, - var gamma7: Double) extends Serializable - -object SVDPlusPlus { - /** - * Implement SVD++ based on "Factorization Meets the Neighborhood: a Multifaceted Collaborative Filtering Model", - * paper is available at [[http://public.research.att.com/~volinsky/netflix/kdd08koren.pdf]]. - * The prediction rule is rui = u + bu + bi + qi*(pu + |N(u)|^(-0.5)*sum(y)), see the details on page 6. - * - * @param edges edges for constructing the graph - * - * @param conf SVDPlusPlus parameters - * - * @return a graph with vertex attributes containing the trained model - */ - - def run(edges: RDD[Edge[Double]], conf: SVDPlusPlusConf): (Graph[(RealVector, RealVector, Double, Double), Double], Double) = { - - // generate default vertex attribute - def defaultF(rank: Int): (RealVector, RealVector, Double, Double) = { - val v1 = new ArrayRealVector(rank) - val v2 = new ArrayRealVector(rank) - for (i <- 0 until rank) { - v1.setEntry(i, Random.nextDouble) - v2.setEntry(i, Random.nextDouble) - } - (v1, v2, 0.0, 0.0) - } - - // calculate global rating mean - edges.cache() - val (rs, rc) = edges.map(e => (e.attr, 1L)).reduce((a, b) => (a._1 + b._1, a._2 + b._2)) - val u = rs / rc - - // construct graph - var g = Graph.fromEdges(edges, defaultF(conf.rank)).cache() - - // calculate initial bias and norm - var t0 = g.mapReduceTriplets(et => - Iterator((et.srcId, (1L, et.attr)), (et.dstId, (1L, et.attr))), (g1: (Long, Double), g2: (Long, Double)) => (g1._1 + g2._1, g1._2 + g2._2)) - g = g.outerJoinVertices(t0) { (vid: VertexID, vd: (RealVector, RealVector, Double, Double), msg: Option[(Long, Double)]) => - (vd._1, vd._2, msg.get._2 / msg.get._1, 1.0 / scala.math.sqrt(msg.get._1)) - } - - def mapTrainF(conf: SVDPlusPlusConf, u: Double)(et: EdgeTriplet[(RealVector, RealVector, Double, Double), Double]) - : Iterator[(VertexID, (RealVector, RealVector, Double))] = { - val (usr, itm) = (et.srcAttr, et.dstAttr) - val (p, q) = (usr._1, itm._1) - var pred = u + usr._3 + itm._3 + q.dotProduct(usr._2) - pred = math.max(pred, conf.minVal) - pred = math.min(pred, conf.maxVal) - val err = et.attr - pred - val updateP = ((q.mapMultiply(err)).subtract(p.mapMultiply(conf.gamma7))).mapMultiply(conf.gamma2) - val updateQ = ((usr._2.mapMultiply(err)).subtract(q.mapMultiply(conf.gamma7))).mapMultiply(conf.gamma2) - val updateY = ((q.mapMultiply(err * usr._4)).subtract((itm._2).mapMultiply(conf.gamma7))).mapMultiply(conf.gamma2) - Iterator((et.srcId, (updateP, updateY, (err - conf.gamma6 * usr._3) * conf.gamma1)), - (et.dstId, (updateQ, updateY, (err - conf.gamma6 * itm._3) * conf.gamma1))) - } - - for (i <- 0 until conf.maxIters) { - // phase 1, calculate pu + |N(u)|^(-0.5)*sum(y) for user nodes - g.cache() - var t1 = g.mapReduceTriplets(et => Iterator((et.srcId, et.dstAttr._2)), (g1: RealVector, g2: RealVector) => g1.add(g2)) - g = g.outerJoinVertices(t1) { (vid: VertexID, vd: (RealVector, RealVector, Double, Double), msg: Option[RealVector]) => - if (msg.isDefined) (vd._1, vd._1.add(msg.get.mapMultiply(vd._4)), vd._3, vd._4) else vd - } - // phase 2, update p for user nodes and q, y for item nodes - g.cache() - val t2 = g.mapReduceTriplets(mapTrainF(conf, u), (g1: (RealVector, RealVector, Double), g2: (RealVector, RealVector, Double)) => - (g1._1.add(g2._1), g1._2.add(g2._2), g1._3 + g2._3)) - g = g.outerJoinVertices(t2) { (vid: VertexID, vd: (RealVector, RealVector, Double, Double), msg: Option[(RealVector, RealVector, Double)]) => - (vd._1.add(msg.get._1), vd._2.add(msg.get._2), vd._3 + msg.get._3, vd._4) - } - } - - // calculate error on training set - def mapTestF(conf: SVDPlusPlusConf, u: Double)(et: EdgeTriplet[(RealVector, RealVector, Double, Double), Double]): Iterator[(VertexID, Double)] = { - val (usr, itm) = (et.srcAttr, et.dstAttr) - val (p, q) = (usr._1, itm._1) - var pred = u + usr._3 + itm._3 + q.dotProduct(usr._2) - pred = math.max(pred, conf.minVal) - pred = math.min(pred, conf.maxVal) - val err = (et.attr - pred) * (et.attr - pred) - Iterator((et.dstId, err)) - } - g.cache() - val t3 = g.mapReduceTriplets(mapTestF(conf, u), (g1: Double, g2: Double) => g1 + g2) - g = g.outerJoinVertices(t3) { (vid: VertexID, vd: (RealVector, RealVector, Double, Double), msg: Option[Double]) => - if (msg.isDefined) (vd._1, vd._2, vd._3, msg.get) else vd - } - (g, u) - } -} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/StronglyConnectedComponents.scala b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/StronglyConnectedComponents.scala deleted file mode 100644 index 864f0ec57c..0000000000 --- a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/StronglyConnectedComponents.scala +++ /dev/null @@ -1,88 +0,0 @@ -package org.apache.spark.graphx.algorithms - -import scala.reflect.ClassTag - -import org.apache.spark.graphx._ - -object StronglyConnectedComponents { - - /** - * Compute the strongly connected component (SCC) of each vertex and return a graph with the - * vertex value containing the lowest vertex id in the SCC containing that vertex. - * - * @tparam VD the vertex attribute type (discarded in the computation) - * @tparam ED the edge attribute type (preserved in the computation) - * - * @param graph the graph for which to compute the SCC - * - * @return a graph with vertex attributes containing the smallest vertex id in each SCC - */ - def run[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED], numIter: Int): Graph[VertexID, ED] = { - - // the graph we update with final SCC ids, and the graph we return at the end - var sccGraph = graph.mapVertices { case (vid, _) => vid } - // graph we are going to work with in our iterations - var sccWorkGraph = graph.mapVertices { case (vid, _) => (vid, false) }.cache() - - var numVertices = sccWorkGraph.numVertices - var iter = 0 - while (sccWorkGraph.numVertices > 0 && iter < numIter) { - iter += 1 - do { - numVertices = sccWorkGraph.numVertices - sccWorkGraph = sccWorkGraph.outerJoinVertices(sccWorkGraph.outDegrees) { - (vid, data, degreeOpt) => if (degreeOpt.isDefined) data else (vid, true) - }.outerJoinVertices(sccWorkGraph.inDegrees) { - (vid, data, degreeOpt) => if (degreeOpt.isDefined) data else (vid, true) - }.cache() - - // get all vertices to be removed - val finalVertices = sccWorkGraph.vertices - .filter { case (vid, (scc, isFinal)) => isFinal} - .mapValues { (vid, data) => data._1} - - // write values to sccGraph - sccGraph = sccGraph.outerJoinVertices(finalVertices) { - (vid, scc, opt) => opt.getOrElse(scc) - } - // only keep vertices that are not final - sccWorkGraph = sccWorkGraph.subgraph(vpred = (vid, data) => !data._2).cache() - } while (sccWorkGraph.numVertices < numVertices) - - sccWorkGraph = sccWorkGraph.mapVertices{ case (vid, (color, isFinal)) => (vid, isFinal) } - - // collect min of all my neighbor's scc values, update if it's smaller than mine - // then notify any neighbors with scc values larger than mine - sccWorkGraph = GraphLab[(VertexID, Boolean), ED, VertexID](sccWorkGraph, Integer.MAX_VALUE)( - (vid, e) => e.otherVertexAttr(vid)._1, - (vid1, vid2) => math.min(vid1, vid2), - (vid, scc, optScc) => - (math.min(scc._1, optScc.getOrElse(scc._1)), scc._2), - (vid, e) => e.vertexAttr(vid)._1 < e.otherVertexAttr(vid)._1 - ) - - // start at root of SCCs. Traverse values in reverse, notify all my neighbors - // do not propagate if colors do not match! - sccWorkGraph = GraphLab[(VertexID, Boolean), ED, Boolean]( - sccWorkGraph, - Integer.MAX_VALUE, - EdgeDirection.Out, - EdgeDirection.In - )( - // vertex is final if it is the root of a color - // or it has the same color as a neighbor that is final - (vid, e) => (vid == e.vertexAttr(vid)._1) || (e.vertexAttr(vid)._1 == e.otherVertexAttr(vid)._1), - (final1, final2) => final1 || final2, - (vid, scc, optFinal) => - (scc._1, scc._2 || optFinal.getOrElse(false)), - // activate neighbor if they are not final, you are, and you have the same color - (vid, e) => e.vertexAttr(vid)._2 && - !e.otherVertexAttr(vid)._2 && (e.vertexAttr(vid)._1 == e.otherVertexAttr(vid)._1), - // start at root of colors - (vid, data) => vid == data._1 - ) - } - sccGraph - } - -} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/TriangleCount.scala b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/TriangleCount.scala deleted file mode 100644 index b5a93c1bd1..0000000000 --- a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/TriangleCount.scala +++ /dev/null @@ -1,78 +0,0 @@ -package org.apache.spark.graphx.algorithms - -import scala.reflect.ClassTag - -import org.apache.spark.graphx._ - - -object TriangleCount { - /** - * Compute the number of triangles passing through each vertex. - * - * The algorithm is relatively straightforward and can be computed in three steps: - * - * 1) Compute the set of neighbors for each vertex - * 2) For each edge compute the intersection of the sets and send the - * count to both vertices. - * 3) Compute the sum at each vertex and divide by two since each - * triangle is counted twice. - * - * - * @param graph a graph with `sourceId` less than `destId`. The graph must have been partitioned - * using Graph.partitionBy. - * - * @return - */ - def run[VD: ClassTag, ED: ClassTag](graph: Graph[VD,ED]): Graph[Int, ED] = { - // Remove redundant edges - val g = graph.groupEdges((a, b) => a).cache - - // Construct set representations of the neighborhoods - val nbrSets: VertexRDD[VertexSet] = - g.collectNeighborIds(EdgeDirection.Both).mapValues { (vid, nbrs) => - val set = new VertexSet(4) - var i = 0 - while (i < nbrs.size) { - // prevent self cycle - if(nbrs(i) != vid) { - set.add(nbrs(i)) - } - i += 1 - } - set - } - // join the sets with the graph - val setGraph: Graph[VertexSet, ED] = g.outerJoinVertices(nbrSets) { - (vid, _, optSet) => optSet.getOrElse(null) - } - // Edge function computes intersection of smaller vertex with larger vertex - def edgeFunc(et: EdgeTriplet[VertexSet, ED]): Iterator[(VertexID, Int)] = { - assert(et.srcAttr != null) - assert(et.dstAttr != null) - val (smallSet, largeSet) = if (et.srcAttr.size < et.dstAttr.size) { - (et.srcAttr, et.dstAttr) - } else { - (et.dstAttr, et.srcAttr) - } - val iter = smallSet.iterator - var counter: Int = 0 - while (iter.hasNext) { - val vid = iter.next - if (vid != et.srcId && vid != et.dstId && largeSet.contains(vid)) { counter += 1 } - } - Iterator((et.srcId, counter), (et.dstId, counter)) - } - // compute the intersection along edges - val counters: VertexRDD[Int] = setGraph.mapReduceTriplets(edgeFunc, _ + _) - // Merge counters with the graph and divide by two since each triangle is counted twice - g.outerJoinVertices(counters) { - (vid, _, optCounter: Option[Int]) => - val dblCount = optCounter.getOrElse(0) - // double count should be even (divisible by two) - assert((dblCount & 1) == 0) - dblCount / 2 - } - - } // end of TriangleCount - -} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/package.scala b/graphx/src/main/scala/org/apache/spark/graphx/algorithms/package.scala deleted file mode 100644 index fbabf1257c..0000000000 --- a/graphx/src/main/scala/org/apache/spark/graphx/algorithms/package.scala +++ /dev/null @@ -1,8 +0,0 @@ -package org.apache.spark.graphx - -import scala.reflect.ClassTag - -package object algorithms { - implicit def graphToAlgorithms[VD: ClassTag, ED: ClassTag]( - graph: Graph[VD, ED]): Algorithms[VD, ED] = new Algorithms(graph) -} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/Algorithms.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/Algorithms.scala new file mode 100644 index 0000000000..cbcd9c24a0 --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/Algorithms.scala @@ -0,0 +1,66 @@ +package org.apache.spark.graphx.lib + +import scala.reflect.ClassTag + +import org.apache.spark.graphx._ + +/** + * Provides graph algorithms directly on [[org.apache.spark.graphx.Graph]] via an implicit + * conversion. + * @example + * {{{ + * import org.apache.spark.graph.lib._ + * val graph: Graph[_, _] = loadGraph() + * graph.connectedComponents() + * }}} + */ +class Algorithms[VD: ClassTag, ED: ClassTag](self: Graph[VD, ED]) { + /** + * Run a dynamic version of PageRank returning a graph with vertex attributes containing the + * PageRank and edge attributes containing the normalized edge weight. + * + * @see [[org.apache.spark.graphx.lib.PageRank]], method `runUntilConvergence`. + */ + def pageRank(tol: Double, resetProb: Double = 0.15): Graph[Double, Double] = { + PageRank.runUntilConvergence(self, tol, resetProb) + } + + /** + * Run PageRank for a fixed number of iterations returning a graph with vertex attributes + * containing the PageRank and edge attributes the normalized edge weight. + * + * @see [[org.apache.spark.graphx.lib.PageRank]], method `run`. + */ + def staticPageRank(numIter: Int, resetProb: Double = 0.15): Graph[Double, Double] = { + PageRank.run(self, numIter, resetProb) + } + + /** + * Compute the connected component membership of each vertex and return a graph with the vertex + * value containing the lowest vertex id in the connected component containing that vertex. + * + * @see [[org.apache.spark.graphx.lib.ConnectedComponents]] + */ + def connectedComponents(): Graph[VertexID, ED] = { + ConnectedComponents.run(self) + } + + /** + * Compute the number of triangles passing through each vertex. + * + * @see [[org.apache.spark.graphx.lib.TriangleCount]] + */ + def triangleCount(): Graph[Int, ED] = { + TriangleCount.run(self) + } + + /** + * Compute the strongly connected component (SCC) of each vertex and return a graph with the + * vertex value containing the lowest vertex id in the SCC containing that vertex. + * + * @see [[org.apache.spark.graphx.lib.StronglyConnectedComponents]] + */ + def stronglyConnectedComponents(numIter: Int): Graph[VertexID, ED] = { + StronglyConnectedComponents.run(self, numIter) + } +} 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 new file mode 100644 index 0000000000..191e2aa051 --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/Analytics.scala @@ -0,0 +1,157 @@ +package org.apache.spark.graphx.lib + +import org.apache.spark._ +import org.apache.spark.graphx._ + +/** + * Driver program for running graph algorithms. + */ +object Analytics extends Logging { + + def main(args: Array[String]) = { + val host = args(0) + val taskType = args(1) + val fname = args(2) + val options = args.drop(3).map { arg => + arg.dropWhile(_ == '-').split('=') match { + case Array(opt, v) => (opt -> v) + case _ => throw new IllegalArgumentException("Invalid argument: " + arg) + } + } + + def setLogLevels(level: org.apache.log4j.Level, loggers: TraversableOnce[String]) = { + loggers.map{ + loggerName => + val logger = org.apache.log4j.Logger.getLogger(loggerName) + val prevLevel = logger.getLevel() + logger.setLevel(level) + loggerName -> prevLevel + }.toMap + } + + def pickPartitioner(v: String): PartitionStrategy = { + v match { + case "RandomVertexCut" => RandomVertexCut + case "EdgePartition1D" => EdgePartition1D + case "EdgePartition2D" => EdgePartition2D + case "CanonicalRandomVertexCut" => CanonicalRandomVertexCut + case _ => throw new IllegalArgumentException("Invalid Partition Strategy: " + v) + } + } + val serializer = "org.apache.spark.serializer.KryoSerializer" + System.setProperty("spark.serializer", serializer) + System.setProperty("spark.kryo.registrator", "org.apache.spark.graphx.GraphKryoRegistrator") + + taskType match { + case "pagerank" => { + + var tol:Float = 0.001F + var outFname = "" + var numVPart = 4 + var numEPart = 4 + var partitionStrategy: Option[PartitionStrategy] = None + + options.foreach{ + case ("tol", v) => tol = v.toFloat + case ("output", v) => outFname = v + case ("numVPart", v) => numVPart = v.toInt + case ("numEPart", v) => numEPart = v.toInt + case ("partStrategy", v) => partitionStrategy = Some(pickPartitioner(v)) + case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) + } + + println("======================================") + println("| PageRank |") + println("======================================") + + val sc = new SparkContext(host, "PageRank(" + fname + ")") + + val unpartitionedGraph = GraphLoader.edgeListFile(sc, fname, + minEdgePartitions = numEPart).cache() + val graph = partitionStrategy.foldLeft(unpartitionedGraph)(_.partitionBy(_)) + + println("GRAPHX: Number of vertices " + graph.vertices.count) + println("GRAPHX: Number of edges " + graph.edges.count) + + val pr = graph.pageRank(tol).vertices.cache() + + println("GRAPHX: Total rank: " + pr.map(_._2).reduce(_+_)) + + if (!outFname.isEmpty) { + logWarning("Saving pageranks of pages to " + outFname) + pr.map{case (id, r) => id + "\t" + r}.saveAsTextFile(outFname) + } + + sc.stop() + } + + case "cc" => { + + var numIter = Int.MaxValue + var numVPart = 4 + var numEPart = 4 + var isDynamic = false + var partitionStrategy: Option[PartitionStrategy] = None + + options.foreach{ + case ("numIter", v) => numIter = v.toInt + case ("dynamic", v) => isDynamic = v.toBoolean + case ("numEPart", v) => numEPart = v.toInt + case ("numVPart", v) => numVPart = v.toInt + case ("partStrategy", v) => partitionStrategy = Some(pickPartitioner(v)) + case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) + } + + if(!isDynamic && numIter == Int.MaxValue) { + println("Set number of iterations!") + sys.exit(1) + } + println("======================================") + println("| Connected Components |") + println("--------------------------------------") + println(" Using parameters:") + println(" \tDynamic: " + isDynamic) + println(" \tNumIter: " + numIter) + println("======================================") + + val sc = new SparkContext(host, "ConnectedComponents(" + fname + ")") + val unpartitionedGraph = GraphLoader.edgeListFile(sc, fname, + minEdgePartitions = numEPart).cache() + val graph = partitionStrategy.foldLeft(unpartitionedGraph)(_.partitionBy(_)) + + val cc = ConnectedComponents.run(graph) + println("Components: " + cc.vertices.map{ case (vid,data) => data}.distinct()) + sc.stop() + } + + case "triangles" => { + var numVPart = 4 + var numEPart = 4 + // TriangleCount requires the graph to be partitioned + var partitionStrategy: PartitionStrategy = RandomVertexCut + + options.foreach{ + case ("numEPart", v) => numEPart = v.toInt + case ("numVPart", v) => numVPart = v.toInt + case ("partStrategy", v) => partitionStrategy = pickPartitioner(v) + case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) + } + println("======================================") + println("| Triangle Count |") + println("--------------------------------------") + val sc = new SparkContext(host, "TriangleCount(" + fname + ")") + val graph = GraphLoader.edgeListFile(sc, fname, canonicalOrientation = true, + minEdgePartitions = numEPart).partitionBy(partitionStrategy).cache() + val triangles = TriangleCount.run(graph) + println("Triangles: " + triangles.vertices.map { + case (vid,data) => data.toLong + }.reduce(_+_) / 3) + sc.stop() + } + + case _ => { + println("Invalid task type.") + } + } + } +} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/ConnectedComponents.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/ConnectedComponents.scala new file mode 100644 index 0000000000..4a83e2dbb8 --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/ConnectedComponents.scala @@ -0,0 +1,39 @@ +package org.apache.spark.graphx.lib + +import scala.reflect.ClassTag + +import org.apache.spark.graphx._ + + +object ConnectedComponents { + /** + * Compute the connected component membership of each vertex and return a graph with the vertex + * value containing the lowest vertex id in the connected component containing that vertex. + * + * @tparam VD the vertex attribute type (discarded in the computation) + * @tparam ED the edge attribute type (preserved in the computation) + * + * @param graph the graph for which to compute the connected components + * + * @return a graph with vertex attributes containing the smallest vertex in each + * connected component + */ + def run[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]): Graph[VertexID, ED] = { + val ccGraph = graph.mapVertices { case (vid, _) => vid } + + def sendMessage(edge: EdgeTriplet[VertexID, ED]) = { + if (edge.srcAttr < edge.dstAttr) { + Iterator((edge.dstId, edge.srcAttr)) + } else if (edge.srcAttr > edge.dstAttr) { + Iterator((edge.srcId, edge.dstAttr)) + } else { + Iterator.empty + } + } + val initialMessage = Long.MaxValue + Pregel(ccGraph, initialMessage)( + vprog = (id, attr, msg) => math.min(attr, msg), + sendMsg = sendMessage, + mergeMsg = (a, b) => math.min(a, b)) + } // end of connectedComponents +} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala new file mode 100644 index 0000000000..809b6d0855 --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala @@ -0,0 +1,160 @@ +package org.apache.spark.graphx.lib + +import scala.reflect.ClassTag + +import org.apache.spark.Logging +import org.apache.spark.graphx._ + + +object PageRank extends Logging { + + /** + * Run PageRank for a fixed number of iterations returning a graph + * with vertex attributes containing the PageRank and edge + * attributes the normalized edge weight. + * + * The following PageRank fixed point is computed for each vertex. + * + * {{{ + * var PR = Array.fill(n)( 1.0 ) + * val oldPR = Array.fill(n)( 1.0 ) + * for( iter <- 0 until numIter ) { + * swap(oldPR, PR) + * for( i <- 0 until n ) { + * PR[i] = alpha + (1 - alpha) * inNbrs[i].map(j => oldPR[j] / outDeg[j]).sum + * } + * } + * }}} + * + * where `alpha` is the random reset probability (typically 0.15), + * `inNbrs[i]` is the set of neighbors whick link to `i` and + * `outDeg[j]` is the out degree of vertex `j`. + * + * Note that this is not the "normalized" PageRank and as a consequence pages that have no + * inlinks will have a PageRank of alpha. + * + * @tparam VD the original vertex attribute (not used) + * @tparam ED the original edge attribute (not used) + * + * @param graph the graph on which to compute PageRank + * @param numIter the number of iterations of PageRank to run + * @param resetProb the random reset probability (alpha) + * + * @return the graph containing with each vertex containing the PageRank and each edge + * containing the normalized weight. + * + */ + def run[VD: ClassTag, ED: ClassTag]( + graph: Graph[VD, ED], numIter: Int, resetProb: Double = 0.15): Graph[Double, Double] = + { + + /** + * Initialize the pagerankGraph with each edge attribute having + * weight 1/outDegree and each vertex with attribute 1.0. + */ + val pagerankGraph: Graph[Double, Double] = graph + // Associate the degree with each vertex + .outerJoinVertices(graph.outDegrees){ + (vid, vdata, deg) => deg.getOrElse(0) + } + // Set the weight on the edges based on the degree + .mapTriplets( e => 1.0 / e.srcAttr ) + // Set the vertex attributes to the initial pagerank values + .mapVertices( (id, attr) => 1.0 ) + .cache() + + // Display statistics about pagerank + logInfo(pagerankGraph.statistics.toString) + + // Define the three functions needed to implement PageRank in the GraphX + // version of Pregel + def vertexProgram(id: VertexID, attr: Double, msgSum: Double): Double = + resetProb + (1.0 - resetProb) * msgSum + def sendMessage(edge: EdgeTriplet[Double, Double]) = + Iterator((edge.dstId, edge.srcAttr * edge.attr)) + def messageCombiner(a: Double, b: Double): Double = a + b + // The initial message received by all vertices in PageRank + val initialMessage = 0.0 + + // Execute pregel for a fixed number of iterations. + Pregel(pagerankGraph, initialMessage, numIter)( + vertexProgram, sendMessage, messageCombiner) + } + + /** + * Run a dynamic version of PageRank returning a graph with vertex attributes containing the + * PageRank and edge attributes containing the normalized edge weight. + * + * {{{ + * var PR = Array.fill(n)( 1.0 ) + * val oldPR = Array.fill(n)( 0.0 ) + * while( max(abs(PR - oldPr)) > tol ) { + * swap(oldPR, PR) + * for( i <- 0 until n if abs(PR[i] - oldPR[i]) > tol ) { + * PR[i] = alpha + (1 - \alpha) * inNbrs[i].map(j => oldPR[j] / outDeg[j]).sum + * } + * } + * }}} + * + * where `alpha` is the random reset probability (typically 0.15), `inNbrs[i]` is the set of + * neighbors whick link to `i` and `outDeg[j]` is the out degree of vertex `j`. + * + * Note that this is not the "normalized" PageRank and as a consequence pages that have no + * inlinks will have a PageRank of alpha. + * + * @tparam VD the original vertex attribute (not used) + * @tparam ED the original edge attribute (not used) + * + * @param graph the graph on which to compute PageRank + * @param tol the tolerance allowed at convergence (smaller => more * accurate). + * @param resetProb the random reset probability (alpha) + * + * @return the graph containing with each vertex containing the PageRank and each edge + * containing the normalized weight. + */ + def runUntilConvergence[VD: ClassTag, ED: ClassTag]( + graph: Graph[VD, ED], tol: Double, resetProb: Double = 0.15): Graph[Double, Double] = + { + // Initialize the pagerankGraph with each edge attribute + // having weight 1/outDegree and each vertex with attribute 1.0. + val pagerankGraph: Graph[(Double, Double), Double] = graph + // Associate the degree with each vertex + .outerJoinVertices(graph.outDegrees) { + (vid, vdata, deg) => deg.getOrElse(0) + } + // Set the weight on the edges based on the degree + .mapTriplets( e => 1.0 / e.srcAttr ) + // Set the vertex attributes to (initalPR, delta = 0) + .mapVertices( (id, attr) => (0.0, 0.0) ) + .cache() + + // Display statistics about pagerank + logInfo(pagerankGraph.statistics.toString) + + // Define the three functions needed to implement PageRank in the GraphX + // version of Pregel + def vertexProgram(id: VertexID, attr: (Double, Double), msgSum: Double): (Double, Double) = { + val (oldPR, lastDelta) = attr + val newPR = oldPR + (1.0 - resetProb) * msgSum + (newPR, newPR - oldPR) + } + + def sendMessage(edge: EdgeTriplet[(Double, Double), Double]) = { + if (edge.srcAttr._2 > tol) { + Iterator((edge.dstId, edge.srcAttr._2 * edge.attr)) + } else { + Iterator.empty + } + } + + def messageCombiner(a: Double, b: Double): Double = a + b + + // The initial message received by all vertices in PageRank + val initialMessage = resetProb / (1.0 - resetProb) + + // Execute a dynamic version of Pregel. + Pregel(pagerankGraph, initialMessage)(vertexProgram, sendMessage, messageCombiner) + .mapVertices((vid, attr) => attr._1) + } // end of deltaPageRank + +} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala new file mode 100644 index 0000000000..f5570daec1 --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala @@ -0,0 +1,107 @@ +package org.apache.spark.graphx.lib + +import org.apache.spark.rdd._ +import org.apache.spark.graphx._ +import scala.util.Random +import org.apache.commons.math.linear._ + +class SVDPlusPlusConf( // SVDPlusPlus parameters + var rank: Int, + var maxIters: Int, + var minVal: Double, + var maxVal: Double, + var gamma1: Double, + var gamma2: Double, + var gamma6: Double, + var gamma7: Double) extends Serializable + +object SVDPlusPlus { + /** + * Implement SVD++ based on "Factorization Meets the Neighborhood: a Multifaceted Collaborative Filtering Model", + * paper is available at [[http://public.research.att.com/~volinsky/netflix/kdd08koren.pdf]]. + * The prediction rule is rui = u + bu + bi + qi*(pu + |N(u)|^(-0.5)*sum(y)), see the details on page 6. + * + * @param edges edges for constructing the graph + * + * @param conf SVDPlusPlus parameters + * + * @return a graph with vertex attributes containing the trained model + */ + + def run(edges: RDD[Edge[Double]], conf: SVDPlusPlusConf): (Graph[(RealVector, RealVector, Double, Double), Double], Double) = { + + // generate default vertex attribute + def defaultF(rank: Int): (RealVector, RealVector, Double, Double) = { + val v1 = new ArrayRealVector(rank) + val v2 = new ArrayRealVector(rank) + for (i <- 0 until rank) { + v1.setEntry(i, Random.nextDouble) + v2.setEntry(i, Random.nextDouble) + } + (v1, v2, 0.0, 0.0) + } + + // calculate global rating mean + edges.cache() + val (rs, rc) = edges.map(e => (e.attr, 1L)).reduce((a, b) => (a._1 + b._1, a._2 + b._2)) + val u = rs / rc + + // construct graph + var g = Graph.fromEdges(edges, defaultF(conf.rank)).cache() + + // calculate initial bias and norm + var t0 = g.mapReduceTriplets(et => + Iterator((et.srcId, (1L, et.attr)), (et.dstId, (1L, et.attr))), (g1: (Long, Double), g2: (Long, Double)) => (g1._1 + g2._1, g1._2 + g2._2)) + g = g.outerJoinVertices(t0) { (vid: VertexID, vd: (RealVector, RealVector, Double, Double), msg: Option[(Long, Double)]) => + (vd._1, vd._2, msg.get._2 / msg.get._1, 1.0 / scala.math.sqrt(msg.get._1)) + } + + def mapTrainF(conf: SVDPlusPlusConf, u: Double)(et: EdgeTriplet[(RealVector, RealVector, Double, Double), Double]) + : Iterator[(VertexID, (RealVector, RealVector, Double))] = { + val (usr, itm) = (et.srcAttr, et.dstAttr) + val (p, q) = (usr._1, itm._1) + var pred = u + usr._3 + itm._3 + q.dotProduct(usr._2) + pred = math.max(pred, conf.minVal) + pred = math.min(pred, conf.maxVal) + val err = et.attr - pred + val updateP = ((q.mapMultiply(err)).subtract(p.mapMultiply(conf.gamma7))).mapMultiply(conf.gamma2) + val updateQ = ((usr._2.mapMultiply(err)).subtract(q.mapMultiply(conf.gamma7))).mapMultiply(conf.gamma2) + val updateY = ((q.mapMultiply(err * usr._4)).subtract((itm._2).mapMultiply(conf.gamma7))).mapMultiply(conf.gamma2) + Iterator((et.srcId, (updateP, updateY, (err - conf.gamma6 * usr._3) * conf.gamma1)), + (et.dstId, (updateQ, updateY, (err - conf.gamma6 * itm._3) * conf.gamma1))) + } + + for (i <- 0 until conf.maxIters) { + // phase 1, calculate pu + |N(u)|^(-0.5)*sum(y) for user nodes + g.cache() + var t1 = g.mapReduceTriplets(et => Iterator((et.srcId, et.dstAttr._2)), (g1: RealVector, g2: RealVector) => g1.add(g2)) + g = g.outerJoinVertices(t1) { (vid: VertexID, vd: (RealVector, RealVector, Double, Double), msg: Option[RealVector]) => + if (msg.isDefined) (vd._1, vd._1.add(msg.get.mapMultiply(vd._4)), vd._3, vd._4) else vd + } + // phase 2, update p for user nodes and q, y for item nodes + g.cache() + val t2 = g.mapReduceTriplets(mapTrainF(conf, u), (g1: (RealVector, RealVector, Double), g2: (RealVector, RealVector, Double)) => + (g1._1.add(g2._1), g1._2.add(g2._2), g1._3 + g2._3)) + g = g.outerJoinVertices(t2) { (vid: VertexID, vd: (RealVector, RealVector, Double, Double), msg: Option[(RealVector, RealVector, Double)]) => + (vd._1.add(msg.get._1), vd._2.add(msg.get._2), vd._3 + msg.get._3, vd._4) + } + } + + // calculate error on training set + def mapTestF(conf: SVDPlusPlusConf, u: Double)(et: EdgeTriplet[(RealVector, RealVector, Double, Double), Double]): Iterator[(VertexID, Double)] = { + val (usr, itm) = (et.srcAttr, et.dstAttr) + val (p, q) = (usr._1, itm._1) + var pred = u + usr._3 + itm._3 + q.dotProduct(usr._2) + pred = math.max(pred, conf.minVal) + pred = math.min(pred, conf.maxVal) + val err = (et.attr - pred) * (et.attr - pred) + Iterator((et.dstId, err)) + } + g.cache() + val t3 = g.mapReduceTriplets(mapTestF(conf, u), (g1: Double, g2: Double) => g1 + g2) + g = g.outerJoinVertices(t3) { (vid: VertexID, vd: (RealVector, RealVector, Double, Double), msg: Option[Double]) => + if (msg.isDefined) (vd._1, vd._2, vd._3, msg.get) else vd + } + (g, u) + } +} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/StronglyConnectedComponents.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/StronglyConnectedComponents.scala new file mode 100644 index 0000000000..9bd227309a --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/StronglyConnectedComponents.scala @@ -0,0 +1,88 @@ +package org.apache.spark.graphx.lib + +import scala.reflect.ClassTag + +import org.apache.spark.graphx._ + +object StronglyConnectedComponents { + + /** + * Compute the strongly connected component (SCC) of each vertex and return a graph with the + * vertex value containing the lowest vertex id in the SCC containing that vertex. + * + * @tparam VD the vertex attribute type (discarded in the computation) + * @tparam ED the edge attribute type (preserved in the computation) + * + * @param graph the graph for which to compute the SCC + * + * @return a graph with vertex attributes containing the smallest vertex id in each SCC + */ + def run[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED], numIter: Int): Graph[VertexID, ED] = { + + // the graph we update with final SCC ids, and the graph we return at the end + var sccGraph = graph.mapVertices { case (vid, _) => vid } + // graph we are going to work with in our iterations + var sccWorkGraph = graph.mapVertices { case (vid, _) => (vid, false) }.cache() + + var numVertices = sccWorkGraph.numVertices + var iter = 0 + while (sccWorkGraph.numVertices > 0 && iter < numIter) { + iter += 1 + do { + numVertices = sccWorkGraph.numVertices + sccWorkGraph = sccWorkGraph.outerJoinVertices(sccWorkGraph.outDegrees) { + (vid, data, degreeOpt) => if (degreeOpt.isDefined) data else (vid, true) + }.outerJoinVertices(sccWorkGraph.inDegrees) { + (vid, data, degreeOpt) => if (degreeOpt.isDefined) data else (vid, true) + }.cache() + + // get all vertices to be removed + val finalVertices = sccWorkGraph.vertices + .filter { case (vid, (scc, isFinal)) => isFinal} + .mapValues { (vid, data) => data._1} + + // write values to sccGraph + sccGraph = sccGraph.outerJoinVertices(finalVertices) { + (vid, scc, opt) => opt.getOrElse(scc) + } + // only keep vertices that are not final + sccWorkGraph = sccWorkGraph.subgraph(vpred = (vid, data) => !data._2).cache() + } while (sccWorkGraph.numVertices < numVertices) + + sccWorkGraph = sccWorkGraph.mapVertices{ case (vid, (color, isFinal)) => (vid, isFinal) } + + // collect min of all my neighbor's scc values, update if it's smaller than mine + // then notify any neighbors with scc values larger than mine + sccWorkGraph = GraphLab[(VertexID, Boolean), ED, VertexID](sccWorkGraph, Integer.MAX_VALUE)( + (vid, e) => e.otherVertexAttr(vid)._1, + (vid1, vid2) => math.min(vid1, vid2), + (vid, scc, optScc) => + (math.min(scc._1, optScc.getOrElse(scc._1)), scc._2), + (vid, e) => e.vertexAttr(vid)._1 < e.otherVertexAttr(vid)._1 + ) + + // start at root of SCCs. Traverse values in reverse, notify all my neighbors + // do not propagate if colors do not match! + sccWorkGraph = GraphLab[(VertexID, Boolean), ED, Boolean]( + sccWorkGraph, + Integer.MAX_VALUE, + EdgeDirection.Out, + EdgeDirection.In + )( + // vertex is final if it is the root of a color + // or it has the same color as a neighbor that is final + (vid, e) => (vid == e.vertexAttr(vid)._1) || (e.vertexAttr(vid)._1 == e.otherVertexAttr(vid)._1), + (final1, final2) => final1 || final2, + (vid, scc, optFinal) => + (scc._1, scc._2 || optFinal.getOrElse(false)), + // activate neighbor if they are not final, you are, and you have the same color + (vid, e) => e.vertexAttr(vid)._2 && + !e.otherVertexAttr(vid)._2 && (e.vertexAttr(vid)._1 == e.otherVertexAttr(vid)._1), + // start at root of colors + (vid, data) => vid == data._1 + ) + } + sccGraph + } + +} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/TriangleCount.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/TriangleCount.scala new file mode 100644 index 0000000000..c6b1c736dd --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/TriangleCount.scala @@ -0,0 +1,78 @@ +package org.apache.spark.graphx.lib + +import scala.reflect.ClassTag + +import org.apache.spark.graphx._ + + +object TriangleCount { + /** + * Compute the number of triangles passing through each vertex. + * + * The algorithm is relatively straightforward and can be computed in three steps: + * + * 1) Compute the set of neighbors for each vertex + * 2) For each edge compute the intersection of the sets and send the + * count to both vertices. + * 3) Compute the sum at each vertex and divide by two since each + * triangle is counted twice. + * + * + * @param graph a graph with `sourceId` less than `destId`. The graph must have been partitioned + * using Graph.partitionBy. + * + * @return + */ + def run[VD: ClassTag, ED: ClassTag](graph: Graph[VD,ED]): Graph[Int, ED] = { + // Remove redundant edges + val g = graph.groupEdges((a, b) => a).cache + + // Construct set representations of the neighborhoods + val nbrSets: VertexRDD[VertexSet] = + g.collectNeighborIds(EdgeDirection.Both).mapValues { (vid, nbrs) => + val set = new VertexSet(4) + var i = 0 + while (i < nbrs.size) { + // prevent self cycle + if(nbrs(i) != vid) { + set.add(nbrs(i)) + } + i += 1 + } + set + } + // join the sets with the graph + val setGraph: Graph[VertexSet, ED] = g.outerJoinVertices(nbrSets) { + (vid, _, optSet) => optSet.getOrElse(null) + } + // Edge function computes intersection of smaller vertex with larger vertex + def edgeFunc(et: EdgeTriplet[VertexSet, ED]): Iterator[(VertexID, Int)] = { + assert(et.srcAttr != null) + assert(et.dstAttr != null) + val (smallSet, largeSet) = if (et.srcAttr.size < et.dstAttr.size) { + (et.srcAttr, et.dstAttr) + } else { + (et.dstAttr, et.srcAttr) + } + val iter = smallSet.iterator + var counter: Int = 0 + while (iter.hasNext) { + val vid = iter.next + if (vid != et.srcId && vid != et.dstId && largeSet.contains(vid)) { counter += 1 } + } + Iterator((et.srcId, counter), (et.dstId, counter)) + } + // compute the intersection along edges + val counters: VertexRDD[Int] = setGraph.mapReduceTriplets(edgeFunc, _ + _) + // Merge counters with the graph and divide by two since each triangle is counted twice + g.outerJoinVertices(counters) { + (vid, _, optCounter: Option[Int]) => + val dblCount = optCounter.getOrElse(0) + // double count should be even (divisible by two) + assert((dblCount & 1) == 0) + dblCount / 2 + } + + } // end of TriangleCount + +} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/package.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/package.scala new file mode 100644 index 0000000000..f6f2626c9d --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/package.scala @@ -0,0 +1,8 @@ +package org.apache.spark.graphx + +import scala.reflect.ClassTag + +package object lib { + implicit def graphToAlgorithms[VD: ClassTag, ED: ClassTag]( + graph: Graph[VD, ED]): Algorithms[VD, ED] = new Algorithms(graph) +} diff --git a/graphx/src/test/scala/org/apache/spark/graphx/algorithms/ConnectedComponentsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/ConnectedComponentsSuite.scala deleted file mode 100644 index 16fc3fe5a2..0000000000 --- a/graphx/src/test/scala/org/apache/spark/graphx/algorithms/ConnectedComponentsSuite.scala +++ /dev/null @@ -1,83 +0,0 @@ -package org.apache.spark.graphx.algorithms - -import org.scalatest.FunSuite - -import org.apache.spark.SparkContext -import org.apache.spark.SparkContext._ -import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.GraphGenerators -import org.apache.spark.rdd._ - - -class ConnectedComponentsSuite extends FunSuite with LocalSparkContext { - - test("Grid Connected Components") { - withSpark { sc => - val gridGraph = GraphGenerators.gridGraph(sc, 10, 10) - val ccGraph = gridGraph.connectedComponents() - val maxCCid = ccGraph.vertices.map { case (vid, ccId) => ccId }.sum - assert(maxCCid === 0) - } - } // end of Grid connected components - - - test("Reverse Grid Connected Components") { - withSpark { sc => - val gridGraph = GraphGenerators.gridGraph(sc, 10, 10).reverse - val ccGraph = gridGraph.connectedComponents() - val maxCCid = ccGraph.vertices.map { case (vid, ccId) => ccId }.sum - assert(maxCCid === 0) - } - } // end of Grid connected components - - - test("Chain Connected Components") { - withSpark { sc => - val chain1 = (0 until 9).map(x => (x, x+1) ) - val chain2 = (10 until 20).map(x => (x, x+1) ) - val rawEdges = sc.parallelize(chain1 ++ chain2, 3).map { case (s,d) => (s.toLong, d.toLong) } - val twoChains = Graph.fromEdgeTuples(rawEdges, 1.0) - val ccGraph = twoChains.connectedComponents() - val vertices = ccGraph.vertices.collect() - for ( (id, cc) <- vertices ) { - if(id < 10) { assert(cc === 0) } - else { assert(cc === 10) } - } - val ccMap = vertices.toMap - for (id <- 0 until 20) { - if (id < 10) { - assert(ccMap(id) === 0) - } else { - assert(ccMap(id) === 10) - } - } - } - } // end of chain connected components - - test("Reverse Chain Connected Components") { - withSpark { sc => - val chain1 = (0 until 9).map(x => (x, x+1) ) - val chain2 = (10 until 20).map(x => (x, x+1) ) - val rawEdges = sc.parallelize(chain1 ++ chain2, 3).map { case (s,d) => (s.toLong, d.toLong) } - val twoChains = Graph.fromEdgeTuples(rawEdges, true).reverse - val ccGraph = twoChains.connectedComponents() - val vertices = ccGraph.vertices.collect - for ( (id, cc) <- vertices ) { - if (id < 10) { - assert(cc === 0) - } else { - assert(cc === 10) - } - } - val ccMap = vertices.toMap - for ( id <- 0 until 20 ) { - if (id < 10) { - assert(ccMap(id) === 0) - } else { - assert(ccMap(id) === 10) - } - } - } - } // end of reverse chain connected components - -} diff --git a/graphx/src/test/scala/org/apache/spark/graphx/algorithms/PageRankSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/PageRankSuite.scala deleted file mode 100644 index de2c2d1107..0000000000 --- a/graphx/src/test/scala/org/apache/spark/graphx/algorithms/PageRankSuite.scala +++ /dev/null @@ -1,119 +0,0 @@ -package org.apache.spark.graphx.algorithms - -import org.scalatest.FunSuite - -import org.apache.spark.SparkContext -import org.apache.spark.SparkContext._ -import org.apache.spark.graphx._ -import org.apache.spark.graphx.algorithms._ -import org.apache.spark.graphx.util.GraphGenerators -import org.apache.spark.rdd._ - -object GridPageRank { - def apply(nRows: Int, nCols: Int, nIter: Int, resetProb: Double) = { - val inNbrs = Array.fill(nRows * nCols)(collection.mutable.MutableList.empty[Int]) - val outDegree = Array.fill(nRows * nCols)(0) - // Convert row column address into vertex ids (row major order) - def sub2ind(r: Int, c: Int): Int = r * nCols + c - // Make the grid graph - for (r <- 0 until nRows; c <- 0 until nCols) { - val ind = sub2ind(r,c) - if (r+1 < nRows) { - outDegree(ind) += 1 - inNbrs(sub2ind(r+1,c)) += ind - } - if (c+1 < nCols) { - outDegree(ind) += 1 - inNbrs(sub2ind(r,c+1)) += ind - } - } - // compute the pagerank - var pr = Array.fill(nRows * nCols)(resetProb) - for (iter <- 0 until nIter) { - val oldPr = pr - pr = new Array[Double](nRows * nCols) - for (ind <- 0 until (nRows * nCols)) { - pr(ind) = resetProb + (1.0 - resetProb) * - inNbrs(ind).map( nbr => oldPr(nbr) / outDegree(nbr)).sum - } - } - (0L until (nRows * nCols)).zip(pr) - } - -} - - -class PageRankSuite extends FunSuite with LocalSparkContext { - - def compareRanks(a: VertexRDD[Double], b: VertexRDD[Double]): Double = { - a.leftJoin(b) { case (id, a, bOpt) => (a - bOpt.getOrElse(0.0)) * (a - bOpt.getOrElse(0.0)) } - .map { case (id, error) => error }.sum - } - - test("Star PageRank") { - withSpark { sc => - val nVertices = 100 - val starGraph = GraphGenerators.starGraph(sc, nVertices).cache() - val resetProb = 0.15 - val errorTol = 1.0e-5 - - val staticRanks1 = starGraph.staticPageRank(numIter = 1, resetProb).vertices - val staticRanks2 = starGraph.staticPageRank(numIter = 2, resetProb).vertices.cache() - - // Static PageRank should only take 2 iterations to converge - val notMatching = staticRanks1.innerZipJoin(staticRanks2) { (vid, pr1, pr2) => - if (pr1 != pr2) 1 else 0 - }.map { case (vid, test) => test }.sum - assert(notMatching === 0) - - val staticErrors = staticRanks2.map { case (vid, pr) => - val correct = (vid > 0 && pr == resetProb) || - (vid == 0 && math.abs(pr - (resetProb + (1.0 - resetProb) * (resetProb * (nVertices - 1)) )) < 1.0E-5) - if (!correct) 1 else 0 - } - assert(staticErrors.sum === 0) - - val dynamicRanks = starGraph.pageRank(0, resetProb).vertices.cache() - assert(compareRanks(staticRanks2, dynamicRanks) < errorTol) - } - } // end of test Star PageRank - - - - test("Grid PageRank") { - withSpark { sc => - val rows = 10 - val cols = 10 - val resetProb = 0.15 - val tol = 0.0001 - val numIter = 50 - val errorTol = 1.0e-5 - val gridGraph = GraphGenerators.gridGraph(sc, rows, cols).cache() - - val staticRanks = gridGraph.staticPageRank(numIter, resetProb).vertices.cache() - val dynamicRanks = gridGraph.pageRank(tol, resetProb).vertices.cache() - val referenceRanks = VertexRDD(sc.parallelize(GridPageRank(rows, cols, numIter, resetProb))).cache() - - assert(compareRanks(staticRanks, referenceRanks) < errorTol) - assert(compareRanks(dynamicRanks, referenceRanks) < errorTol) - } - } // end of Grid PageRank - - - test("Chain PageRank") { - withSpark { sc => - val chain1 = (0 until 9).map(x => (x, x+1) ) - val rawEdges = sc.parallelize(chain1, 1).map { case (s,d) => (s.toLong, d.toLong) } - val chain = Graph.fromEdgeTuples(rawEdges, 1.0).cache() - val resetProb = 0.15 - val tol = 0.0001 - val numIter = 10 - val errorTol = 1.0e-5 - - val staticRanks = chain.staticPageRank(numIter, resetProb).vertices - val dynamicRanks = chain.pageRank(tol, resetProb).vertices - - assert(compareRanks(staticRanks, dynamicRanks) < errorTol) - } - } -} diff --git a/graphx/src/test/scala/org/apache/spark/graphx/algorithms/SVDPlusPlusSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/SVDPlusPlusSuite.scala deleted file mode 100644 index 7bd93e0e6c..0000000000 --- a/graphx/src/test/scala/org/apache/spark/graphx/algorithms/SVDPlusPlusSuite.scala +++ /dev/null @@ -1,31 +0,0 @@ -package org.apache.spark.graphx.algorithms - -import org.scalatest.FunSuite - -import org.apache.spark.SparkContext -import org.apache.spark.SparkContext._ -import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.GraphGenerators -import org.apache.spark.rdd._ - - -class SVDPlusPlusSuite extends FunSuite with LocalSparkContext { - - test("Test SVD++ with mean square error on training set") { - withSpark { sc => - val svdppErr = 8.0 - val edges = sc.textFile("mllib/data/als/test.data").map { line => - val fields = line.split(",") - Edge(fields(0).toLong * 2, fields(1).toLong * 2 + 1, fields(2).toDouble) - } - val conf = new SVDPlusPlusConf(10, 2, 0.0, 5.0, 0.007, 0.007, 0.005, 0.015) // 2 iterations - var (graph, u) = SVDPlusPlus.run(edges, conf) - graph.cache() - val err = graph.vertices.collect.map{ case (vid, vd) => - if (vid % 2 == 1) vd._4 else 0.0 - }.reduce(_ + _) / graph.triplets.collect.size - assert(err <= svdppErr) - } - } - -} diff --git a/graphx/src/test/scala/org/apache/spark/graphx/algorithms/StronglyConnectedComponentsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/StronglyConnectedComponentsSuite.scala deleted file mode 100644 index fee7d20161..0000000000 --- a/graphx/src/test/scala/org/apache/spark/graphx/algorithms/StronglyConnectedComponentsSuite.scala +++ /dev/null @@ -1,57 +0,0 @@ -package org.apache.spark.graphx.algorithms - -import org.scalatest.FunSuite - -import org.apache.spark.SparkContext -import org.apache.spark.SparkContext._ -import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.GraphGenerators -import org.apache.spark.rdd._ - - -class StronglyConnectedComponentsSuite extends FunSuite with LocalSparkContext { - - test("Island Strongly Connected Components") { - withSpark { sc => - val vertices = sc.parallelize((1L to 5L).map(x => (x, -1))) - val edges = sc.parallelize(Seq.empty[Edge[Int]]) - val graph = Graph(vertices, edges) - val sccGraph = graph.stronglyConnectedComponents(5) - for ((id, scc) <- sccGraph.vertices.collect) { - assert(id == scc) - } - } - } - - test("Cycle Strongly Connected Components") { - withSpark { sc => - val rawEdges = sc.parallelize((0L to 6L).map(x => (x, (x + 1) % 7))) - val graph = Graph.fromEdgeTuples(rawEdges, -1) - val sccGraph = graph.stronglyConnectedComponents(20) - for ((id, scc) <- sccGraph.vertices.collect) { - assert(0L == scc) - } - } - } - - test("2 Cycle Strongly Connected Components") { - withSpark { sc => - val edges = - Array(0L -> 1L, 1L -> 2L, 2L -> 0L) ++ - Array(3L -> 4L, 4L -> 5L, 5L -> 3L) ++ - Array(6L -> 0L, 5L -> 7L) - val rawEdges = sc.parallelize(edges) - val graph = Graph.fromEdgeTuples(rawEdges, -1) - val sccGraph = graph.stronglyConnectedComponents(20) - for ((id, scc) <- sccGraph.vertices.collect) { - if (id < 3) - assert(0L == scc) - else if (id < 6) - assert(3L == scc) - else - assert(id == scc) - } - } - } - -} diff --git a/graphx/src/test/scala/org/apache/spark/graphx/algorithms/TriangleCountSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/algorithms/TriangleCountSuite.scala deleted file mode 100644 index b85b289da6..0000000000 --- a/graphx/src/test/scala/org/apache/spark/graphx/algorithms/TriangleCountSuite.scala +++ /dev/null @@ -1,73 +0,0 @@ -package org.apache.spark.graphx.algorithms - -import org.scalatest.FunSuite - -import org.apache.spark.SparkContext -import org.apache.spark.SparkContext._ -import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.GraphGenerators -import org.apache.spark.rdd._ - - -class TriangleCountSuite extends FunSuite with LocalSparkContext { - - test("Count a single triangle") { - withSpark { sc => - val rawEdges = sc.parallelize(Array( 0L->1L, 1L->2L, 2L->0L ), 2) - val graph = Graph.fromEdgeTuples(rawEdges, true).cache() - val triangleCount = graph.triangleCount() - val verts = triangleCount.vertices - verts.collect.foreach { case (vid, count) => assert(count === 1) } - } - } - - test("Count two triangles") { - withSpark { sc => - val triangles = Array(0L -> 1L, 1L -> 2L, 2L -> 0L) ++ - Array(0L -> -1L, -1L -> -2L, -2L -> 0L) - val rawEdges = sc.parallelize(triangles, 2) - val graph = Graph.fromEdgeTuples(rawEdges, true).cache() - val triangleCount = graph.triangleCount() - val verts = triangleCount.vertices - verts.collect().foreach { case (vid, count) => - if (vid == 0) { - assert(count === 2) - } else { - assert(count === 1) - } - } - } - } - - test("Count two triangles with bi-directed edges") { - withSpark { sc => - val triangles = - Array(0L -> 1L, 1L -> 2L, 2L -> 0L) ++ - Array(0L -> -1L, -1L -> -2L, -2L -> 0L) - val revTriangles = triangles.map { case (a,b) => (b,a) } - val rawEdges = sc.parallelize(triangles ++ revTriangles, 2) - val graph = Graph.fromEdgeTuples(rawEdges, true).cache() - val triangleCount = graph.triangleCount() - val verts = triangleCount.vertices - verts.collect().foreach { case (vid, count) => - if (vid == 0) { - assert(count === 4) - } else { - assert(count === 2) - } - } - } - } - - test("Count a single triangle with duplicate edges") { - withSpark { sc => - val rawEdges = sc.parallelize(Array(0L -> 1L, 1L -> 2L, 2L -> 0L) ++ - Array(0L -> 1L, 1L -> 2L, 2L -> 0L), 2) - val graph = Graph.fromEdgeTuples(rawEdges, true, uniqueEdges = Some(RandomVertexCut)).cache() - val triangleCount = graph.triangleCount() - val verts = triangleCount.vertices - verts.collect.foreach { case (vid, count) => assert(count === 1) } - } - } - -} diff --git a/graphx/src/test/scala/org/apache/spark/graphx/lib/ConnectedComponentsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/lib/ConnectedComponentsSuite.scala new file mode 100644 index 0000000000..66612b381f --- /dev/null +++ b/graphx/src/test/scala/org/apache/spark/graphx/lib/ConnectedComponentsSuite.scala @@ -0,0 +1,83 @@ +package org.apache.spark.graphx.lib + +import org.scalatest.FunSuite + +import org.apache.spark.SparkContext +import org.apache.spark.SparkContext._ +import org.apache.spark.graphx._ +import org.apache.spark.graphx.util.GraphGenerators +import org.apache.spark.rdd._ + + +class ConnectedComponentsSuite extends FunSuite with LocalSparkContext { + + test("Grid Connected Components") { + withSpark { sc => + val gridGraph = GraphGenerators.gridGraph(sc, 10, 10) + val ccGraph = gridGraph.connectedComponents() + val maxCCid = ccGraph.vertices.map { case (vid, ccId) => ccId }.sum + assert(maxCCid === 0) + } + } // end of Grid connected components + + + test("Reverse Grid Connected Components") { + withSpark { sc => + val gridGraph = GraphGenerators.gridGraph(sc, 10, 10).reverse + val ccGraph = gridGraph.connectedComponents() + val maxCCid = ccGraph.vertices.map { case (vid, ccId) => ccId }.sum + assert(maxCCid === 0) + } + } // end of Grid connected components + + + test("Chain Connected Components") { + withSpark { sc => + val chain1 = (0 until 9).map(x => (x, x+1) ) + val chain2 = (10 until 20).map(x => (x, x+1) ) + val rawEdges = sc.parallelize(chain1 ++ chain2, 3).map { case (s,d) => (s.toLong, d.toLong) } + val twoChains = Graph.fromEdgeTuples(rawEdges, 1.0) + val ccGraph = twoChains.connectedComponents() + val vertices = ccGraph.vertices.collect() + for ( (id, cc) <- vertices ) { + if(id < 10) { assert(cc === 0) } + else { assert(cc === 10) } + } + val ccMap = vertices.toMap + for (id <- 0 until 20) { + if (id < 10) { + assert(ccMap(id) === 0) + } else { + assert(ccMap(id) === 10) + } + } + } + } // end of chain connected components + + test("Reverse Chain Connected Components") { + withSpark { sc => + val chain1 = (0 until 9).map(x => (x, x+1) ) + val chain2 = (10 until 20).map(x => (x, x+1) ) + val rawEdges = sc.parallelize(chain1 ++ chain2, 3).map { case (s,d) => (s.toLong, d.toLong) } + val twoChains = Graph.fromEdgeTuples(rawEdges, true).reverse + val ccGraph = twoChains.connectedComponents() + val vertices = ccGraph.vertices.collect + for ( (id, cc) <- vertices ) { + if (id < 10) { + assert(cc === 0) + } else { + assert(cc === 10) + } + } + val ccMap = vertices.toMap + for ( id <- 0 until 20 ) { + if (id < 10) { + assert(ccMap(id) === 0) + } else { + assert(ccMap(id) === 10) + } + } + } + } // end of reverse chain connected components + +} diff --git a/graphx/src/test/scala/org/apache/spark/graphx/lib/PageRankSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/lib/PageRankSuite.scala new file mode 100644 index 0000000000..fe7e4261f8 --- /dev/null +++ b/graphx/src/test/scala/org/apache/spark/graphx/lib/PageRankSuite.scala @@ -0,0 +1,119 @@ +package org.apache.spark.graphx.lib + +import org.scalatest.FunSuite + +import org.apache.spark.SparkContext +import org.apache.spark.SparkContext._ +import org.apache.spark.graphx._ +import org.apache.spark.graphx.lib._ +import org.apache.spark.graphx.util.GraphGenerators +import org.apache.spark.rdd._ + +object GridPageRank { + def apply(nRows: Int, nCols: Int, nIter: Int, resetProb: Double) = { + val inNbrs = Array.fill(nRows * nCols)(collection.mutable.MutableList.empty[Int]) + val outDegree = Array.fill(nRows * nCols)(0) + // Convert row column address into vertex ids (row major order) + def sub2ind(r: Int, c: Int): Int = r * nCols + c + // Make the grid graph + for (r <- 0 until nRows; c <- 0 until nCols) { + val ind = sub2ind(r,c) + if (r+1 < nRows) { + outDegree(ind) += 1 + inNbrs(sub2ind(r+1,c)) += ind + } + if (c+1 < nCols) { + outDegree(ind) += 1 + inNbrs(sub2ind(r,c+1)) += ind + } + } + // compute the pagerank + var pr = Array.fill(nRows * nCols)(resetProb) + for (iter <- 0 until nIter) { + val oldPr = pr + pr = new Array[Double](nRows * nCols) + for (ind <- 0 until (nRows * nCols)) { + pr(ind) = resetProb + (1.0 - resetProb) * + inNbrs(ind).map( nbr => oldPr(nbr) / outDegree(nbr)).sum + } + } + (0L until (nRows * nCols)).zip(pr) + } + +} + + +class PageRankSuite extends FunSuite with LocalSparkContext { + + def compareRanks(a: VertexRDD[Double], b: VertexRDD[Double]): Double = { + a.leftJoin(b) { case (id, a, bOpt) => (a - bOpt.getOrElse(0.0)) * (a - bOpt.getOrElse(0.0)) } + .map { case (id, error) => error }.sum + } + + test("Star PageRank") { + withSpark { sc => + val nVertices = 100 + val starGraph = GraphGenerators.starGraph(sc, nVertices).cache() + val resetProb = 0.15 + val errorTol = 1.0e-5 + + val staticRanks1 = starGraph.staticPageRank(numIter = 1, resetProb).vertices + val staticRanks2 = starGraph.staticPageRank(numIter = 2, resetProb).vertices.cache() + + // Static PageRank should only take 2 iterations to converge + val notMatching = staticRanks1.innerZipJoin(staticRanks2) { (vid, pr1, pr2) => + if (pr1 != pr2) 1 else 0 + }.map { case (vid, test) => test }.sum + assert(notMatching === 0) + + val staticErrors = staticRanks2.map { case (vid, pr) => + val correct = (vid > 0 && pr == resetProb) || + (vid == 0 && math.abs(pr - (resetProb + (1.0 - resetProb) * (resetProb * (nVertices - 1)) )) < 1.0E-5) + if (!correct) 1 else 0 + } + assert(staticErrors.sum === 0) + + val dynamicRanks = starGraph.pageRank(0, resetProb).vertices.cache() + assert(compareRanks(staticRanks2, dynamicRanks) < errorTol) + } + } // end of test Star PageRank + + + + test("Grid PageRank") { + withSpark { sc => + val rows = 10 + val cols = 10 + val resetProb = 0.15 + val tol = 0.0001 + val numIter = 50 + val errorTol = 1.0e-5 + val gridGraph = GraphGenerators.gridGraph(sc, rows, cols).cache() + + val staticRanks = gridGraph.staticPageRank(numIter, resetProb).vertices.cache() + val dynamicRanks = gridGraph.pageRank(tol, resetProb).vertices.cache() + val referenceRanks = VertexRDD(sc.parallelize(GridPageRank(rows, cols, numIter, resetProb))).cache() + + assert(compareRanks(staticRanks, referenceRanks) < errorTol) + assert(compareRanks(dynamicRanks, referenceRanks) < errorTol) + } + } // end of Grid PageRank + + + test("Chain PageRank") { + withSpark { sc => + val chain1 = (0 until 9).map(x => (x, x+1) ) + val rawEdges = sc.parallelize(chain1, 1).map { case (s,d) => (s.toLong, d.toLong) } + val chain = Graph.fromEdgeTuples(rawEdges, 1.0).cache() + val resetProb = 0.15 + val tol = 0.0001 + val numIter = 10 + val errorTol = 1.0e-5 + + val staticRanks = chain.staticPageRank(numIter, resetProb).vertices + val dynamicRanks = chain.pageRank(tol, resetProb).vertices + + assert(compareRanks(staticRanks, dynamicRanks) < errorTol) + } + } +} diff --git a/graphx/src/test/scala/org/apache/spark/graphx/lib/SVDPlusPlusSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/lib/SVDPlusPlusSuite.scala new file mode 100644 index 0000000000..a4a1cdab18 --- /dev/null +++ b/graphx/src/test/scala/org/apache/spark/graphx/lib/SVDPlusPlusSuite.scala @@ -0,0 +1,31 @@ +package org.apache.spark.graphx.lib + +import org.scalatest.FunSuite + +import org.apache.spark.SparkContext +import org.apache.spark.SparkContext._ +import org.apache.spark.graphx._ +import org.apache.spark.graphx.util.GraphGenerators +import org.apache.spark.rdd._ + + +class SVDPlusPlusSuite extends FunSuite with LocalSparkContext { + + test("Test SVD++ with mean square error on training set") { + withSpark { sc => + val svdppErr = 8.0 + val edges = sc.textFile("mllib/data/als/test.data").map { line => + val fields = line.split(",") + Edge(fields(0).toLong * 2, fields(1).toLong * 2 + 1, fields(2).toDouble) + } + val conf = new SVDPlusPlusConf(10, 2, 0.0, 5.0, 0.007, 0.007, 0.005, 0.015) // 2 iterations + var (graph, u) = SVDPlusPlus.run(edges, conf) + graph.cache() + val err = graph.vertices.collect.map{ case (vid, vd) => + if (vid % 2 == 1) vd._4 else 0.0 + }.reduce(_ + _) / graph.triplets.collect.size + assert(err <= svdppErr) + } + } + +} diff --git a/graphx/src/test/scala/org/apache/spark/graphx/lib/StronglyConnectedComponentsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/lib/StronglyConnectedComponentsSuite.scala new file mode 100644 index 0000000000..0458311661 --- /dev/null +++ b/graphx/src/test/scala/org/apache/spark/graphx/lib/StronglyConnectedComponentsSuite.scala @@ -0,0 +1,57 @@ +package org.apache.spark.graphx.lib + +import org.scalatest.FunSuite + +import org.apache.spark.SparkContext +import org.apache.spark.SparkContext._ +import org.apache.spark.graphx._ +import org.apache.spark.graphx.util.GraphGenerators +import org.apache.spark.rdd._ + + +class StronglyConnectedComponentsSuite extends FunSuite with LocalSparkContext { + + test("Island Strongly Connected Components") { + withSpark { sc => + val vertices = sc.parallelize((1L to 5L).map(x => (x, -1))) + val edges = sc.parallelize(Seq.empty[Edge[Int]]) + val graph = Graph(vertices, edges) + val sccGraph = graph.stronglyConnectedComponents(5) + for ((id, scc) <- sccGraph.vertices.collect) { + assert(id == scc) + } + } + } + + test("Cycle Strongly Connected Components") { + withSpark { sc => + val rawEdges = sc.parallelize((0L to 6L).map(x => (x, (x + 1) % 7))) + val graph = Graph.fromEdgeTuples(rawEdges, -1) + val sccGraph = graph.stronglyConnectedComponents(20) + for ((id, scc) <- sccGraph.vertices.collect) { + assert(0L == scc) + } + } + } + + test("2 Cycle Strongly Connected Components") { + withSpark { sc => + val edges = + Array(0L -> 1L, 1L -> 2L, 2L -> 0L) ++ + Array(3L -> 4L, 4L -> 5L, 5L -> 3L) ++ + Array(6L -> 0L, 5L -> 7L) + val rawEdges = sc.parallelize(edges) + val graph = Graph.fromEdgeTuples(rawEdges, -1) + val sccGraph = graph.stronglyConnectedComponents(20) + for ((id, scc) <- sccGraph.vertices.collect) { + if (id < 3) + assert(0L == scc) + else if (id < 6) + assert(3L == scc) + else + assert(id == scc) + } + } + } + +} diff --git a/graphx/src/test/scala/org/apache/spark/graphx/lib/TriangleCountSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/lib/TriangleCountSuite.scala new file mode 100644 index 0000000000..a286b7d03b --- /dev/null +++ b/graphx/src/test/scala/org/apache/spark/graphx/lib/TriangleCountSuite.scala @@ -0,0 +1,73 @@ +package org.apache.spark.graphx.lib + +import org.scalatest.FunSuite + +import org.apache.spark.SparkContext +import org.apache.spark.SparkContext._ +import org.apache.spark.graphx._ +import org.apache.spark.graphx.util.GraphGenerators +import org.apache.spark.rdd._ + + +class TriangleCountSuite extends FunSuite with LocalSparkContext { + + test("Count a single triangle") { + withSpark { sc => + val rawEdges = sc.parallelize(Array( 0L->1L, 1L->2L, 2L->0L ), 2) + val graph = Graph.fromEdgeTuples(rawEdges, true).cache() + val triangleCount = graph.triangleCount() + val verts = triangleCount.vertices + verts.collect.foreach { case (vid, count) => assert(count === 1) } + } + } + + test("Count two triangles") { + withSpark { sc => + val triangles = Array(0L -> 1L, 1L -> 2L, 2L -> 0L) ++ + Array(0L -> -1L, -1L -> -2L, -2L -> 0L) + val rawEdges = sc.parallelize(triangles, 2) + val graph = Graph.fromEdgeTuples(rawEdges, true).cache() + val triangleCount = graph.triangleCount() + val verts = triangleCount.vertices + verts.collect().foreach { case (vid, count) => + if (vid == 0) { + assert(count === 2) + } else { + assert(count === 1) + } + } + } + } + + test("Count two triangles with bi-directed edges") { + withSpark { sc => + val triangles = + Array(0L -> 1L, 1L -> 2L, 2L -> 0L) ++ + Array(0L -> -1L, -1L -> -2L, -2L -> 0L) + val revTriangles = triangles.map { case (a,b) => (b,a) } + val rawEdges = sc.parallelize(triangles ++ revTriangles, 2) + val graph = Graph.fromEdgeTuples(rawEdges, true).cache() + val triangleCount = graph.triangleCount() + val verts = triangleCount.vertices + verts.collect().foreach { case (vid, count) => + if (vid == 0) { + assert(count === 4) + } else { + assert(count === 2) + } + } + } + } + + test("Count a single triangle with duplicate edges") { + withSpark { sc => + val rawEdges = sc.parallelize(Array(0L -> 1L, 1L -> 2L, 2L -> 0L) ++ + Array(0L -> 1L, 1L -> 2L, 2L -> 0L), 2) + val graph = Graph.fromEdgeTuples(rawEdges, true, uniqueEdges = Some(RandomVertexCut)).cache() + val triangleCount = graph.triangleCount() + val verts = triangleCount.vertices + verts.collect.foreach { case (vid, count) => assert(count === 1) } + } + } + +} -- cgit v1.2.3 From 0b5c49ebad9dfb69074e2638c05a07b5ab94e13a Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Sat, 11 Jan 2014 00:34:23 -0800 Subject: Make nullValue and VertexSet package-private --- graphx/src/main/scala/org/apache/spark/graphx/package.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) (limited to 'graphx') diff --git a/graphx/src/main/scala/org/apache/spark/graphx/package.scala b/graphx/src/main/scala/org/apache/spark/graphx/package.scala index e6877d3195..2501314ca8 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/package.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/package.scala @@ -8,8 +8,8 @@ package object graphx { // TODO: Consider using Char. type PartitionID = Int - type VertexSet = OpenHashSet[VertexID] + private[graphx] type VertexSet = OpenHashSet[VertexID] - /** * Returns the default null-like value for a data type T. */ - def nullValue[T] = null.asInstanceOf[T] + /** Returns the default null-like value for a data type T. */ + private[graphx] def nullValue[T] = null.asInstanceOf[T] } -- cgit v1.2.3 From 732333d78e46ee23025d81ca9fbe6d1e13e9f253 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Sat, 11 Jan 2014 11:49:21 -0800 Subject: Remove GraphLab --- docs/graphx-programming-guide.md | 13 +- .../scala/org/apache/spark/graphx/GraphLab.scala | 138 --------------------- .../scala/org/apache/spark/graphx/Pregel.scala | 9 +- .../graphx/lib/StronglyConnectedComponents.scala | 50 ++++---- 4 files changed, 40 insertions(+), 170 deletions(-) delete mode 100644 graphx/src/main/scala/org/apache/spark/graphx/GraphLab.scala (limited to 'graphx') diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md index a5e75e2cb0..b19c6b69de 100644 --- a/docs/graphx-programming-guide.md +++ b/docs/graphx-programming-guide.md @@ -18,13 +18,12 @@ title: GraphX Programming Guide GraphX is the new (alpha) Spark API for graphs and graph-parallel computation. At a high-level, GraphX extends the Spark -[RDD](api/core/index.html#org.apache.spark.rdd.RDD) by -introducing the [Resilient Distributed property Graph (RDG)](#property_graph): -a directed graph with properties attached to each vertex and edge. -To support graph computation, GraphX exposes a set of functions -(e.g., [mapReduceTriplets](#mrTriplets)) as well as optimized variants of the -[Pregel](http://giraph.apache.org) and [GraphLab](http://graphlab.org) -APIs. In addition, GraphX includes a growing collection of graph +[RDD](api/core/index.html#org.apache.spark.rdd.RDD) by introducing the +[Resilient Distributed property Graph (RDG)](#property_graph): a directed graph +with properties attached to each vertex and edge. To support graph computation, +GraphX exposes a set of functions (e.g., [mapReduceTriplets](#mrTriplets)) as +well as an optimized variant of the [Pregel](http://giraph.apache.org) API. In +addition, GraphX includes a growing collection of graph [algorithms](#graph_algorithms) and [builders](#graph_builders) to simplify graph analytics tasks. diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphLab.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphLab.scala deleted file mode 100644 index 2f828ad807..0000000000 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphLab.scala +++ /dev/null @@ -1,138 +0,0 @@ -package org.apache.spark.graphx - -import scala.reflect.ClassTag - -import org.apache.spark.Logging -import scala.collection.JavaConversions._ -import org.apache.spark.rdd.RDD - -/** - * Implements the GraphLab gather-apply-scatter API. - */ -object GraphLab extends Logging { - - /** - * Executes the GraphLab Gather-Apply-Scatter API. - * - * @param graph the graph on which to execute the GraphLab API - * @param gatherFunc executed on each edge triplet - * adjacent to a vertex. Returns an accumulator which - * is then merged using the merge function. - * @param mergeFunc an accumulative associative operation on the result of - * the gather type. - * @param applyFunc takes a vertex and the final result of the merge operations - * on the adjacent edges and returns a new vertex value. - * @param scatterFunc executed after the apply function. Takes - * a triplet and signals whether the neighboring vertex program - * must be recomputed. - * @param startVertices a predicate to determine which vertices to start the computation on. - * These will be the active vertices in the first iteration. - * @param numIter the maximum number of iterations to run - * @param gatherDirection the direction of edges to consider during the gather phase - * @param scatterDirection the direction of edges to consider during the scatter phase - * - * @tparam VD the graph vertex attribute type - * @tparam ED the graph edge attribute type - * @tparam A the type accumulated during the gather phase - * @return the resulting graph after the algorithm converges - * - * @note Unlike [[Pregel]], this implementation of [[GraphLab]] does not unpersist RDDs from - * previous iterations. As a result, long-running iterative GraphLab programs will eventually fill - * the Spark cache. Though Spark will evict RDDs from old iterations eventually, garbage - * collection will take longer than necessary since it must examine the entire cache. This will be - * fixed in a future update. - */ - def apply[VD: ClassTag, ED: ClassTag, A: ClassTag] - (graph: Graph[VD, ED], numIter: Int, - gatherDirection: EdgeDirection = EdgeDirection.In, - scatterDirection: EdgeDirection = EdgeDirection.Out) - (gatherFunc: (VertexID, EdgeTriplet[VD, ED]) => A, - mergeFunc: (A, A) => A, - applyFunc: (VertexID, VD, Option[A]) => VD, - scatterFunc: (VertexID, EdgeTriplet[VD, ED]) => Boolean, - startVertices: (VertexID, VD) => Boolean = (vid: VertexID, data: VD) => true) - : Graph[VD, ED] = { - - - // Add an active attribute to all vertices to track convergence. - var activeGraph: Graph[(Boolean, VD), ED] = graph.mapVertices { - case (id, data) => (startVertices(id, data), data) - }.cache() - - // The gather function wrapper strips the active attribute and - // only invokes the gather function on active vertices - def gather(vid: VertexID, e: EdgeTriplet[(Boolean, VD), ED]): Option[A] = { - if (e.vertexAttr(vid)._1) { - val edgeTriplet = new EdgeTriplet[VD,ED] - edgeTriplet.set(e) - edgeTriplet.srcAttr = e.srcAttr._2 - edgeTriplet.dstAttr = e.dstAttr._2 - Some(gatherFunc(vid, edgeTriplet)) - } else { - None - } - } - - // The apply function wrapper strips the vertex of the active attribute - // and only invokes the apply function on active vertices - def apply(vid: VertexID, data: (Boolean, VD), accum: Option[A]): (Boolean, VD) = { - val (active, vData) = data - if (active) (true, applyFunc(vid, vData, accum)) - else (false, vData) - } - - // The scatter function wrapper strips the vertex of the active attribute - // and only invokes the scatter function on active vertices - def scatter(rawVertexID: VertexID, e: EdgeTriplet[(Boolean, VD), ED]): Option[Boolean] = { - val vid = e.otherVertexId(rawVertexID) - if (e.vertexAttr(vid)._1) { - val edgeTriplet = new EdgeTriplet[VD,ED] - edgeTriplet.set(e) - edgeTriplet.srcAttr = e.srcAttr._2 - edgeTriplet.dstAttr = e.dstAttr._2 - Some(scatterFunc(vid, edgeTriplet)) - } else { - None - } - } - - // Used to set the active status of vertices for the next round - def applyActive( - vid: VertexID, data: (Boolean, VD), newActiveOpt: Option[Boolean]): (Boolean, VD) = { - val (prevActive, vData) = data - (newActiveOpt.getOrElse(false), vData) - } - - // Main Loop --------------------------------------------------------------------- - var i = 0 - var numActive = activeGraph.numVertices - var prevActiveGraph: Graph[(Boolean, VD), ED] = null - while (i < numIter && numActive > 0) { - - // Gather - val gathered: RDD[(VertexID, A)] = - activeGraph.aggregateNeighbors(gather, mergeFunc, gatherDirection) - - // Apply - val applied = activeGraph.outerJoinVertices(gathered)(apply).cache() - - // Scatter is basically a gather in the opposite direction so we reverse the edge direction - val scattered: RDD[(VertexID, Boolean)] = - applied.aggregateNeighbors(scatter, _ || _, scatterDirection.reverse) - - prevActiveGraph = activeGraph - activeGraph = applied.outerJoinVertices(scattered)(applyActive).cache() - - // Calculate the number of active vertices. - numActive = activeGraph.vertices.map{ - case (vid, data) => if (data._1) 1 else 0 - }.reduce(_ + _) - logInfo("Number active vertices: " + numActive) - - i += 1 - } - - // Remove the active attribute from the vertex data before returning the graph - activeGraph.mapVertices{case (vid, data) => data._2 } - } -} 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 2e6453484c..57b087213f 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala @@ -65,6 +65,10 @@ object Pregel { * * @param maxIterations the maximum number of iterations to run for * + * @param activeDirection the direction of edges incident to a vertex that received a message in + * the previous round on which to run `sendMsg`. For example, if this is `EdgeDirection.Out`, only + * out-edges of vertices that received a message in the previous round will run. + * * @param vprog the user-defined vertex program which runs on each * vertex and receives the inbound message and computes a new vertex * value. On the first iteration the vertex program is invoked on @@ -85,7 +89,8 @@ object Pregel { * */ def apply[VD: ClassTag, ED: ClassTag, A: ClassTag] - (graph: Graph[VD, ED], initialMsg: A, maxIterations: Int = Int.MaxValue)( + (graph: Graph[VD, ED], initialMsg: A, maxIterations: Int = Int.MaxValue, + activeDirection: EdgeDirection = EdgeDirection.Out)( vprog: (VertexID, VD, A) => VD, sendMsg: EdgeTriplet[VD, ED] => Iterator[(VertexID,A)], mergeMsg: (A, A) => A) @@ -110,7 +115,7 @@ object Pregel { // Send new messages. Vertices that didn't get any messages don't appear in newVerts, so don't // get to send messages. We must cache messages so it can be materialized on the next line, // allowing us to uncache the previous iteration. - messages = g.mapReduceTriplets(sendMsg, mergeMsg, Some((newVerts, EdgeDirection.Out))).cache() + messages = g.mapReduceTriplets(sendMsg, mergeMsg, Some((newVerts, activeDirection))).cache() // The call to count() materializes `messages`, `newVerts`, and the vertices of `g`. This // hides oldMessages (depended on by newVerts), newVerts (depended on by messages), and the // vertices of prevG (depended on by newVerts, oldMessages, and the vertices of g). diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/StronglyConnectedComponents.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/StronglyConnectedComponents.scala index 9bd227309a..43c4b9cf2d 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/StronglyConnectedComponents.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/StronglyConnectedComponents.scala @@ -53,34 +53,38 @@ object StronglyConnectedComponents { // collect min of all my neighbor's scc values, update if it's smaller than mine // then notify any neighbors with scc values larger than mine - sccWorkGraph = GraphLab[(VertexID, Boolean), ED, VertexID](sccWorkGraph, Integer.MAX_VALUE)( - (vid, e) => e.otherVertexAttr(vid)._1, - (vid1, vid2) => math.min(vid1, vid2), - (vid, scc, optScc) => - (math.min(scc._1, optScc.getOrElse(scc._1)), scc._2), - (vid, e) => e.vertexAttr(vid)._1 < e.otherVertexAttr(vid)._1 - ) + sccWorkGraph = Pregel[(VertexID, Boolean), ED, VertexID](sccWorkGraph, Long.MaxValue)( + (vid, myScc, neighborScc) => (math.min(myScc._1, neighborScc), myScc._2), + e => { + if (e.srcId < e.dstId) { + Iterator((e.dstId, e.srcAttr._1)) + } else { + Iterator() + } + }, + (vid1, vid2) => math.min(vid1, vid2)) // start at root of SCCs. Traverse values in reverse, notify all my neighbors // do not propagate if colors do not match! - sccWorkGraph = GraphLab[(VertexID, Boolean), ED, Boolean]( - sccWorkGraph, - Integer.MAX_VALUE, - EdgeDirection.Out, - EdgeDirection.In - )( + sccWorkGraph = Pregel[(VertexID, Boolean), ED, Boolean]( + sccWorkGraph, false, activeDirection = EdgeDirection.In)( // vertex is final if it is the root of a color // or it has the same color as a neighbor that is final - (vid, e) => (vid == e.vertexAttr(vid)._1) || (e.vertexAttr(vid)._1 == e.otherVertexAttr(vid)._1), - (final1, final2) => final1 || final2, - (vid, scc, optFinal) => - (scc._1, scc._2 || optFinal.getOrElse(false)), - // activate neighbor if they are not final, you are, and you have the same color - (vid, e) => e.vertexAttr(vid)._2 && - !e.otherVertexAttr(vid)._2 && (e.vertexAttr(vid)._1 == e.otherVertexAttr(vid)._1), - // start at root of colors - (vid, data) => vid == data._1 - ) + (vid, myScc, existsSameColorFinalNeighbor) => { + val isColorRoot = vid == myScc._1 + (myScc._1, myScc._2 || isColorRoot || existsSameColorFinalNeighbor) + }, + // activate neighbor if they are not final, you are, and you have the same color + e => { + val sameColor = e.dstAttr._1 == e.srcAttr._1 + val onlyDstIsFinal = e.dstAttr._2 && !e.srcAttr._2 + if (sameColor && onlyDstIsFinal) { + Iterator((e.srcId, e.dstAttr._2)) + } else { + Iterator() + } + }, + (final1, final2) => final1 || final2) } sccGraph } -- cgit v1.2.3 From 64f73f73a012e99053069b624aaeeb2daa13ee22 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Sat, 11 Jan 2014 12:33:06 -0800 Subject: Fix indent and use SparkConf in Analytics --- .../org/apache/spark/graphx/lib/Analytics.scala | 245 ++++++++++----------- 1 file changed, 115 insertions(+), 130 deletions(-) (limited to 'graphx') 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 191e2aa051..8c35f4206e 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 @@ -19,139 +19,124 @@ object Analytics extends Logging { } } - def setLogLevels(level: org.apache.log4j.Level, loggers: TraversableOnce[String]) = { - loggers.map{ - loggerName => - val logger = org.apache.log4j.Logger.getLogger(loggerName) - val prevLevel = logger.getLevel() - logger.setLevel(level) - loggerName -> prevLevel - }.toMap - } - def pickPartitioner(v: String): PartitionStrategy = { v match { - case "RandomVertexCut" => RandomVertexCut - case "EdgePartition1D" => EdgePartition1D - case "EdgePartition2D" => EdgePartition2D - case "CanonicalRandomVertexCut" => CanonicalRandomVertexCut - case _ => throw new IllegalArgumentException("Invalid Partition Strategy: " + v) - } + case "RandomVertexCut" => RandomVertexCut + case "EdgePartition1D" => EdgePartition1D + case "EdgePartition2D" => EdgePartition2D + case "CanonicalRandomVertexCut" => CanonicalRandomVertexCut + case _ => throw new IllegalArgumentException("Invalid PartitionStrategy: " + v) + } } - val serializer = "org.apache.spark.serializer.KryoSerializer" - System.setProperty("spark.serializer", serializer) - System.setProperty("spark.kryo.registrator", "org.apache.spark.graphx.GraphKryoRegistrator") - - taskType match { - case "pagerank" => { - - var tol:Float = 0.001F - var outFname = "" - var numVPart = 4 - var numEPart = 4 - var partitionStrategy: Option[PartitionStrategy] = None - - options.foreach{ - case ("tol", v) => tol = v.toFloat - case ("output", v) => outFname = v - case ("numVPart", v) => numVPart = v.toInt - case ("numEPart", v) => numEPart = v.toInt - case ("partStrategy", v) => partitionStrategy = Some(pickPartitioner(v)) - case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) - } - - println("======================================") - println("| PageRank |") - println("======================================") - - val sc = new SparkContext(host, "PageRank(" + fname + ")") - - val unpartitionedGraph = GraphLoader.edgeListFile(sc, fname, - minEdgePartitions = numEPart).cache() - val graph = partitionStrategy.foldLeft(unpartitionedGraph)(_.partitionBy(_)) - - println("GRAPHX: Number of vertices " + graph.vertices.count) - println("GRAPHX: Number of edges " + graph.edges.count) - - val pr = graph.pageRank(tol).vertices.cache() - - println("GRAPHX: Total rank: " + pr.map(_._2).reduce(_+_)) - - if (!outFname.isEmpty) { - logWarning("Saving pageranks of pages to " + outFname) - pr.map{case (id, r) => id + "\t" + r}.saveAsTextFile(outFname) - } - - sc.stop() - } - - case "cc" => { - - var numIter = Int.MaxValue - var numVPart = 4 - var numEPart = 4 - var isDynamic = false - var partitionStrategy: Option[PartitionStrategy] = None - - options.foreach{ - case ("numIter", v) => numIter = v.toInt - case ("dynamic", v) => isDynamic = v.toBoolean - case ("numEPart", v) => numEPart = v.toInt - case ("numVPart", v) => numVPart = v.toInt - case ("partStrategy", v) => partitionStrategy = Some(pickPartitioner(v)) - case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) - } - - if(!isDynamic && numIter == Int.MaxValue) { - println("Set number of iterations!") - sys.exit(1) - } - println("======================================") - println("| Connected Components |") - println("--------------------------------------") - println(" Using parameters:") - println(" \tDynamic: " + isDynamic) - println(" \tNumIter: " + numIter) - println("======================================") - - val sc = new SparkContext(host, "ConnectedComponents(" + fname + ")") - val unpartitionedGraph = GraphLoader.edgeListFile(sc, fname, - minEdgePartitions = numEPart).cache() - val graph = partitionStrategy.foldLeft(unpartitionedGraph)(_.partitionBy(_)) - - val cc = ConnectedComponents.run(graph) - println("Components: " + cc.vertices.map{ case (vid,data) => data}.distinct()) - sc.stop() + + val conf = new SparkConf() + .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + .set("spark.kryo.registrator", "org.apache.spark.graphx.GraphKryoRegistrator") + + taskType match { + case "pagerank" => + var tol: Float = 0.001F + var outFname = "" + var numVPart = 4 + var numEPart = 4 + var partitionStrategy: Option[PartitionStrategy] = None + + options.foreach{ + case ("tol", v) => tol = v.toFloat + case ("output", v) => outFname = v + case ("numVPart", v) => numVPart = v.toInt + case ("numEPart", v) => numEPart = v.toInt + case ("partStrategy", v) => partitionStrategy = Some(pickPartitioner(v)) + case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) } - case "triangles" => { - var numVPart = 4 - var numEPart = 4 - // TriangleCount requires the graph to be partitioned - var partitionStrategy: PartitionStrategy = RandomVertexCut - - options.foreach{ - case ("numEPart", v) => numEPart = v.toInt - case ("numVPart", v) => numVPart = v.toInt - case ("partStrategy", v) => partitionStrategy = pickPartitioner(v) - case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) - } - println("======================================") - println("| Triangle Count |") - println("--------------------------------------") - val sc = new SparkContext(host, "TriangleCount(" + fname + ")") - val graph = GraphLoader.edgeListFile(sc, fname, canonicalOrientation = true, - minEdgePartitions = numEPart).partitionBy(partitionStrategy).cache() - val triangles = TriangleCount.run(graph) - println("Triangles: " + triangles.vertices.map { - case (vid,data) => data.toLong - }.reduce(_+_) / 3) - sc.stop() - } - - case _ => { - println("Invalid task type.") - } - } - } + println("======================================") + println("| PageRank |") + println("======================================") + + val sc = new SparkContext(host, "PageRank(" + fname + ")", conf) + + val unpartitionedGraph = GraphLoader.edgeListFile(sc, fname, + minEdgePartitions = numEPart).cache() + val graph = partitionStrategy.foldLeft(unpartitionedGraph)(_.partitionBy(_)) + + println("GRAPHX: Number of vertices " + graph.vertices.count) + println("GRAPHX: Number of edges " + graph.edges.count) + + val pr = graph.pageRank(tol).vertices.cache() + + println("GRAPHX: Total rank: " + pr.map(_._2).reduce(_+_)) + + if (!outFname.isEmpty) { + logWarning("Saving pageranks of pages to " + outFname) + pr.map{case (id, r) => id + "\t" + r}.saveAsTextFile(outFname) + } + + sc.stop() + + case "cc" => + var numIter = Int.MaxValue + var numVPart = 4 + var numEPart = 4 + var isDynamic = false + var partitionStrategy: Option[PartitionStrategy] = None + + options.foreach{ + case ("numIter", v) => numIter = v.toInt + case ("dynamic", v) => isDynamic = v.toBoolean + case ("numEPart", v) => numEPart = v.toInt + case ("numVPart", v) => numVPart = v.toInt + case ("partStrategy", v) => partitionStrategy = Some(pickPartitioner(v)) + case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) + } + + if(!isDynamic && numIter == Int.MaxValue) { + println("Set number of iterations!") + sys.exit(1) + } + println("======================================") + println("| Connected Components |") + println("--------------------------------------") + println(" Using parameters:") + println(" \tDynamic: " + isDynamic) + println(" \tNumIter: " + numIter) + println("======================================") + + val sc = new SparkContext(host, "ConnectedComponents(" + fname + ")", conf) + val unpartitionedGraph = GraphLoader.edgeListFile(sc, fname, + minEdgePartitions = numEPart).cache() + val graph = partitionStrategy.foldLeft(unpartitionedGraph)(_.partitionBy(_)) + + val cc = ConnectedComponents.run(graph) + println("Components: " + cc.vertices.map{ case (vid,data) => data}.distinct()) + sc.stop() + + case "triangles" => + var numVPart = 4 + var numEPart = 4 + // TriangleCount requires the graph to be partitioned + var partitionStrategy: PartitionStrategy = RandomVertexCut + + options.foreach{ + case ("numEPart", v) => numEPart = v.toInt + case ("numVPart", v) => numVPart = v.toInt + case ("partStrategy", v) => partitionStrategy = pickPartitioner(v) + case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) + } + println("======================================") + println("| Triangle Count |") + println("--------------------------------------") + val sc = new SparkContext(host, "TriangleCount(" + fname + ")", conf) + val graph = GraphLoader.edgeListFile(sc, fname, canonicalOrientation = true, + minEdgePartitions = numEPart).partitionBy(partitionStrategy).cache() + val triangles = TriangleCount.run(graph) + println("Triangles: " + triangles.vertices.map { + case (vid,data) => data.toLong + }.reduce(_ + _) / 3) + sc.stop() + + case _ => + println("Invalid task type.") + } + } } -- cgit v1.2.3 From 55101f5821c01b7f8f355582782761f412f4fb7e Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Sat, 11 Jan 2014 12:35:41 -0800 Subject: One-line Scaladoc comments in Edge and EdgeDirection --- .../src/main/scala/org/apache/spark/graphx/Edge.scala | 19 +++++++------------ .../scala/org/apache/spark/graphx/EdgeDirection.scala | 13 +++---------- 2 files changed, 10 insertions(+), 22 deletions(-) (limited to 'graphx') diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala b/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala index b3a2a21862..85f27d2c8d 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala @@ -8,18 +8,13 @@ package org.apache.spark.graphx * @tparam ED type of the edge attribute */ case class Edge[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED] ( - /** - * The vertex id of the source vertex - */ - var srcId: VertexID = 0, - /** - * The vertex id of the target vertex. - */ - var dstId: VertexID = 0, - /** - * The attribute associated with the edge. - */ - var attr: ED = nullValue[ED]) extends Serializable { + /** The vertex id of the source vertex */ + var srcId: VertexID = 0, + /** The vertex id of the target vertex. */ + var dstId: VertexID = 0, + /** The attribute associated with the edge. */ + var attr: ED = nullValue[ED]) + extends Serializable { /** * Given one vertex in the edge return the other vertex. diff --git a/graphx/src/main/scala/org/apache/spark/graphx/EdgeDirection.scala b/graphx/src/main/scala/org/apache/spark/graphx/EdgeDirection.scala index 99910ab87b..05103f6883 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/EdgeDirection.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeDirection.scala @@ -1,6 +1,5 @@ package org.apache.spark.graphx - /** * The direction of a directed edge relative to a vertex. */ @@ -18,18 +17,12 @@ sealed abstract class EdgeDirection { object EdgeDirection { - /** - * Edges arriving at a vertex. - */ + /** Edges arriving at a vertex. */ case object In extends EdgeDirection - /** - * Edges originating from a vertex - */ + /** Edges originating from a vertex. */ case object Out extends EdgeDirection - /** - * All edges adjacent to a vertex - */ + /** All edges adjacent to a vertex. */ case object Both extends EdgeDirection } -- cgit v1.2.3 From 574c0d28c2e859a6ed31a2c193cf04e0aa7404a5 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Sat, 11 Jan 2014 12:38:35 -0800 Subject: Use SparkConf in GraphX tests (via LocalSparkContext) --- .../test/scala/org/apache/spark/graphx/LocalSparkContext.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) (limited to 'graphx') diff --git a/graphx/src/test/scala/org/apache/spark/graphx/LocalSparkContext.scala b/graphx/src/test/scala/org/apache/spark/graphx/LocalSparkContext.scala index 6aec2ea8a9..aa9ba84084 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/LocalSparkContext.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/LocalSparkContext.scala @@ -3,20 +3,20 @@ package org.apache.spark.graphx import org.scalatest.Suite import org.scalatest.BeforeAndAfterEach +import org.apache.spark.SparkConf import org.apache.spark.SparkContext - /** * Provides a method to run tests against a {@link SparkContext} variable that is correctly stopped * after each test. */ trait LocalSparkContext { - System.setProperty("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - System.setProperty("spark.kryo.registrator", "org.apache.spark.graphx.GraphKryoRegistrator") - /** Runs `f` on a new SparkContext and ensures that it is stopped afterwards. */ def withSpark[T](f: SparkContext => T) = { - val sc = new SparkContext("local", "test") + val conf = new SparkConf() + .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + .set("spark.kryo.registrator", "org.apache.spark.graphx.GraphKryoRegistrator") + val sc = new SparkContext("local", "test", conf) try { f(sc) } finally { -- cgit v1.2.3 From 02771aa087f1ee8f8e766f85d4092f4fc040f89f Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Sat, 11 Jan 2014 13:15:46 -0800 Subject: Make EdgeDirection val instead of case object for Java compat. --- .../scala/org/apache/spark/graphx/EdgeDirection.scala | 17 +++++++++++++---- .../org/apache/spark/graphx/GraphKryoRegistrator.scala | 1 + .../scala/org/apache/spark/graphx/impl/GraphImpl.scala | 2 +- 3 files changed, 15 insertions(+), 5 deletions(-) (limited to 'graphx') diff --git a/graphx/src/main/scala/org/apache/spark/graphx/EdgeDirection.scala b/graphx/src/main/scala/org/apache/spark/graphx/EdgeDirection.scala index 05103f6883..9d37f6513f 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/EdgeDirection.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeDirection.scala @@ -3,7 +3,7 @@ package org.apache.spark.graphx /** * The direction of a directed edge relative to a vertex. */ -sealed abstract class EdgeDirection { +class EdgeDirection private (private val name: String) extends Serializable { /** * Reverse the direction of an edge. An in becomes out, * out becomes in and both remains both. @@ -13,16 +13,25 @@ sealed abstract class EdgeDirection { case EdgeDirection.Out => EdgeDirection.In case EdgeDirection.Both => EdgeDirection.Both } + + override def toString: String = "EdgeDirection." + name + + override def equals(o: Any) = o match { + case other: EdgeDirection => other.name == name + case _ => false + } + + override def hashCode = name.hashCode } object EdgeDirection { /** Edges arriving at a vertex. */ - case object In extends EdgeDirection + final val In = new EdgeDirection("In") /** Edges originating from a vertex. */ - case object Out extends EdgeDirection + final val Out = new EdgeDirection("Out") /** All edges adjacent to a vertex. */ - case object Both extends EdgeDirection + final val Both = new EdgeDirection("Both") } 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 681074ba10..d79bdf9618 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala @@ -23,6 +23,7 @@ class GraphKryoRegistrator extends KryoRegistrator { kryo.register(classOf[VertexAttributeBlock[Object]]) 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) 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 c66b8c804f..6a2abc71cc 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 @@ -283,7 +283,7 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( } case Some(EdgeDirection.In) => edgePartition.iterator.filter(e => vPart.isActive(e.dstId)) - case None => + case _ => // None edgePartition.iterator } -- cgit v1.2.3 From f096f4eaf1f8e936eafc2006ecd01faa2f208cf2 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Sun, 12 Jan 2014 10:55:29 -0800 Subject: Link methods in programming guide; document VertexID --- docs/graphx-programming-guide.md | 155 ++++++++++++--------- .../scala/org/apache/spark/graphx/package.scala | 4 + 2 files changed, 90 insertions(+), 69 deletions(-) (limited to 'graphx') diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md index 9a7c4ac179..7f93754edb 100644 --- a/docs/graphx-programming-guide.md +++ b/docs/graphx-programming-guide.md @@ -68,15 +68,14 @@ in graph-parallel systems, GraphX is able to optimize the execution of graph ope Prior to the release of GraphX, graph computation in Spark was expressed using Bagel, an implementation of Pregel. GraphX improves upon Bagel by exposing a richer property graph API, a more streamlined version of the Pregel abstraction, and system optimizations to improve performance -and reduce memory overhead. While we plan to eventually deprecate the Bagel, we will continue to -support the [Bagel API](api/bagel/index.html#org.apache.spark.bagel.package) and [Bagel programming -guide](bagel-programming-guide.html). However, we encourage Bagel users to explore the new GraphX -API and comment on issues that may complicate the transition from Bagel. +and reduce memory overhead. While we plan to eventually deprecate Bagel, we will continue to +support the [Bagel API](api/bagel/index.html#org.apache.spark.bagel.package) and +[Bagel programming guide](bagel-programming-guide.html). However, we encourage Bagel users to +explore the new GraphX API and comment on issues that may complicate the transition from Bagel. # Getting Started -To get started you first need to import Spark and GraphX into your project. This can be done by -importing the following: +To get started you first need to import Spark and GraphX into your project, as follows: {% highlight scala %} import org.apache.spark._ @@ -89,11 +88,11 @@ If you are not using the Spark shell you will also need a Spark context. The [property graph](api/graphx/index.html#org.apache.spark.graphx.Graph) is a directed multigraph -graph with user defined objects attached to each vertex and edge. A directed multigraph is a -directed graph with potentially multiple parallel edges sharing the same source and destination -vertex. The ability to support parallel edges simplifies modeling scenarios where there can be -multiple relationships (e.g., co-worker and friend) between the same vertices. Each vertex is keyed -by a *unique* 64-bit long identifier (`VertexId`). Similarly, edges have corresponding source and +with user defined objects attached to each vertex and edge. A directed multigraph is a directed +graph with potentially multiple parallel edges sharing the same source and destination vertex. The +ability to support parallel edges simplifies modeling scenarios where there can be multiple +relationships (e.g., co-worker and friend) between the same vertices. Each vertex is keyed by a +*unique* 64-bit long identifier (`VertexId`). Similarly, edges have corresponding source and destination vertex identifiers. GraphX does not impose any ordering or constraints on the vertex identifiers. The property graph is parameterized over the vertex `VD` and edge `ED` types. These are the types of the objects associated with each vertex and edge respectively. @@ -102,8 +101,8 @@ are the types of the objects associated with each vertex and edge respectively. > int, double, etc...) reducing the in memory footprint. In some cases we may wish to have vertices with different property types in the same graph. This can -be accomplished through inheritance. For example to model users and products as a bipartie graph we -might do the following: +be accomplished through inheritance. For example to model users and products as a bipartite graph +we might do the following: {% highlight scala %} case class VertexProperty @@ -159,8 +158,8 @@ val userGraph: Graph[(String, String), String] There are numerous ways to construct a property graph from raw files, RDDs, and even synthetic generators and these are discussed in more detail in the section on [graph builders](#graph_builders). Probably the most general method is to use the -[graph singleton](api/graphx/index.html#org.apache.spark.graphx.Graph$). -For example the following code constructs a graph from a collection of RDDs: +[Graph object](api/graphx/index.html#org.apache.spark.graphx.Graph$). For example the following +code constructs a graph from a collection of RDDs: {% highlight scala %} // Assume the SparkContext has already been constructed @@ -179,10 +178,11 @@ val defaultUser = ("John Doe", "Missing") val graph = Graph(users, relationships, defaultUser) {% endhighlight %} -In the above example we make use of the [`Edge`](api/graphx/index.html#org.apache.spark.graphx.Edge) -case class. Edges have a `srcId` and a `dstId` corresponding to the source and destination vertex -identifiers. In addition, the `Edge` class contains the `attr` member which contains the edge -property. +In the above example we make use of the [`Edge`][Edge] case class. Edges have a `srcId` and a +`dstId` corresponding to the source and destination vertex identifiers. In addition, the `Edge` +class contains the `attr` member which contains the edge property. + +[Edge]: api/graphx/index.html#org.apache.spark.graphx.Edge We can deconstruct a graph into the respective vertex and edge views by using the `graph.vertices` and `graph.edges` members respectively. @@ -196,18 +196,19 @@ graph.edges.filter(e => e.srcId > e.dstId).count {% endhighlight %} > Note that `graph.vertices` returns an `VertexRDD[(String, String)]` which extends -> `RDD[(VertexId, (String, String))]` and so we use the scala `case` expression to deconstruct -> the tuple. Alternatively, `graph.edges` returns an `EdgeRDD` containing `Edge[String]` objects. +> `RDD[(VertexId, (String, String))]` and so we use the scala `case` expression to deconstruct the +> tuple. On the other hand, `graph.edges` returns an `EdgeRDD` containing `Edge[String]` objects. > We could have also used the case class type constructor as in the following: > {% highlight scala %} graph.edges.filter { case Edge(src, dst, prop) => src < dst }.count {% endhighlight %} In addition to the vertex and edge views of the property graph, GraphX also exposes a triplet view. -The triplet view logically joins the vertex and edge properties yielding an `RDD[EdgeTriplet[VD, -ED]]` containing instances of the -[`EdgeTriplet`](api/graphx/index.html#org.apache.spark.graphx.EdgeTriplet) class. This *join* can be -expressed in the following SQL expression: +The triplet view logically joins the vertex and edge properties yielding an +`RDD[EdgeTriplet[VD, ED]]` containing instances of the [`EdgeTriplet`][EdgeTriplet] class. This +*join* can be expressed in the following SQL expression: + +[EdgeTriplet]: api/graphx/index.html#org.apache.spark.graphx.EdgeTriplet {% highlight sql %} SELECT src.id, dst.id, src.attr, e.attr, dst.attr @@ -225,8 +226,7 @@ or graphically as:

-The [`EdgeTriplet`](api/graphx/index.html#org.apache.spark.graphx.EdgeTriplet) class extends the -[`Edge`](api/graphx/index.html#org.apache.spark.graphx.Edge) class by adding the `srcAttr` and +The [`EdgeTriplet`][EdgeTriplet] class extends the [`Edge`][Edge] class by adding the `srcAttr` and `dstAttr` members which contain the source and destination properties respectively. We can use the triplet view of a graph to render a collection of strings describing relationships between users. @@ -240,14 +240,15 @@ val facts: RDD[String] = # Graph Operators Just as RDDs have basic operations like `map`, `filter`, and `reduceByKey`, property graphs also -have a collection of basic operators that take user defined function and produce new graphs with +have a collection of basic operators that take user defined functions and produce new graphs with transformed properties and structure. The core operators that have optimized implementations are -defined in [`Graph.scala`](api/graphx/index.html#org.apache.spark.graphx.Graph) and convenient -operators that are expressed as a compositions of the core operators are defined in -['GraphOps.scala'](api/graphx/index.html#org.apache.spark.graphx.GraphOps). However, thanks to -Scala implicits the operators in `GraphOps.scala` are automatically available as members of -`Graph.scala`. For example, we can compute the in-degree of each vertex (defined in -'GraphOps.scala') by the following: +defined in [`Graph`][Graph] and convenient operators that are expressed as a compositions of the +core operators are defined in [`GraphOps`][GraphOps]. However, thanks to Scala implicits the +operators in `GraphOps` are automatically available as members of `Graph`. For example, we can +compute the in-degree of each vertex (defined in `GraphOps`) by the following: + +[Graph]: api/graphx/index.html#org.apache.spark.graphx.Graph +[GraphOps]: api/graphx/index.html#org.apache.spark.graphx.GraphOps {% highlight scala %} val graph: Graph[(String, String), String] @@ -272,20 +273,24 @@ def mapTriplets[ED2](map: EdgeTriplet[VD, ED] => ED2): Graph[VD, ED2] Each of these operators yields a new graph with the vertex or edge properties modified by the user defined `map` function. -> Note that in all cases the graph structure is unaffected. This is a key feature of these -> operators which allows the resulting graph to reuse the structural indicies and the unaffected -> properties of the original graph. -> While the following is logically equivalent to `graph.mapVertices(mapUDF)`, it -> does not preserve the structural indicies and would not benefit from the substantial system -> optimizations in GraphX. +> Note that in all cases the graph structure is unaffected. This is a key feature of these operators +> which allows the resulting graph to reuse the structural indices of the original graph. The +> following snippets are logically equivalent, but the first one does not preserve the structural +> indices and would not benefit from the GraphX system optimizations: > {% highlight scala %} val newVertices = graph.vertices.map { case (id, attr) => (id, mapUdf(id, attr)) } val newGraph = Graph(newVertices, graph.edges) {% endhighlight %} +> Instead, use [`mapVertices`][Graph.mapVertices] to preserve the indices: +> {% highlight scala %} +val newGraph = graph.mapVertices((id, attr) => mapUdf(id, attr)) +{% endhighlight %} + +[Graph.mapVertices]: api/graphx/index.html#org.apache.spark.graphx.Graph@mapVertices[VD2]((VertexID,VD)⇒VD2)(ClassTag[VD2]):Graph[VD2,ED] These operators are often used to initialize the graph for a particular computation or project away unnecessary properties. For example, given a graph with the out-degrees as the vertex properties -(we describe how to construct such a graph later) we initialize for PageRank: +(we describe how to construct such a graph later), we initialize it for PageRank: {% highlight scala %} // Given a graph where the vertex property is the out-degree @@ -293,7 +298,7 @@ val inputGraph: Graph[Int, String] // Construct a graph where each edge contains the weight // and each vertex is the initial PageRank val outputGraph: Graph[Double, Double] = - inputGraph.mapTriplets(et => 1.0/et.srcAttr).mapVertices(v => 1.0) + inputGraph.mapTriplets(et => 1.0 / et.srcAttr).mapVertices(v => 1.0) {% endhighlight %} ## Structural Operators @@ -310,16 +315,20 @@ def mask[VD2, ED2](other: Graph[VD2, ED2]): Graph[VD, ED] def groupEdges(merge: (ED, ED) => ED): Graph[VD,ED] {% endhighlight %} -The `reverse` operator returns a new graph with all the edge directions reversed. This can be -useful when, for example, trying to compute the inverse PageRank. Because the reverse operation -does not modify vertex or edge properties or change the number of edges, it can be implemented -efficiently without data-movement or duplication. +The [`reverse`][Graph.reverse] operator returns a new graph with all the edge directions reversed. +This can be useful when, for example, trying to compute the inverse PageRank. Because the reverse +operation does not modify vertex or edge properties or change the number of edges, it can be +implemented efficiently without data-movement or duplication. + +[Graph.reverse]: api/graphx/index.html#org.apache.spark.graphx.Graph@reverse:Graph[VD,ED] -The `subgraph` operator takes vertex and edge predicates and returns the graph containing only the -vertices that satisfy the vertex predicate (evaluate to true) and edges that satisfy the edge -predicate *and connect vertices that satisfy the vertex predicate*. The `subgraph` operator can be -used in number of situations to restrict the graph to the vertices and edges of interest or -eliminate broken links. For example in the following code we remove broken links: +The [`subgraph`][Graph.subgraph] operator takes vertex and edge predicates and returns the graph +containing only the vertices that satisfy the vertex predicate (evaluate to true) and edges that +satisfy the edge predicate *and connect vertices that satisfy the vertex predicate*. The `subgraph` +operator can be used in number of situations to restrict the graph to the vertices and edges of +interest or eliminate broken links. For example in the following code we remove broken links: + +[Graph.subgraph]: api/graphx/index.html#org.apache.spark.graphx.Graph@subgraph((EdgeTriplet[VD,ED])⇒Boolean,(VertexID,VD)⇒Boolean):Graph[VD,ED] {% highlight scala %} val users: RDD[(VertexId, (String, String))] @@ -335,11 +344,13 @@ val validGraph = graph.subgraph((id, attr) => attr._2 != "Missing") > Note in the above example only the vertex predicate is provided. The `subgraph` operator defaults > to `true` if the vertex or edge predicates are not provided. -The `mask` operator also constructs a subgraph by returning a graph that contains the vertices and -edges that are also found in the input graph. This can be used in conjunction with the `subgraph` -operator to restrict a graph based on the properties in another related graph. For example, we -might run connected components using the graph with missing vertices and then restrict the answer to -the valid subgraph. +The [`mask`][Graph.mask] operator also constructs a subgraph by returning a graph that contains the +vertices and edges that are also found in the input graph. This can be used in conjunction with the +`subgraph` operator to restrict a graph based on the properties in another related graph. For +example, we might run connected components using the graph with missing vertices and then restrict +the answer to the valid subgraph. + +[Graph.mask]: api/graphx/index.html#org.apache.spark.graphx.Graph@mask[VD2,ED2](Graph[VD2,ED2])(ClassTag[VD2],ClassTag[ED2]):Graph[VD,ED] {% highlight scala %} // Run Connected Components @@ -350,9 +361,11 @@ val validGraph = graph.subgraph((id, attr) => attr._2 != "Missing") val validCCGraph = ccGraph.mask(validGraph) {% endhighlight %} -The `groupEdges` operator merges parallel edges (i.e., duplicate edges between pairs of vertices) in -the multigraph. In many numerical applications, parallel edges can be *added* (their weights -combined) into a single edge thereby reducing the size of the graph. +The [`groupEdges`][Graph.groupEdges] operator merges parallel edges (i.e., duplicate edges between +pairs of vertices) in the multigraph. In many numerical applications, parallel edges can be *added* +(their weights combined) into a single edge thereby reducing the size of the graph. + +[Graph.groupEdges]: api/graphx/index.html#org.apache.spark.graphx.Graph@groupEdges((ED,ED)⇒ED):Graph[VD,ED] ## Join Operators @@ -369,11 +382,12 @@ def outerJoinVertices[U, VD2](table: RDD[(VertexID, U)])(map: (VertexID, VD, Opt : Graph[VD2, ED] {% endhighlight %} -The `joinVertices` operator, defined in -[`GraphOps.scala`](api/graphx/index.html#org.apache.spark.graphx.GraphOps), joins the vertices with -the input RDD and returns a new graph with the vertex properties obtained by applying the user -defined `map` function to the result of the joined vertices. Vertices without a matching value in -the RDD retain their original value. +The [`joinVertices`][GraphOps.joinVertices] operator joins the vertices with the input RDD and +returns a new graph with the vertex properties obtained by applying the user defined `map` function +to the result of the joined vertices. Vertices without a matching value in the RDD retain their +original value. + +[GraphOps.joinVertices]: api/graphx/index.html#org.apache.spark.graphx.GraphOps@joinVertices[U](RDD[(VertexID,U)])((VertexID,VD,U)⇒VD)(ClassTag[U]):Graph[VD,ED] > Note that if the RDD contains more than one value for a given vertex only one will be used. It > is therefore recommended that the input RDD be first made unique using the following which will @@ -386,11 +400,14 @@ val joinedGraph = graph.joinVertices(uniqueCosts)( (id, oldCost, extraCost) => oldCost + extraCost) {% endhighlight %} -The more general `outerJoinVertices` behaves similarly to `joinVertices` except that the user -defined `map` function is applied to all vertices and can change the vertex property type. Because -not all vertices may have a matching value in the input RDD the `map` function takes an `Option` -type. For example, we can setup a graph for PageRank by initializing vertex properties with their -`outDegree`. +The more general [`outerJoinVertices`][Graph.outerJoinVertices] behaves similarly to `joinVertices` +except that the user defined `map` function is applied to all vertices and can change the vertex +property type. Because not all vertices may have a matching value in the input RDD the `map` +function takes an `Option` type. For example, we can setup a graph for PageRank by initializing +vertex properties with their `outDegree`. + +[Graph.outerJoinVertices]: api/graphx/index.html#org.apache.spark.graphx.Graph@outerJoinVertices[U,VD2](RDD[(VertexID,U)])((VertexID,VD,Option[U])⇒VD2)(ClassTag[U],ClassTag[VD2]):Graph[VD2,ED] + {% highlight scala %} val outDegrees: VertexRDD[Int] = graph.outDegrees diff --git a/graphx/src/main/scala/org/apache/spark/graphx/package.scala b/graphx/src/main/scala/org/apache/spark/graphx/package.scala index 2501314ca8..e70d2fd09f 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/package.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/package.scala @@ -3,6 +3,10 @@ package org.apache.spark import org.apache.spark.util.collection.OpenHashSet package object graphx { + /** + * A 64-bit vertex identifier that uniquely identifies a vertex within a graph. It does not need + * to follow any ordering or any constraints other than uniqueness. + */ type VertexID = Long // TODO: Consider using Char. -- cgit v1.2.3 From 5e35d39e0f26db3b669bc2318bd7b3f9f6c5fc50 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Sun, 12 Jan 2014 13:10:53 -0800 Subject: Add PageRank example and data --- docs/graphx-programming-guide.md | 32 +++++++++++++++++++++- graphx/data/followers.txt | 12 ++++++++ graphx/data/users.txt | 6 ++++ .../org/apache/spark/graphx/lib/PageRank.scala | 2 +- 4 files changed, 50 insertions(+), 2 deletions(-) create mode 100644 graphx/data/followers.txt create mode 100644 graphx/data/users.txt (limited to 'graphx') diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md index 7f93754edb..52668b07c8 100644 --- a/docs/graphx-programming-guide.md +++ b/docs/graphx-programming-guide.md @@ -470,10 +470,40 @@ things to worry about.) # Graph Algorithms -This section should describe the various algorithms and how they are used. +GraphX includes a set of graph algorithms in to simplify analytics. The algorithms are contained in the `org.apache.spark.graphx.lib` package and can be accessed directly as methods on `Graph` via an implicit conversion to [`Algorithms`][Algorithms]. This section describes the algorithms and how they are used. + +[Algorithms]: api/graphx/index.html#org.apache.spark.graphx.lib.Algorithms ## PageRank +PageRank measures the importance of each vertex in a graph, assuming an edge from *u* to *v* represents an endorsement of *v*'s importance by *u*. For example, if a Twitter user is followed by many others, the user will be ranked highly. + +Spark includes an example social network dataset that we can run PageRank on. A set of users is given in `graphx/data/users.txt`, and a set of relationships between users is given in `graphx/data/followers.txt`. We can compute the PageRank of each user as follows: + +{% highlight scala %} +// Load the implicit conversion to Algorithms +import org.apache.spark.graphx.lib._ +// Load the datasets into a graph +val users = sc.textFile("graphx/data/users.txt").map { line => + val fields = line.split("\\s+") + (fields(0).toLong, fields(1)) +} +val followers = sc.textFile("graphx/data/followers.txt").map { line => + val fields = line.split("\\s+") + Edge(fields(0).toLong, fields(1).toLong, 1) +} +val graph = Graph(users, followers) +// Run PageRank +val ranks = graph.pageRank(0.0001).vertices +// Join the ranks with the usernames +val ranksByUsername = users.leftOuterJoin(ranks).map { + case (id, (username, rankOpt)) => (username, rankOpt.getOrElse(0.0)) +} +// Print the result +println(ranksByUsername.collect().mkString("\n")) +{% endhighlight %} + + ## Connected Components ## Shortest Path diff --git a/graphx/data/followers.txt b/graphx/data/followers.txt new file mode 100644 index 0000000000..0f46d80806 --- /dev/null +++ b/graphx/data/followers.txt @@ -0,0 +1,12 @@ +2 1 +3 1 +4 1 +6 1 +3 2 +6 2 +7 2 +6 3 +7 3 +7 6 +6 7 +3 7 diff --git a/graphx/data/users.txt b/graphx/data/users.txt new file mode 100644 index 0000000000..ce3d06c600 --- /dev/null +++ b/graphx/data/users.txt @@ -0,0 +1,6 @@ +1 BarackObama +2 ericschmidt +3 jeresig +4 justinbieber +6 matei_zaharia +7 odersky diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala index 809b6d0855..cf95267e77 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala @@ -106,7 +106,7 @@ object PageRank extends Logging { * @tparam ED the original edge attribute (not used) * * @param graph the graph on which to compute PageRank - * @param tol the tolerance allowed at convergence (smaller => more * accurate). + * @param tol the tolerance allowed at convergence (smaller => more accurate). * @param resetProb the random reset probability (alpha) * * @return the graph containing with each vertex containing the PageRank and each edge -- cgit v1.2.3 From 7a4bb863c7c11e22332763081793e4989af8c526 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Sun, 12 Jan 2014 16:58:18 -0800 Subject: Add connected components example to doc --- docs/graphx-programming-guide.md | 20 +++++++++++++++++++- graphx/data/followers.txt | 6 +----- graphx/data/users.txt | 2 +- 3 files changed, 21 insertions(+), 7 deletions(-) (limited to 'graphx') diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md index 52668b07c8..22feccb7ad 100644 --- a/docs/graphx-programming-guide.md +++ b/docs/graphx-programming-guide.md @@ -475,6 +475,7 @@ GraphX includes a set of graph algorithms in to simplify analytics. The algorith [Algorithms]: api/graphx/index.html#org.apache.spark.graphx.lib.Algorithms ## PageRank + PageRank measures the importance of each vertex in a graph, assuming an edge from *u* to *v* represents an endorsement of *v*'s importance by *u*. For example, if a Twitter user is followed by many others, the user will be ranked highly. @@ -503,9 +504,26 @@ val ranksByUsername = users.leftOuterJoin(ranks).map { println(ranksByUsername.collect().mkString("\n")) {% endhighlight %} - ## Connected Components +The connected components algorithm labels each connected component of the graph with the ID of its lowest-numbered vertex. For example, in a social network, connected components can approximate clusters. We can compute the connected components of the example social network dataset from the [PageRank section](#pagerank) as follows: + +{% highlight scala %} +// Load the implicit conversion and graph as in the PageRank example +import org.apache.spark.graphx.lib._ +val users = ... +val followers = ... +val graph = Graph(users, followers) +// Find the connected components +val cc = graph.connectedComponents().vertices +// Join the connected components with the usernames +val ccByUsername = graph.vertices.innerJoin(cc) { (id, username, cc) => + (username, cc) +} +// Print the result +println(ccByUsername.collect().mkString("\n")) +{% endhighlight %} + ## Shortest Path ## Triangle Counting diff --git a/graphx/data/followers.txt b/graphx/data/followers.txt index 0f46d80806..7bb8e900e2 100644 --- a/graphx/data/followers.txt +++ b/graphx/data/followers.txt @@ -1,10 +1,6 @@ 2 1 -3 1 4 1 -6 1 -3 2 -6 2 -7 2 +1 2 6 3 7 3 7 6 diff --git a/graphx/data/users.txt b/graphx/data/users.txt index ce3d06c600..26e3b3bb4d 100644 --- a/graphx/data/users.txt +++ b/graphx/data/users.txt @@ -1,5 +1,5 @@ 1 BarackObama -2 ericschmidt +2 ladygaga 3 jeresig 4 justinbieber 6 matei_zaharia -- cgit v1.2.3 From 2216319f485ca2d00a946c4478dedc8a0e1c6053 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Sun, 12 Jan 2014 21:26:37 -0800 Subject: adding Pregel as an operator in GraphOps and cleaning up documentation of GraphOps --- .../scala/org/apache/spark/graphx/GraphOps.scala | 92 +++++++++++++++++----- .../scala/org/apache/spark/graphx/Pregel.scala | 4 +- 2 files changed, 74 insertions(+), 22 deletions(-) (limited to 'graphx') 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 0121cb1449..4fdff29f5a 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala @@ -6,10 +6,9 @@ import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext._ import org.apache.spark.SparkException - /** * Contains additional functionality for [[Graph]]. All operations are expressed in terms of the - * efficient GraphX API. This class is implicitly constructed for each Graph object. + * efficient GraphX API. This class is implicitly constructed for each Graph object. * * @tparam VD the vertex attribute type * @tparam ED the edge attribute type @@ -19,32 +18,27 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) { /** The number of edges in the graph. */ lazy val numEdges: Long = graph.edges.count() - /** The number of vertices in the graph. */ lazy val numVertices: Long = graph.vertices.count() - /** * The in-degree of each vertex in the graph. * @note Vertices with no in-edges are not returned in the resulting RDD. */ lazy val inDegrees: VertexRDD[Int] = degreesRDD(EdgeDirection.In) - /** * The out-degree of each vertex in the graph. * @note Vertices with no out-edges are not returned in the resulting RDD. */ lazy val outDegrees: VertexRDD[Int] = degreesRDD(EdgeDirection.Out) - /** * The degree of each vertex in the graph. * @note Vertices with no edges are not returned in the resulting RDD. */ lazy val degrees: VertexRDD[Int] = degreesRDD(EdgeDirection.Both) - /** * Computes the neighboring vertex degrees. * @@ -76,10 +70,10 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) { * age for each user: * * {{{ - * val graph: Graph[Int,Int] = loadGraph() + * val graph: Graph[Int,Int] = GraphLoader.edgeListFile(sc, "webgraph") * val averageFollowerAge: RDD[(Int, Int)] = * graph.aggregateNeighbors[(Int,Double)]( - * (vid, edge) => (edge.otherVertex(vid).data, 1), + * (vid, edge) => Some((edge.otherVertex(vid).data, 1)), * (a, b) => (a._1 + b._1, a._2 + b._2), * -1, * EdgeDirection.In) @@ -111,11 +105,9 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) { case (Some(srcA), Some(dstA)) => Iterator((et.srcId, srcA), (et.dstId, dstA)) } } - graph.mapReduceTriplets(mf, reduceFunc) } // end of aggregateNeighbors - /** * Collect the neighbor vertex ids for each vertex. * @@ -147,7 +139,6 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) { } } // end of collectNeighborIds - /** * Collect the neighbor vertex attributes for each vertex. * @@ -173,7 +164,6 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) { } } // end of collectNeighbor - /** * Join the vertices with an RDD and then apply a function from the * the vertex and RDD entry to a new vertex value. The input table @@ -188,17 +178,14 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) { * corresponding entry in the table otherwise the old vertex value * is used. * - * @note for small tables this function can be much more efficient - * than leftJoinVertices - * * @example This function is used to update the vertices with new * values based on external data. For example we could add the out * degree to each vertex record * * {{{ - * val rawGraph: Graph[Int,()] = Graph.textFile("webgraph") + * val rawGraph: Graph[Int, Int] = GraphLoader.edgeListFile(sc, "webgraph") * .mapVertices(v => 0) - * val outDeg: RDD[(Int, Int)] = rawGraph.outDegrees() + * val outDeg: RDD[(Int, Int)] = rawGraph.outDegrees * val graph = rawGraph.leftJoinVertices[Int,Int](outDeg, * (v, deg) => deg ) * }}} @@ -219,8 +206,10 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) { * Filter the graph by computing some values to filter on, and applying the predicates. * * @param preprocess a function to compute new vertex and edge data before filtering - * @param epred edge pred to filter on after preprocess, see more details under Graph#subgraph - * @param vpred vertex pred to filter on after prerocess, see more details under Graph#subgraph + * @param epred edge pred to filter on after preprocess, see more details under + * [[org.apache.spark.graphx.Graph#subgraph]] + * @param vpred vertex pred to filter on after prerocess, see more details under + * [[org.apache.spark.graphx.Graph#subgraph]] * @tparam VD2 vertex type the vpred operates on * @tparam ED2 edge type the epred operates on * @return a subgraph of the orginal graph, with its data unchanged @@ -246,4 +235,67 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) { vpred: (VertexID, VD2) => Boolean = (v:VertexID, d:VD2) => true): Graph[VD, ED] = { graph.mask(preprocess(graph).subgraph(epred, vpred)) } + + /** + * Execute a Pregel-like iterative vertex-parallel abstraction. The + * user-defined vertex-program `vprog` is executed in parallel on + * each vertex receiving any inbound messages and computing a new + * value for the vertex. The `sendMsg` function is then invoked on + * all out-edges and is used to compute an optional message to the + * destination vertex. The `mergeMsg` function is a commutative + * associative function used to combine messages destined to the + * same vertex. + * + * On the first iteration all vertices receive the `initialMsg` and + * on subsequent iterations if a vertex does not receive a message + * then the vertex-program is not invoked. + * + * This function iterates until there are no remaining messages, or + * for `maxIterations` iterations. + * + * @tparam VD the vertex data type + * @tparam ED the edge data type + * @tparam A the Pregel message type + * + * @param graph the input graph. + * + * @param initialMsg the message each vertex will receive at the on + * the first iteration + * + * @param maxIterations the maximum number of iterations to run for + * + * @param activeDirection the direction of edges incident to a vertex that received a message in + * the previous round on which to run `sendMsg`. For example, if this is `EdgeDirection.Out`, only + * out-edges of vertices that received a message in the previous round will run. + * + * @param vprog the user-defined vertex program which runs on each + * vertex and receives the inbound message and computes a new vertex + * value. On the first iteration the vertex program is invoked on + * all vertices and is passed the default message. On subsequent + * iterations the vertex program is only invoked on those vertices + * that receive messages. + * + * @param sendMsg a user supplied function that is applied to out + * edges of vertices that received messages in the current + * iteration + * + * @param mergeMsg a user supplied function that takes two incoming + * messages of type A and merges them into a single message of type + * A. ''This function must be commutative and associative and + * ideally the size of A should not increase.'' + * + * @return the resulting graph at the end of the computation + * + */ + def pregel[A: ClassTag]( + initialMsg: A, + maxIterations: Int = Int.MaxValue, + activeDirection: EdgeDirection = EdgeDirection.Out)( + vprog: (VertexID, VD, A) => VD, + sendMsg: EdgeTriplet[VD, ED] => Iterator[(VertexID,A)], + mergeMsg: (A, A) => A) + : Graph[VD, ED] = { + Pregel(graph, initialMsg, maxIterations, activeDirection)(vprog, sendMsg, mergeMsg) + } + } // end of GraphOps 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 57b087213f..83e28d0ab2 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala @@ -25,8 +25,8 @@ import scala.reflect.ClassTag * * def vertexProgram(id: VertexID, attr: Double, msgSum: Double): Double = * resetProb + (1.0 - resetProb) * msgSum - * def sendMessage(id: VertexID, edge: EdgeTriplet[Double, Double]): Option[Double] = - * Some(edge.srcAttr * edge.attr) + * def sendMessage(id: VertexID, edge: EdgeTriplet[Double, Double]): Iterator[(VertexId, Double)] = + * Iterator((edge.dstId, edge.srcAttr * edge.attr)) * def messageCombiner(a: Double, b: Double): Double = a + b * val initialMessage = 0.0 * // Execute Pregel for a fixed number of iterations. -- cgit v1.2.3 From 20c509b805dbfd0ebb11d2d7bd53a4379249a86f Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Sun, 12 Jan 2014 21:41:21 -0800 Subject: Add TriangleCount example --- docs/graphx-programming-guide.md | 31 +++++++++++++++++++--- .../apache/spark/graphx/lib/TriangleCount.scala | 5 ++-- 2 files changed, 29 insertions(+), 7 deletions(-) (limited to 'graphx') diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md index 89759416f4..0e228d8f28 100644 --- a/docs/graphx-programming-guide.md +++ b/docs/graphx-programming-guide.md @@ -676,7 +676,9 @@ GraphX includes a set of graph algorithms in to simplify analytics. The algorith PageRank measures the importance of each vertex in a graph, assuming an edge from *u* to *v* represents an endorsement of *v*'s importance by *u*. For example, if a Twitter user is followed by many others, the user will be ranked highly. -Spark includes an example social network dataset that we can run PageRank on. A set of users is given in `graphx/data/users.txt`, and a set of relationships between users is given in `graphx/data/followers.txt`. We can compute the PageRank of each user as follows: +GraphX comes with static and dynamic implementations of PageRank as methods on the [`PageRank` object][PageRank]. Static PageRank runs for a fixed number of iterations, while dynamic PageRank runs until the ranks converge (i.e., stop changing by more than a specified tolerance). GraphX also includes an example social network dataset that we can run PageRank on. A set of users is given in `graphx/data/users.txt`, and a set of relationships between users is given in `graphx/data/followers.txt`. We compute the PageRank of each user as follows: + +[PageRank]: api/graphx/index.html#org.apache.spark.graphx.lib.PageRank$ {% highlight scala %} // Load the implicit conversion to Algorithms @@ -703,7 +705,9 @@ println(ranksByUsername.collect().mkString("\n")) ## Connected Components -The connected components algorithm labels each connected component of the graph with the ID of its lowest-numbered vertex. For example, in a social network, connected components can approximate clusters. We can compute the connected components of the example social network dataset from the [PageRank section](#pagerank) as follows: +The connected components algorithm labels each connected component of the graph with the ID of its lowest-numbered vertex. For example, in a social network, connected components can approximate clusters. GraphX contains an implementation of the algorithm in the [`ConnectedComponents` object][ConnectedComponents], and we compute the connected components of the example social network dataset from the [PageRank section](#pagerank) as follows: + +[ConnectedComponents]: api/graphx/index.html#org.apache.spark.graphx.lib.ConnectedComponents$ {% highlight scala %} // Load the implicit conversion and graph as in the PageRank example @@ -721,10 +725,29 @@ val ccByUsername = graph.vertices.innerJoin(cc) { (id, username, cc) => println(ccByUsername.collect().mkString("\n")) {% endhighlight %} -## Shortest Path - ## Triangle Counting +A vertex is part of a triangle when it has two adjacent vertices with an edge between them. GraphX implements a triangle counting algorithm in the [`TriangleCount` object][TriangleCount] that determines the number of triangles passing through each vertex, providing a measure of clustering. We compute the triangle count of the social network dataset from the [PageRank section](#pagerank). *Note that `TriangleCount` requires the edges to be in canonical orientation (`srcId < dstId`) and the graph to be partitioned using [`Graph#partitionBy`][Graph.partitionBy].* + +[TriangleCount]: api/graphx/index.html#org.apache.spark.graphx.lib.TriangleCount$ +[Graph.partitionBy]: api/graphx/index.html#org.apache.spark.graphx.Graph@partitionBy(PartitionStrategy):Graph[VD,ED] + +{% highlight scala %} +// Load the implicit conversion and graph as in the PageRank example +import org.apache.spark.graphx.lib._ +val users = ... +// Load the edges in canonical order and partition the graph for triangle count +val graph = GraphLoader.edgeListFile(sc, "graphx/data/followers.txt", true).partitionBy(RandomVertexCut) +// Find the triangle count for each vertex +val triCounts = graph.triangleCount().vertices +// Join the triangle counts with the usernames +val triCountByUsername = graph.vertices.innerJoin(triCounts) { (id, username, tc) => + (username, tc) +} +// Print the result +println(triCountByUsername.collect().mkString("\n")) +{% endhighlight %} + ## K-Core ## LDA diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/TriangleCount.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/TriangleCount.scala index c6b1c736dd..58da9e3aed 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/TriangleCount.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/TriangleCount.scala @@ -19,9 +19,8 @@ object TriangleCount { * * * @param graph a graph with `sourceId` less than `destId`. The graph must have been partitioned - * using Graph.partitionBy. - * - * @return + * using [[org.apache.spark.graphx.Graph#partitionBy]], and its edges must be in canonical + * orientation (srcId < dstId). */ def run[VD: ClassTag, ED: ClassTag](graph: Graph[VD,ED]): Graph[Int, ED] = { // Remove redundant edges -- cgit v1.2.3 From d691e9f47ed9b43b422712047183142d01c5e8c2 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Sun, 12 Jan 2014 21:47:16 -0800 Subject: Move algorithms to GraphOps --- docs/graphx-programming-guide.md | 12 +--- .../main/scala/org/apache/spark/graphx/Graph.scala | 4 +- .../scala/org/apache/spark/graphx/GraphOps.scala | 51 ++++++++++++++++- .../org/apache/spark/graphx/lib/Algorithms.scala | 66 ---------------------- .../org/apache/spark/graphx/lib/package.scala | 8 --- 5 files changed, 54 insertions(+), 87 deletions(-) delete mode 100644 graphx/src/main/scala/org/apache/spark/graphx/lib/Algorithms.scala delete mode 100644 graphx/src/main/scala/org/apache/spark/graphx/lib/package.scala (limited to 'graphx') diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md index 0e228d8f28..572afc101b 100644 --- a/docs/graphx-programming-guide.md +++ b/docs/graphx-programming-guide.md @@ -667,9 +667,7 @@ things to worry about.) # Graph Algorithms -GraphX includes a set of graph algorithms in to simplify analytics. The algorithms are contained in the `org.apache.spark.graphx.lib` package and can be accessed directly as methods on `Graph` via an implicit conversion to [`Algorithms`][Algorithms]. This section describes the algorithms and how they are used. - -[Algorithms]: api/graphx/index.html#org.apache.spark.graphx.lib.Algorithms +GraphX includes a set of graph algorithms in to simplify analytics. The algorithms are contained in the `org.apache.spark.graphx.lib` package and can be accessed directly as methods on `Graph` via [`GraphOps`][GraphOps]. This section describes the algorithms and how they are used. ## PageRank @@ -681,8 +679,6 @@ GraphX comes with static and dynamic implementations of PageRank as methods on t [PageRank]: api/graphx/index.html#org.apache.spark.graphx.lib.PageRank$ {% highlight scala %} -// Load the implicit conversion to Algorithms -import org.apache.spark.graphx.lib._ // Load the datasets into a graph val users = sc.textFile("graphx/data/users.txt").map { line => val fields = line.split("\\s+") @@ -710,8 +706,7 @@ The connected components algorithm labels each connected component of the graph [ConnectedComponents]: api/graphx/index.html#org.apache.spark.graphx.lib.ConnectedComponents$ {% highlight scala %} -// Load the implicit conversion and graph as in the PageRank example -import org.apache.spark.graphx.lib._ +// Load the graph as in the PageRank example val users = ... val followers = ... val graph = Graph(users, followers) @@ -733,8 +728,7 @@ A vertex is part of a triangle when it has two adjacent vertices with an edge be [Graph.partitionBy]: api/graphx/index.html#org.apache.spark.graphx.Graph@partitionBy(PartitionStrategy):Graph[VD,ED] {% highlight scala %} -// Load the implicit conversion and graph as in the PageRank example -import org.apache.spark.graphx.lib._ +// Load the graph as in the PageRank example val users = ... // Load the edges in canonical order and partition the graph for triangle count val graph = GraphLoader.edgeListFile(sc, "graphx/data/followers.txt", true).partitionBy(RandomVertexCut) 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 56513cac20..7d4f0de3d6 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala @@ -15,9 +15,7 @@ import org.apache.spark.storage.StorageLevel * RDDs, the graph is a functional data-structure in which mutating * operations return new graphs. * - * @note [[GraphOps]] contains additional convenience operations. - * [[lib.Algorithms]] contains graph algorithms; to access these, - * import `org.apache.spark.graphx.lib._`. + * @note [[GraphOps]] contains additional convenience operations and graph algorithms. * * @tparam VD the vertex attribute type * @tparam ED the edge attribute type 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 4fdff29f5a..2b3b95e2ca 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala @@ -2,9 +2,10 @@ package org.apache.spark.graphx import scala.reflect.ClassTag -import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext._ import org.apache.spark.SparkException +import org.apache.spark.graphx.lib._ +import org.apache.spark.rdd.RDD /** * Contains additional functionality for [[Graph]]. All operations are expressed in terms of the @@ -298,4 +299,52 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) { Pregel(graph, initialMsg, maxIterations, activeDirection)(vprog, sendMsg, mergeMsg) } + /** + * Run a dynamic version of PageRank returning a graph with vertex attributes containing the + * PageRank and edge attributes containing the normalized edge weight. + * + * @see [[org.apache.spark.graphx.lib.PageRank]], method `runUntilConvergence`. + */ + def pageRank(tol: Double, resetProb: Double = 0.15): Graph[Double, Double] = { + PageRank.runUntilConvergence(graph, tol, resetProb) + } + + /** + * Run PageRank for a fixed number of iterations returning a graph with vertex attributes + * containing the PageRank and edge attributes the normalized edge weight. + * + * @see [[org.apache.spark.graphx.lib.PageRank]], method `run`. + */ + def staticPageRank(numIter: Int, resetProb: Double = 0.15): Graph[Double, Double] = { + PageRank.run(graph, numIter, resetProb) + } + + /** + * Compute the connected component membership of each vertex and return a graph with the vertex + * value containing the lowest vertex id in the connected component containing that vertex. + * + * @see [[org.apache.spark.graphx.lib.ConnectedComponents]] + */ + def connectedComponents(): Graph[VertexID, ED] = { + ConnectedComponents.run(graph) + } + + /** + * Compute the number of triangles passing through each vertex. + * + * @see [[org.apache.spark.graphx.lib.TriangleCount]] + */ + def triangleCount(): Graph[Int, ED] = { + TriangleCount.run(graph) + } + + /** + * Compute the strongly connected component (SCC) of each vertex and return a graph with the + * vertex value containing the lowest vertex id in the SCC containing that vertex. + * + * @see [[org.apache.spark.graphx.lib.StronglyConnectedComponents]] + */ + def stronglyConnectedComponents(numIter: Int): Graph[VertexID, ED] = { + StronglyConnectedComponents.run(graph, numIter) + } } // end of GraphOps diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/Algorithms.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/Algorithms.scala deleted file mode 100644 index cbcd9c24a0..0000000000 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/Algorithms.scala +++ /dev/null @@ -1,66 +0,0 @@ -package org.apache.spark.graphx.lib - -import scala.reflect.ClassTag - -import org.apache.spark.graphx._ - -/** - * Provides graph algorithms directly on [[org.apache.spark.graphx.Graph]] via an implicit - * conversion. - * @example - * {{{ - * import org.apache.spark.graph.lib._ - * val graph: Graph[_, _] = loadGraph() - * graph.connectedComponents() - * }}} - */ -class Algorithms[VD: ClassTag, ED: ClassTag](self: Graph[VD, ED]) { - /** - * Run a dynamic version of PageRank returning a graph with vertex attributes containing the - * PageRank and edge attributes containing the normalized edge weight. - * - * @see [[org.apache.spark.graphx.lib.PageRank]], method `runUntilConvergence`. - */ - def pageRank(tol: Double, resetProb: Double = 0.15): Graph[Double, Double] = { - PageRank.runUntilConvergence(self, tol, resetProb) - } - - /** - * Run PageRank for a fixed number of iterations returning a graph with vertex attributes - * containing the PageRank and edge attributes the normalized edge weight. - * - * @see [[org.apache.spark.graphx.lib.PageRank]], method `run`. - */ - def staticPageRank(numIter: Int, resetProb: Double = 0.15): Graph[Double, Double] = { - PageRank.run(self, numIter, resetProb) - } - - /** - * Compute the connected component membership of each vertex and return a graph with the vertex - * value containing the lowest vertex id in the connected component containing that vertex. - * - * @see [[org.apache.spark.graphx.lib.ConnectedComponents]] - */ - def connectedComponents(): Graph[VertexID, ED] = { - ConnectedComponents.run(self) - } - - /** - * Compute the number of triangles passing through each vertex. - * - * @see [[org.apache.spark.graphx.lib.TriangleCount]] - */ - def triangleCount(): Graph[Int, ED] = { - TriangleCount.run(self) - } - - /** - * Compute the strongly connected component (SCC) of each vertex and return a graph with the - * vertex value containing the lowest vertex id in the SCC containing that vertex. - * - * @see [[org.apache.spark.graphx.lib.StronglyConnectedComponents]] - */ - def stronglyConnectedComponents(numIter: Int): Graph[VertexID, ED] = { - StronglyConnectedComponents.run(self, numIter) - } -} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/package.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/package.scala deleted file mode 100644 index f6f2626c9d..0000000000 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/package.scala +++ /dev/null @@ -1,8 +0,0 @@ -package org.apache.spark.graphx - -import scala.reflect.ClassTag - -package object lib { - implicit def graphToAlgorithms[VD: ClassTag, ED: ClassTag]( - graph: Graph[VD, ED]): Algorithms[VD, ED] = new Algorithms(graph) -} -- cgit v1.2.3 From ea69cff7110e4eaf4e92bc3f2f9d1e765f172b0e Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Mon, 13 Jan 2014 12:52:52 -0800 Subject: Further improve VertexRDD scaladocs --- .../scala/org/apache/spark/graphx/VertexRDD.scala | 39 ++++++++++++++-------- 1 file changed, 25 insertions(+), 14 deletions(-) (limited to 'graphx') 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 d20745d0d4..9a95364cb1 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala @@ -28,17 +28,14 @@ import org.apache.spark.graphx.impl.MsgRDDFunctions import org.apache.spark.graphx.impl.VertexPartition /** - * `VertexRDD[VD]` extends the `RDD[(VertexID, VD)]` by ensuring that there is only one entry for - * each vertex and by pre-indexing the entries for fast, efficient joins. Two VertexRDDs with the - * same index can be joined efficiently. + * Extends `RDD[(VertexID, VD)]` by ensuring that there is only one entry for each vertex and by + * pre-indexing the entries for fast, efficient joins. Two VertexRDDs with the same index can be + * joined efficiently. All operations except [[reindex]] preserve the index. To construct a + * `VertexRDD`, use the [[org.apache.spark.graphx.VertexRDD$ VertexRDD object]]. * - * @tparam VD the vertex attribute associated with each vertex in the set. - * - * To construct a `VertexRDD` use the singleton object: - * - * @example Construct a `VertexRDD` from a plain RDD + * @example Construct a `VertexRDD` from a plain RDD: * {{{ - * // Construct an intial vertex set + * // Construct an initial vertex set * val someData: RDD[(VertexID, SomeType)] = loadData(someFile) * val vset = VertexRDD(someData) * // If there were redundant values in someData we would use a reduceFunc @@ -50,6 +47,7 @@ import org.apache.spark.graphx.impl.VertexPartition * val vset4: VertexRDD[(SomeType, OtherType)] = vset.leftJoin(vset3) * }}} * + * @tparam VD the vertex attribute associated with each vertex in the set. */ class VertexRDD[@specialized VD: ClassTag]( val partitionsRDD: RDD[VertexPartition[VD]]) @@ -146,7 +144,7 @@ class VertexRDD[@specialized VD: ClassTag]( this.mapVertexPartitions(_.map(f)) /** - * Hides vertices that are the same between `this` and `other`. For vertices that are different, + * Hides vertices that are the same between `this` and `other`; for vertices that are different, * keeps the values from `other`. */ def diff(other: VertexRDD[VD]): VertexRDD[VD] = { @@ -188,7 +186,7 @@ class VertexRDD[@specialized VD: ClassTag]( /** * Left joins this VertexRDD with an RDD containing vertex attribute pairs. If the other RDD is * backed by a VertexRDD with the same index then the efficient [[leftZipJoin]] implementation is - * used. The resulting vertex set contains an entry for each vertex in this set. If `other` is + * used. The resulting VertexRDD contains an entry for each vertex in `this`. If `other` is * missing any vertex in this VertexRDD, `f` is passed `None`. If there are duplicates, the vertex * is picked arbitrarily. * @@ -223,8 +221,8 @@ class VertexRDD[@specialized VD: ClassTag]( } /** - * Same effect as `leftJoin(other) { (vid, a, bOpt) => bOpt.getOrElse(a) }`, but `this` and - * `other` must have the same index. + * Efficiently inner joins this VertexRDD with another VertexRDD sharing the same index. See + * [[innerJoin]] for the behavior of the join. */ def innerZipJoin[U: ClassTag, VD2: ClassTag](other: VertexRDD[U]) (f: (VertexID, VD, U) => VD2): VertexRDD[VD2] = { @@ -242,6 +240,12 @@ class VertexRDD[@specialized VD: ClassTag]( * Inner joins this VertexRDD with an RDD containing vertex attribute pairs. If the other RDD is * backed by a VertexRDD with the same index then the efficient [[innerZipJoin]] implementation is * used. + * + * @param other an RDD containing vertices to join. If there are multiple entries for the same + * vertex, one is picked arbitrarily. Use [[aggregateUsingIndex]] to merge multiple entries. + * @param f the join function applied to corresponding values of `this` and `other` + * @return a VertexRDD co-indexed with `this`, containing only vertices that appear in both `this` + * and `other`, with values supplied by `f` */ def innerJoin[U: ClassTag, VD2: ClassTag](other: RDD[(VertexID, U)]) (f: (VertexID, VD, U) => VD2): VertexRDD[VD2] = { @@ -263,8 +267,15 @@ class VertexRDD[@specialized VD: ClassTag]( } /** - * Aggregates vertices in `message` that have the same ids using `reduceFunc`, returning a + * Aggregates vertices in `messages` that have the same ids using `reduceFunc`, returning a * VertexRDD co-indexed with `this`. + * + * @param messages an RDD containing messages to aggregate, where each message is a pair of its + * target vertex ID and the message data + * @param reduceFunc the associative aggregation function for merging messages to the same vertex + * @return a VertexRDD co-indexed with `this`, containing only vertices that received messages. + * For those vertices, their values are the result of applying `reduceFunc` to all received + * messages. */ def aggregateUsingIndex[VD2: ClassTag]( messages: RDD[(VertexID, VD2)], reduceFunc: (VD2, VD2) => VD2): VertexRDD[VD2] = { -- cgit v1.2.3 From 9fe88627b5716c6893a6d034fefcf530a6c470fa Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Mon, 13 Jan 2014 13:16:41 -0800 Subject: Improve EdgeRDD scaladoc --- graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala | 13 +++++++++++-- 1 file changed, 11 insertions(+), 2 deletions(-) (limited to 'graphx') 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 05d3dbe337..0269ed3cf3 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala @@ -49,7 +49,7 @@ class EdgeRDD[@specialized ED: ClassTag]( this } - def mapEdgePartitions[ED2: ClassTag](f: (PartitionID, EdgePartition[ED]) => EdgePartition[ED2]) + private[graphx] def mapEdgePartitions[ED2: ClassTag](f: (PartitionID, EdgePartition[ED]) => EdgePartition[ED2]) : EdgeRDD[ED2] = { new EdgeRDD[ED2](partitionsRDD.mapPartitions({ iter => val (pid, ep) = iter.next() @@ -57,6 +57,15 @@ class EdgeRDD[@specialized ED: ClassTag]( }, preservesPartitioning = true)) } + /** + * Inner joins this EdgeRDD with another EdgeRDD, assuming both are partitioned using the same + * [[PartitionStrategy]]. + * + * @param other the EdgeRDD to join with + * @param f the join function applied to corresponding values of `this` and `other` + * @return a new EdgeRDD containing only edges that appear in both `this` and `other`, with values + * supplied by `f` + */ def innerJoin[ED2: ClassTag, ED3: ClassTag] (other: EdgeRDD[ED2]) (f: (VertexID, VertexID, ED, ED2) => ED3): EdgeRDD[ED3] = { @@ -70,7 +79,7 @@ class EdgeRDD[@specialized ED: ClassTag]( }) } - def collectVertexIDs(): RDD[VertexID] = { + private[graphx] def collectVertexIDs(): RDD[VertexID] = { partitionsRDD.flatMap { case (_, p) => Array.concat(p.srcIds, p.dstIds) } } } -- cgit v1.2.3 From 80e4d98dc656e20dacbd8cdbf92d4912673b42ae Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Mon, 13 Jan 2014 13:40:16 -0800 Subject: Improving documentation and identifying potential bug in CC calculation. --- docs/graphx-programming-guide.md | 33 +++++++++++++--- .../scala/org/apache/spark/graphx/GraphOps.scala | 4 +- .../spark/graphx/lib/ConnectedComponents.scala | 44 +++++++++++++++------- .../graphx/lib/ConnectedComponentsSuite.scala | 30 +++++++++++++++ 4 files changed, 89 insertions(+), 22 deletions(-) (limited to 'graphx') diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md index 2697b2def7..ed976b8989 100644 --- a/docs/graphx-programming-guide.md +++ b/docs/graphx-programming-guide.md @@ -84,7 +84,8 @@ import org.apache.spark.graphx._ import org.apache.spark.rdd.RDD {% endhighlight %} -If you are not using the Spark shell you will also need a Spark context. +If you are not using the Spark shell you will also need a `SparkContext`. To learn more about +getting started with Spark refer to the [Spark Quick Start Guide](quick-start.html). # The Property Graph @@ -190,7 +191,7 @@ and `graph.edges` members respectively. {% highlight scala %} val graph: Graph[(String, String), String] // Constructed from above // Count all users which are postdocs -graph.vertices.filter { case (id, (name, pos)) => pos == "postdoc"}.count +graph.vertices.filter { case (id, (name, pos)) => pos == "postdoc" }.count // Count all the edges where src > dst graph.edges.filter(e => e.srcId > e.dstId).count {% endhighlight %} @@ -258,8 +259,10 @@ val graph: Graph[(String, String), String] val indDegrees: VertexRDD[Int] = graph.inDegrees {% endhighlight %} -The reason for differentiating between core graph operations and GraphOps is to be able to support -various graph representations in the future. +The reason for differentiating between core graph operations and [`GraphOps`][GraphOps] is to be +able to support different graph representations in the future. Each graph representation must +provide implementations of the core operations and reuse many of the useful operations defined in +[`GraphOps`][GraphOps]. ## Property Operators @@ -334,14 +337,32 @@ interest or eliminate broken links. For example in the following code we remove [Graph.subgraph]: api/graphx/index.html#org.apache.spark.graphx.Graph@subgraph((EdgeTriplet[VD,ED])⇒Boolean,(VertexID,VD)⇒Boolean):Graph[VD,ED] {% highlight scala %} -val users: RDD[(VertexId, (String, String))] -val edges: RDD[Edge[String]] +// Create an RDD for the vertices +val users: RDD[(VertexID, (String, String))] = + sc.parallelize(Array((3L, ("rxin", "student")), (7L, ("jgonzal", "postdoc")), + (5L, ("franklin", "prof")), (2L, ("istoica", "prof")), + (4L, ("peter", "student")))) +// Create an RDD for edges +val relationships: RDD[Edge[String]] = + sc.parallelize(Array(Edge(3L, 7L, "collab"), Edge(5L, 3L, "advisor"), + Edge(2L, 5L, "colleague"), Edge(5L, 7L, "pi"), + Edge(4L, 0L, "student"), Edge(5L, 0L, "colleague"))) // Define a default user in case there are relationship with missing user val defaultUser = ("John Doe", "Missing") // Build the initial Graph val graph = Graph(users, relationships, defaultUser) +// Notice that there is a user 0 (for which we have no information) connecting users +// 4 (peter) and 5 (franklin). +graph.triplets.map( + triplet => triplet.srcAttr._1 + " is the " + triplet.attr + " of " + triplet.dstAttr._1 + ).collect.foreach(println(_)) // Remove missing vertices as well as the edges to connected to them val validGraph = graph.subgraph(vpred = (id, attr) => attr._2 != "Missing") +// The valid subgraph will disconnect users 4 and 5 by removing user 0 +validGraph.vertices.collect.foreach(println(_)) +validGraph.triplets.map( + triplet => triplet.srcAttr._1 + " is the " + triplet.attr + " of " + triplet.dstAttr._1 + ).collect.foreach(println(_)) {% endhighlight %} > Note in the above example only the vertex predicate is provided. The `subgraph` operator defaults 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 2b3b95e2ca..a0a40e2d9a 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala @@ -325,8 +325,8 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) { * * @see [[org.apache.spark.graphx.lib.ConnectedComponents]] */ - def connectedComponents(): Graph[VertexID, ED] = { - ConnectedComponents.run(graph) + def connectedComponents(undirected: Boolean = true): Graph[VertexID, ED] = { + ConnectedComponents.run(graph, undirected) } /** diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/ConnectedComponents.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/ConnectedComponents.scala index 4a83e2dbb8..d078d2acdb 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/ConnectedComponents.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/ConnectedComponents.scala @@ -14,26 +14,42 @@ object ConnectedComponents { * @tparam ED the edge attribute type (preserved in the computation) * * @param graph the graph for which to compute the connected components + * @param undirected compute reachability ignoring edge direction. * * @return a graph with vertex attributes containing the smallest vertex in each * connected component */ - def run[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]): Graph[VertexID, ED] = { + def run[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED], undirected: Boolean = true): + Graph[VertexID, ED] = { val ccGraph = graph.mapVertices { case (vid, _) => vid } - - def sendMessage(edge: EdgeTriplet[VertexID, ED]) = { - if (edge.srcAttr < edge.dstAttr) { - Iterator((edge.dstId, edge.srcAttr)) - } else if (edge.srcAttr > edge.dstAttr) { - Iterator((edge.srcId, edge.dstAttr)) - } else { - Iterator.empty + if (undirected) { + def sendMessage(edge: EdgeTriplet[VertexID, ED]) = { + if (edge.srcAttr < edge.dstAttr) { + Iterator((edge.dstId, edge.srcAttr)) + } else if (edge.srcAttr > edge.dstAttr) { + Iterator((edge.srcId, edge.dstAttr)) + } else { + Iterator.empty + } + } + val initialMessage = Long.MaxValue + Pregel(ccGraph, initialMessage, activeDirection = EdgeDirection.Both)( + vprog = (id, attr, msg) => math.min(attr, msg), + sendMsg = sendMessage, + mergeMsg = (a, b) => math.min(a, b)) + } else { + def sendMessage(edge: EdgeTriplet[VertexID, ED]) = { + if (edge.srcAttr < edge.dstAttr) { + Iterator((edge.dstId, edge.srcAttr)) + } else { + Iterator.empty + } } + val initialMessage = Long.MaxValue + Pregel(ccGraph, initialMessage, activeDirection = EdgeDirection.Out)( + vprog = (id, attr, msg) => math.min(attr, msg), + sendMsg = sendMessage, + mergeMsg = (a, b) => math.min(a, b)) } - val initialMessage = Long.MaxValue - Pregel(ccGraph, initialMessage)( - vprog = (id, attr, msg) => math.min(attr, msg), - sendMsg = sendMessage, - mergeMsg = (a, b) => math.min(a, b)) } // end of connectedComponents } diff --git a/graphx/src/test/scala/org/apache/spark/graphx/lib/ConnectedComponentsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/lib/ConnectedComponentsSuite.scala index 66612b381f..86da8f1b46 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/lib/ConnectedComponentsSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/lib/ConnectedComponentsSuite.scala @@ -80,4 +80,34 @@ class ConnectedComponentsSuite extends FunSuite with LocalSparkContext { } } // end of reverse chain connected components + test("Connected Components on a Toy Connected Graph") { + withSpark { sc => + // Create an RDD for the vertices + val users: RDD[(VertexID, (String, String))] = + sc.parallelize(Array((3L, ("rxin", "student")), (7L, ("jgonzal", "postdoc")), + (5L, ("franklin", "prof")), (2L, ("istoica", "prof")), + (4L, ("peter", "student")))) + // Create an RDD for edges + val relationships: RDD[Edge[String]] = + sc.parallelize(Array(Edge(3L, 7L, "collab"), Edge(5L, 3L, "advisor"), + Edge(2L, 5L, "colleague"), Edge(5L, 7L, "pi"), + Edge(4L, 0L, "student"), Edge(5L, 0L, "colleague"))) + // Edges are: + // 2 ---> 5 ---> 3 + // | \ + // V \| + // 4 ---> 0 7 + // + // Define a default user in case there are relationship with missing user + val defaultUser = ("John Doe", "Missing") + // Build the initial Graph + val graph = Graph(users, relationships, defaultUser) + val ccGraph = graph.connectedComponents(undirected = true) + val vertices = ccGraph.vertices.collect + for ( (id, cc) <- vertices ) { + assert(cc == 0) + } + } + } // end of toy connected components + } -- cgit v1.2.3 From 1bd5cefcae2769d48ad5ef4b8197193371c754da Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Mon, 13 Jan 2014 16:15:10 -0800 Subject: Remove aggregateNeighbors --- docs/graphx-programming-guide.md | 17 ------ .../scala/org/apache/spark/graphx/GraphOps.scala | 64 ++-------------------- .../org/apache/spark/graphx/GraphOpsSuite.scala | 26 --------- 3 files changed, 5 insertions(+), 102 deletions(-) (limited to 'graphx') diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md index 002ba0cf73..e6afd092be 100644 --- a/docs/graphx-programming-guide.md +++ b/docs/graphx-programming-guide.md @@ -519,23 +519,6 @@ val avgAgeOlderFollowers: VertexRDD[Double] = > are constant sized (e.g., floats and addition instead of lists and concatenation). More > precisely, the result of `mapReduceTriplets` should be sub-linear in the degree of each vertex. -Because it is often necessary to aggregate information about neighboring vertices we also provide an -alternative interface defined in [`GraphOps`][GraphOps]: - -{% highlight scala %} -def aggregateNeighbors[A]( - map: (VertexID, EdgeTriplet[VD, ED]) => Option[A], - reduce: (A, A) => A, - edgeDir: EdgeDirection) - : VertexRDD[A] -{% endhighlight %} - -The `aggregateNeighbors` operator is implemented directly on top of `mapReduceTriplets` but allows -the user to define the logic in a more vertex centric manner. Here the `map` function is provided -the vertex to which the message is sent as well as one of the edges and returns the optional message -value. The `edgeDir` determines whether the `map` function is run on `In`, `Out`, or `All` edges -adjacent to each vertex. - ### Computing Degree Information A common aggregation task is computing the degree of each vertex: the number of edges adjacent to 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 a0a40e2d9a..578eb331c1 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala @@ -55,60 +55,6 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) { } } - /** - * Computes a statistic for the neighborhood of each vertex. - * - * @param mapFunc the function applied to each edge adjacent to each vertex. The mapFunc can - * optionally return `None`, in which case it does not contribute to the final sum. - * @param reduceFunc the function used to merge the results of each map operation - * @param direction the direction of edges to consider (e.g., In, Out, Both). - * @tparam A the aggregation type - * - * @return an RDD containing tuples of vertex identifiers and - * their resulting value. Vertices with no neighbors will not appear in the RDD. - * - * @example We can use this function to compute the average follower - * age for each user: - * - * {{{ - * val graph: Graph[Int,Int] = GraphLoader.edgeListFile(sc, "webgraph") - * val averageFollowerAge: RDD[(Int, Int)] = - * graph.aggregateNeighbors[(Int,Double)]( - * (vid, edge) => Some((edge.otherVertex(vid).data, 1)), - * (a, b) => (a._1 + b._1, a._2 + b._2), - * -1, - * EdgeDirection.In) - * .mapValues{ case (sum,followers) => sum.toDouble / followers} - * }}} - */ - def aggregateNeighbors[A: ClassTag]( - mapFunc: (VertexID, EdgeTriplet[VD, ED]) => Option[A], - reduceFunc: (A, A) => A, - dir: EdgeDirection) - : VertexRDD[A] = { - // Define a new map function over edge triplets - val mf = (et: EdgeTriplet[VD,ED]) => { - // Compute the message to the dst vertex - val dst = - if (dir == EdgeDirection.In || dir == EdgeDirection.Both) { - mapFunc(et.dstId, et) - } else { Option.empty[A] } - // Compute the message to the source vertex - val src = - if (dir == EdgeDirection.Out || dir == EdgeDirection.Both) { - mapFunc(et.srcId, et) - } else { Option.empty[A] } - // construct the return array - (src, dst) match { - case (None, None) => Iterator.empty - case (Some(srcA),None) => Iterator((et.srcId, srcA)) - case (None, Some(dstA)) => Iterator((et.dstId, dstA)) - case (Some(srcA), Some(dstA)) => Iterator((et.srcId, srcA), (et.dstId, dstA)) - } - } - graph.mapReduceTriplets(mf, reduceFunc) - } // end of aggregateNeighbors - /** * Collect the neighbor vertex ids for each vertex. * @@ -152,11 +98,11 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) { * * @return the vertex set of neighboring vertex attributes for each vertex */ - def collectNeighbors(edgeDirection: EdgeDirection) : - VertexRDD[ Array[(VertexID, VD)] ] = { - val nbrs = graph.aggregateNeighbors[Array[(VertexID,VD)]]( - (vid, edge) => - Some(Array( (edge.otherVertexId(vid), edge.otherVertexAttr(vid)) )), + def collectNeighbors(edgeDirection: EdgeDirection): VertexRDD[Array[(VertexID, VD)]] = { + val nbrs = graph.mapReduceTriplets[Array[(VertexID,VD)]]( + edge => Iterator( + (edge.srcId, Array((edge.dstId, edge.dstAttr))), + (edge.dstId, Array((edge.srcId, edge.srcAttr)))), (a, b) => a ++ b, edgeDirection) diff --git a/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala index cd3c0bbd30..7a901409d5 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala @@ -8,32 +8,6 @@ import org.scalatest.FunSuite class GraphOpsSuite extends FunSuite with LocalSparkContext { - test("aggregateNeighbors") { - withSpark { sc => - val n = 3 - val star = - Graph.fromEdgeTuples(sc.parallelize((1 to n).map(x => (0: VertexID, x: VertexID))), 1) - - val indegrees = star.aggregateNeighbors( - (vid, edge) => Some(1), - (a: Int, b: Int) => a + b, - EdgeDirection.In) - assert(indegrees.collect().toSet === (1 to n).map(x => (x, 1)).toSet) - - val outdegrees = star.aggregateNeighbors( - (vid, edge) => Some(1), - (a: Int, b: Int) => a + b, - EdgeDirection.Out) - assert(outdegrees.collect().toSet === Set((0, n))) - - val noVertexValues = star.aggregateNeighbors[Int]( - (vid: VertexID, edge: EdgeTriplet[Int, Int]) => None, - (a: Int, b: Int) => throw new Exception("reduceFunc called unexpectedly"), - EdgeDirection.In) - assert(noVertexValues.collect().toSet === Set.empty[(VertexID, Int)]) - } - } - test("joinVertices") { withSpark { sc => val vertices = -- cgit v1.2.3 From ae4b75d94a4a0f2545e6d90d6f9b8f162bf70ded Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Mon, 13 Jan 2014 16:48:11 -0800 Subject: Add EdgeDirection.Either and use it to fix CC bug The bug was due to a misunderstanding of the activeSetOpt parameter to Graph.mapReduceTriplets. Passing EdgeDirection.Both causes mapReduceTriplets to run only on edges with *both* vertices in the active set. This commit adds EdgeDirection.Either, which causes mapReduceTriplets to run on edges with *either* vertex in the active set. This is what connected components needed. --- .../org/apache/spark/graphx/EdgeDirection.scala | 8 +++-- .../main/scala/org/apache/spark/graphx/Graph.scala | 9 +++-- .../scala/org/apache/spark/graphx/GraphOps.scala | 33 ++++++++++------- .../scala/org/apache/spark/graphx/Pregel.scala | 7 ++-- .../org/apache/spark/graphx/impl/GraphImpl.scala | 4 +++ .../spark/graphx/lib/ConnectedComponents.scala | 41 +++++++--------------- .../org/apache/spark/graphx/lib/PageRank.scala | 5 +-- .../graphx/lib/StronglyConnectedComponents.scala | 3 +- .../apache/spark/graphx/lib/TriangleCount.scala | 2 +- .../org/apache/spark/graphx/GraphOpsSuite.scala | 2 +- .../org/apache/spark/graphx/PregelSuite.scala | 2 +- .../graphx/lib/ConnectedComponentsSuite.scala | 2 +- 12 files changed, 64 insertions(+), 54 deletions(-) (limited to 'graphx') diff --git a/graphx/src/main/scala/org/apache/spark/graphx/EdgeDirection.scala b/graphx/src/main/scala/org/apache/spark/graphx/EdgeDirection.scala index 9d37f6513f..5b58a61bbd 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/EdgeDirection.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeDirection.scala @@ -6,11 +6,12 @@ package org.apache.spark.graphx class EdgeDirection private (private val name: String) extends Serializable { /** * Reverse the direction of an edge. An in becomes out, - * out becomes in and both remains both. + * out becomes in and both and either remain the same. */ def reverse: EdgeDirection = this match { case EdgeDirection.In => EdgeDirection.Out case EdgeDirection.Out => EdgeDirection.In + case EdgeDirection.Either => EdgeDirection.Either case EdgeDirection.Both => EdgeDirection.Both } @@ -32,6 +33,9 @@ object EdgeDirection { /** Edges originating from a vertex. */ final val Out = new EdgeDirection("Out") - /** All edges adjacent to a vertex. */ + /** Edges originating from *or* arriving at a vertex of interest. */ + final val Either = new EdgeDirection("Either") + + /** Edges originating from *and* arriving at a vertex of interest. */ final val Both = new EdgeDirection("Both") } 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 7d4f0de3d6..49705fdf5d 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala @@ -274,9 +274,12 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] { * of the map phase * * @param activeSetOpt optionally, a set of "active" vertices and a direction of edges to consider - * when running `mapFunc`. For example, if the direction is Out, `mapFunc` will only be run on - * edges originating from vertices in the active set. The active set must have the same index as - * the graph's vertices. + * when running `mapFunc`. If the direction is `In`, `mapFunc` will only be run on edges with + * destination in the active set. If the direction is `Out`, `mapFunc` will only be run on edges + * originating from vertices in the active set. If the direction is `Either`, `mapFunc` will be + * run on edges with *either* vertex in the active set. If the direction is `Both`, `mapFunc` will + * be run on edges with *both* vertices in the active set. The active set must have the same index + * as the graph's vertices. * * @example We can use this function to compute the in-degree of each * vertex 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 578eb331c1..66d5180020 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala @@ -38,7 +38,7 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) { * The degree of each vertex in the graph. * @note Vertices with no edges are not returned in the resulting RDD. */ - lazy val degrees: VertexRDD[Int] = degreesRDD(EdgeDirection.Both) + lazy val degrees: VertexRDD[Int] = degreesRDD(EdgeDirection.Either) /** * Computes the neighboring vertex degrees. @@ -50,7 +50,7 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) { graph.mapReduceTriplets(et => Iterator((et.dstId,1)), _ + _) } else if (edgeDirection == EdgeDirection.Out) { graph.mapReduceTriplets(et => Iterator((et.srcId,1)), _ + _) - } else { // EdgeDirection.both + } else { // EdgeDirection.Either graph.mapReduceTriplets(et => Iterator((et.srcId,1), (et.dstId,1)), _ + _) } } @@ -65,7 +65,7 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) { */ def collectNeighborIds(edgeDirection: EdgeDirection): VertexRDD[Array[VertexID]] = { val nbrs = - if (edgeDirection == EdgeDirection.Both) { + if (edgeDirection == EdgeDirection.Either) { graph.mapReduceTriplets[Array[VertexID]]( mapFunc = et => Iterator((et.srcId, Array(et.dstId)), (et.dstId, Array(et.srcId))), reduceFunc = _ ++ _ @@ -79,7 +79,8 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) { mapFunc = et => Iterator((et.dstId, Array(et.srcId))), reduceFunc = _ ++ _) } else { - throw new SparkException("It doesn't make sense to collect neighbor ids without a direction.") + throw new SparkException("It doesn't make sense to collect neighbor ids without a " + + "direction. (EdgeDirection.Both is not supported; use EdgeDirection.Either instead.)") } graph.vertices.leftZipJoin(nbrs) { (vid, vdata, nbrsOpt) => nbrsOpt.getOrElse(Array.empty[VertexID]) @@ -100,11 +101,19 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) { */ def collectNeighbors(edgeDirection: EdgeDirection): VertexRDD[Array[(VertexID, VD)]] = { val nbrs = graph.mapReduceTriplets[Array[(VertexID,VD)]]( - edge => Iterator( - (edge.srcId, Array((edge.dstId, edge.dstAttr))), - (edge.dstId, Array((edge.srcId, edge.srcAttr)))), - (a, b) => a ++ b, - edgeDirection) + edge => { + val msgToSrc = (edge.srcId, Array((edge.dstId, edge.dstAttr))) + val msgToDst = (edge.dstId, Array((edge.srcId, edge.srcAttr))) + edgeDirection match { + case EdgeDirection.Either => Iterator(msgToSrc, msgToDst) + case EdgeDirection.In => Iterator(msgToDst) + case EdgeDirection.Out => Iterator(msgToSrc) + case EdgeDirection.Both => + throw new SparkException("collectNeighbors does not support EdgeDirection.Both. Use" + + "EdgeDirection.Either instead.") + } + }, + (a, b) => a ++ b) graph.vertices.leftZipJoin(nbrs) { (vid, vdata, nbrsOpt) => nbrsOpt.getOrElse(Array.empty[(VertexID, VD)]) @@ -237,7 +246,7 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) { def pregel[A: ClassTag]( initialMsg: A, maxIterations: Int = Int.MaxValue, - activeDirection: EdgeDirection = EdgeDirection.Out)( + activeDirection: EdgeDirection = EdgeDirection.Either)( vprog: (VertexID, VD, A) => VD, sendMsg: EdgeTriplet[VD, ED] => Iterator[(VertexID,A)], mergeMsg: (A, A) => A) @@ -271,8 +280,8 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) { * * @see [[org.apache.spark.graphx.lib.ConnectedComponents]] */ - def connectedComponents(undirected: Boolean = true): Graph[VertexID, ED] = { - ConnectedComponents.run(graph, undirected) + def connectedComponents(): Graph[VertexID, ED] = { + ConnectedComponents.run(graph) } /** 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 83e28d0ab2..75b44ddac9 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala @@ -67,7 +67,10 @@ object Pregel { * * @param activeDirection the direction of edges incident to a vertex that received a message in * the previous round on which to run `sendMsg`. For example, if this is `EdgeDirection.Out`, only - * out-edges of vertices that received a message in the previous round will run. + * out-edges of vertices that received a message in the previous round will run. The default is + * `EdgeDirection.Either`, which will run `sendMsg` on edges where either side received a message + * in the previous round. If this is `EdgeDirection.Both`, `sendMsg` will only run on edges where + * *both* vertices received a message. * * @param vprog the user-defined vertex program which runs on each * vertex and receives the inbound message and computes a new vertex @@ -90,7 +93,7 @@ object Pregel { */ def apply[VD: ClassTag, ED: ClassTag, A: ClassTag] (graph: Graph[VD, ED], initialMsg: A, maxIterations: Int = Int.MaxValue, - activeDirection: EdgeDirection = EdgeDirection.Out)( + activeDirection: EdgeDirection = EdgeDirection.Either)( vprog: (VertexID, VD, A) => VD, sendMsg: EdgeTriplet[VD, ED] => Iterator[(VertexID,A)], mergeMsg: (A, A) => A) 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 6a2abc71cc..c21f8935d9 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala @@ -275,6 +275,10 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( } else { edgePartition.iterator.filter(e => vPart.isActive(e.srcId) && vPart.isActive(e.dstId)) } + case Some(EdgeDirection.Either) => + // TODO: Because we only have a clustered index on the source vertex ID, we can't filter + // the index here. Instead we have to scan all edges and then do the filter. + edgePartition.iterator.filter(e => vPart.isActive(e.srcId) || vPart.isActive(e.dstId)) case Some(EdgeDirection.Out) => if (activeFraction < 0.8) { edgePartition.indexIterator(srcVertexID => vPart.isActive(srcVertexID)) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/ConnectedComponents.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/ConnectedComponents.scala index d078d2acdb..d057c933d7 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/ConnectedComponents.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/ConnectedComponents.scala @@ -19,37 +19,22 @@ object ConnectedComponents { * @return a graph with vertex attributes containing the smallest vertex in each * connected component */ - def run[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED], undirected: Boolean = true): + def run[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]): Graph[VertexID, ED] = { val ccGraph = graph.mapVertices { case (vid, _) => vid } - if (undirected) { - def sendMessage(edge: EdgeTriplet[VertexID, ED]) = { - if (edge.srcAttr < edge.dstAttr) { - Iterator((edge.dstId, edge.srcAttr)) - } else if (edge.srcAttr > edge.dstAttr) { - Iterator((edge.srcId, edge.dstAttr)) - } else { - Iterator.empty - } + def sendMessage(edge: EdgeTriplet[VertexID, ED]) = { + if (edge.srcAttr < edge.dstAttr) { + Iterator((edge.dstId, edge.srcAttr)) + } else if (edge.srcAttr > edge.dstAttr) { + Iterator((edge.srcId, edge.dstAttr)) + } else { + Iterator.empty } - val initialMessage = Long.MaxValue - Pregel(ccGraph, initialMessage, activeDirection = EdgeDirection.Both)( - vprog = (id, attr, msg) => math.min(attr, msg), - sendMsg = sendMessage, - mergeMsg = (a, b) => math.min(a, b)) - } else { - def sendMessage(edge: EdgeTriplet[VertexID, ED]) = { - if (edge.srcAttr < edge.dstAttr) { - Iterator((edge.dstId, edge.srcAttr)) - } else { - Iterator.empty - } - } - val initialMessage = Long.MaxValue - Pregel(ccGraph, initialMessage, activeDirection = EdgeDirection.Out)( - vprog = (id, attr, msg) => math.min(attr, msg), - sendMsg = sendMessage, - mergeMsg = (a, b) => math.min(a, b)) } + val initialMessage = Long.MaxValue + Pregel(ccGraph, initialMessage, activeDirection = EdgeDirection.Either)( + vprog = (id, attr, msg) => math.min(attr, msg), + sendMsg = sendMessage, + mergeMsg = (a, b) => math.min(a, b)) } // end of connectedComponents } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala index cf95267e77..6ced2462eb 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala @@ -77,7 +77,7 @@ object PageRank extends Logging { val initialMessage = 0.0 // Execute pregel for a fixed number of iterations. - Pregel(pagerankGraph, initialMessage, numIter)( + Pregel(pagerankGraph, initialMessage, numIter, activeDirection = EdgeDirection.Out)( vertexProgram, sendMessage, messageCombiner) } @@ -153,7 +153,8 @@ object PageRank extends Logging { val initialMessage = resetProb / (1.0 - resetProb) // Execute a dynamic version of Pregel. - Pregel(pagerankGraph, initialMessage)(vertexProgram, sendMessage, messageCombiner) + Pregel(pagerankGraph, initialMessage, activeDirection = EdgeDirection.Out)( + vertexProgram, sendMessage, messageCombiner) .mapVertices((vid, attr) => attr._1) } // end of deltaPageRank diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/StronglyConnectedComponents.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/StronglyConnectedComponents.scala index 43c4b9cf2d..edffbcc5ac 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/StronglyConnectedComponents.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/StronglyConnectedComponents.scala @@ -53,7 +53,8 @@ object StronglyConnectedComponents { // collect min of all my neighbor's scc values, update if it's smaller than mine // then notify any neighbors with scc values larger than mine - sccWorkGraph = Pregel[(VertexID, Boolean), ED, VertexID](sccWorkGraph, Long.MaxValue)( + sccWorkGraph = Pregel[(VertexID, Boolean), ED, VertexID]( + sccWorkGraph, Long.MaxValue, activeDirection = EdgeDirection.Out)( (vid, myScc, neighborScc) => (math.min(myScc._1, neighborScc), myScc._2), e => { if (e.srcId < e.dstId) { diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/TriangleCount.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/TriangleCount.scala index 58da9e3aed..d3e22b176c 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/TriangleCount.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/TriangleCount.scala @@ -28,7 +28,7 @@ object TriangleCount { // Construct set representations of the neighborhoods val nbrSets: VertexRDD[VertexSet] = - g.collectNeighborIds(EdgeDirection.Both).mapValues { (vid, nbrs) => + g.collectNeighborIds(EdgeDirection.Either).mapValues { (vid, nbrs) => val set = new VertexSet(4) var i = 0 while (i < nbrs.size) { diff --git a/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala index 7a901409d5..280f50e39a 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala @@ -28,7 +28,7 @@ class GraphOpsSuite extends FunSuite with LocalSparkContext { val chain = (0 until 100).map(x => (x, (x+1)%100) ) val rawEdges = sc.parallelize(chain, 3).map { case (s,d) => (s.toLong, d.toLong) } val graph = Graph.fromEdgeTuples(rawEdges, 1.0).cache() - val nbrs = graph.collectNeighborIds(EdgeDirection.Both).cache() + val nbrs = graph.collectNeighborIds(EdgeDirection.Either).cache() assert(nbrs.count === chain.size) assert(graph.numVertices === nbrs.count) nbrs.collect.foreach { case (vid, nbrs) => assert(nbrs.size === 2) } diff --git a/graphx/src/test/scala/org/apache/spark/graphx/PregelSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/PregelSuite.scala index 1ff3d75633..bceff11b8e 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/PregelSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/PregelSuite.scala @@ -32,7 +32,7 @@ class PregelSuite extends FunSuite with LocalSparkContext { Set((1: VertexID, 1)) ++ (2 to n).map(x => (x: VertexID, 0)).toSet) val result = Pregel(chainWithSeed, 0)( (vid, attr, msg) => math.max(msg, attr), - et => Iterator((et.dstId, et.srcAttr)), + et => if (et.dstAttr != et.srcAttr) Iterator((et.dstId, et.srcAttr)) else Iterator.empty, (a: Int, b: Int) => math.max(a, b)) assert(result.vertices.collect.toSet === chain.vertices.mapValues { (vid, attr) => attr + 1 }.collect.toSet) diff --git a/graphx/src/test/scala/org/apache/spark/graphx/lib/ConnectedComponentsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/lib/ConnectedComponentsSuite.scala index 86da8f1b46..27c8705bca 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/lib/ConnectedComponentsSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/lib/ConnectedComponentsSuite.scala @@ -102,7 +102,7 @@ class ConnectedComponentsSuite extends FunSuite with LocalSparkContext { val defaultUser = ("John Doe", "Missing") // Build the initial Graph val graph = Graph(users, relationships, defaultUser) - val ccGraph = graph.connectedComponents(undirected = true) + val ccGraph = graph.connectedComponents() val vertices = ccGraph.vertices.collect for ( (id, cc) <- vertices ) { assert(cc == 0) -- cgit v1.2.3 From cfe4a29dcb516ceae5f243ac3b5d0c3a505d7f5a Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Mon, 13 Jan 2014 17:15:21 -0800 Subject: Improvements in example code for the programming guide as well as adding serialization support for GraphImpl to address issues with failed closure capture. --- docs/graphx-programming-guide.md | 39 ++++++++++++---------- .../org/apache/spark/graphx/impl/GraphImpl.scala | 3 ++ 2 files changed, 25 insertions(+), 17 deletions(-) (limited to 'graphx') diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md index e6afd092be..c82c3d7358 100644 --- a/docs/graphx-programming-guide.md +++ b/docs/graphx-programming-guide.md @@ -478,24 +478,26 @@ def mapReduceTriplets[A]( The [`mapReduceTriplets`][Graph.mapReduceTriplets] operator takes a user defined map function which is applied to each triplet and can yield *messages* destined to either (none or both) vertices in -the triplet. We currently only support messages destined to the source or destination vertex of the -triplet to enable optimized preaggregation. The user defined `reduce` function combines the +the triplet. To facilitate optimized pre-aggregation, we currently only support messages destined +to the source or destination vertex of the triplet. The user defined `reduce` function combines the messages destined to each vertex. The `mapReduceTriplets` operator returns a `VertexRDD[A]` -containing the aggregate message to each vertex. Vertices that do not receive a message are not -included in the returned `VertexRDD`. +containing the aggregate message (of type `A`) destined to each vertex. Vertices that do not +receive a message are not included in the returned `VertexRDD`. -> Note that `mapReduceTriplets takes an additional optional `activeSet` (see API docs) which +> Note that `mapReduceTriplets` takes an additional optional `activeSet` (see API docs) which > restricts the map phase to edges adjacent to the vertices in the provided `VertexRDD`. Restricting > computation to triplets adjacent to a subset of the vertices is often necessary in incremental > iterative computation and is a key part of the GraphX implementation of Pregel. -We can use the `mapReduceTriplets` operator to collect information about adjacent vertices. For -example if we wanted to compute the average age of followers who are older that each user we could -do the following. +In the following example we use the `mapReduceTriplets` operator to compute the average age of the +more senior followers of each user. {% highlight scala %} -// Graph with age as the vertex property -val graph: Graph[Double, String] = getFromSomewhereElse() +// Import Random graph generation library +import org.apache.spark.graphx.util.GraphGenerators +// Create a graph with "age" as the vertex property. Here we use a random graph for simplicity. +val graph: Graph[Double, Int] = + GraphGenerators.logNormalGraph(sc, numVertices = 100).mapVertices( (id, _) => id.toDouble ) // Compute the number of older followers and their total age val olderFollowers: VertexRDD[(Int, Double)] = graph.mapReduceTriplets[(Int, Double)]( triplet => { // Map Function @@ -511,13 +513,16 @@ val olderFollowers: VertexRDD[(Int, Double)] = graph.mapReduceTriplets[(Int, Dou (a, b) => (a._1 + b._1, a._2 + b._2) // Reduce Function ) // Divide total age by number of older followers to get average age of older followers -val avgAgeOlderFollowers: VertexRDD[Double] = - olderFollowers.mapValues { case (count, totalAge) => totalAge / count } +val avgAgeOfOlderFollowers: VertexRDD[Double] = + olderFollowers.mapValues( (id, value) => value match { case (count, totalAge) => totalAge / count } ) +// Display the results +avgAgeOfOlderFollowers.collect.foreach(println(_)) {% endhighlight %} > Note that the `mapReduceTriplets` operation performs optimally when the messages (and their sums) > are constant sized (e.g., floats and addition instead of lists and concatenation). More -> precisely, the result of `mapReduceTriplets` should be sub-linear in the degree of each vertex. +> precisely, the result of `mapReduceTriplets` should ideally be sub-linear in the degree of each +> vertex. ### Computing Degree Information @@ -529,13 +534,13 @@ compute the max in, out, and total degrees: {% highlight scala %} // Define a reduce operation to compute the highest degree vertex -def maxReduce(a: (VertexId, Int), b: (VertexId, Int)): (VertexId, Int) = { +def max(a: (VertexID, Int), b: (VertexID, Int)): (VertexID, Int) = { if (a._2 > b._2) a else b } // Compute the max degrees -val maxInDegree: (VertexId, Int) = graph.inDegrees.reduce(maxReduce) -val maxOutDegree: (VertexId, Int) = graph.outDegrees.reduce(maxReduce) -val maxDegrees: (VertexId, Int) = graph.degrees.reduce(maxReduce) +val maxInDegree: (VertexID, Int) = graph.inDegrees.reduce(max) +val maxOutDegree: (VertexID, Int) = graph.outDegrees.reduce(max) +val maxDegrees: (VertexID, Int) = graph.degrees.reduce(max) {% endhighlight %} 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 c21f8935d9..916eb9763c 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala @@ -32,6 +32,9 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( @transient val replicatedVertexView: ReplicatedVertexView[VD]) extends Graph[VD, ED] with Serializable { + /** Default construct is provided to support serialization */ + protected def this() = this(null, null, null, null) + /** Return a RDD that brings edges together with their source and destination vertices. */ @transient override val triplets: RDD[EdgeTriplet[VD, ED]] = { val vdTag = classTag[VD] -- cgit v1.2.3 From 02a8f54bfa4572908d2d605a85e7a5adf9a36fbc Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 13 Jan 2014 17:40:36 -0800 Subject: Miscel doc update. --- .../main/scala/org/apache/spark/graphx/Edge.scala | 2 +- .../org/apache/spark/graphx/EdgeTriplet.scala | 4 +- .../main/scala/org/apache/spark/graphx/Graph.scala | 75 ++++++++--------- .../org/apache/spark/graphx/GraphLoader.scala | 14 ++-- .../apache/spark/graphx/PartitionStrategy.scala | 1 + .../scala/org/apache/spark/graphx/Pregel.scala | 14 ++-- .../apache/spark/graphx/impl/EdgePartition.scala | 7 +- .../org/apache/spark/graphx/impl/GraphImpl.scala | 15 ++-- .../org/apache/spark/graphx/lib/Analytics.scala | 12 +-- .../spark/graphx/lib/ConnectedComponents.scala | 2 +- .../org/apache/spark/graphx/lib/PageRank.scala | 2 +- .../org/apache/spark/graphx/lib/SVDPlusPlus.scala | 95 ++++++++++++++-------- .../graphx/lib/StronglyConnectedComponents.scala | 1 + .../apache/spark/graphx/lib/TriangleCount.scala | 43 +++++----- .../scala/org/apache/spark/graphx/package.scala | 5 +- .../apache/spark/graphx/util/BytecodeUtils.scala | 7 +- .../util/collection/PrimitiveKeyOpenHashMap.scala | 2 +- 17 files changed, 158 insertions(+), 143 deletions(-) (limited to 'graphx') diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala b/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala index 85f27d2c8d..6c396c3dbe 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala @@ -13,7 +13,7 @@ case class Edge[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED] /** The vertex id of the target vertex. */ var dstId: VertexID = 0, /** The attribute associated with the edge. */ - var attr: ED = nullValue[ED]) + var attr: ED = null.asInstanceOf[ED]) extends Serializable { /** 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 057d63a0ac..4253b24b5a 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/EdgeTriplet.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeTriplet.scala @@ -1,7 +1,5 @@ package org.apache.spark.graphx -import org.apache.spark.graphx.impl.VertexPartition - /** * An edge triplet represents an edge along with the vertex attributes of its neighboring vertices. * @@ -47,5 +45,5 @@ class EdgeTriplet[VD, ED] extends Edge[ED] { def vertexAttr(vid: VertexID): VD = if (srcId == vid) srcAttr else { assert(dstId == vid); dstAttr } - override def toString() = ((srcId, srcAttr), (dstId, dstAttr), attr).toString() + override def toString = ((srcId, srcAttr), (dstId, dstAttr), attr).toString() } 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 7d4f0de3d6..d2ba6fde4a 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala @@ -45,7 +45,8 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] { /** * An RDD containing the edge triplets, which are edges along with the vertex data associated with - * the adjacent vertices. + * the adjacent vertices. The caller should use [[edges]] if the vertex data are not needed, i.e. + * if only the edge data and adjacent vertex ids are needed. * * @return an RDD containing edge triplets * @@ -54,13 +55,9 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] { * different color. * {{{ * type Color = Int - * val graph: Graph[Color, Int] = Graph.textFile("hdfs://file.tsv") + * val graph: Graph[Color, Int] = GraphLoader.edgeListFile("hdfs://file.tsv") * val numInvalid = graph.triplets.map(e => if (e.src.data == e.dst.data) 1 else 0).sum * }}} - * - * @see `edges` if only the edge data and adjacent vertex ids are - * required. - * */ val triplets: RDD[EdgeTriplet[VD, ED]] @@ -68,9 +65,8 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] { * Caches the vertices and edges associated with this graph at the specified storage level. * * @param newLevel the level at which to cache the graph. - - * @return A reference to this graph for convenience. * + * @return A reference to this graph for convenience. */ def persist(newLevel: StorageLevel = StorageLevel.MEMORY_ONLY): Graph[VD, ED] @@ -159,8 +155,8 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] { * @tparam ED2 the new edge data type * */ - def mapEdges[ED2: ClassTag]( - map: (PartitionID, Iterator[Edge[ED]]) => Iterator[ED2]): Graph[VD, ED2] + def mapEdges[ED2: ClassTag](map: (PartitionID, Iterator[Edge[ED]]) => Iterator[ED2]) + : Graph[VD, ED2] /** * Transforms each edge attribute using the map function, passing it the adjacent vertex attributes @@ -203,9 +199,8 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] { * @tparam ED2 the new edge data type * */ - def mapTriplets[ED2: ClassTag]( - map: (PartitionID, Iterator[EdgeTriplet[VD, ED]]) => Iterator[ED2]): - Graph[VD, ED2] + def mapTriplets[ED2: ClassTag](map: (PartitionID, Iterator[EdgeTriplet[VD, ED]]) => Iterator[ED2]) + : Graph[VD, ED2] /** * Reverses all edges in the graph. If this graph contains an edge from a to b then the returned @@ -233,8 +228,10 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] { * @return the subgraph containing only the vertices and edges that * satisfy the predicates */ - def subgraph(epred: EdgeTriplet[VD,ED] => Boolean = (x => true), - vpred: (VertexID, VD) => Boolean = ((v,d) => true) ): Graph[VD, ED] + def subgraph( + epred: EdgeTriplet[VD,ED] => Boolean = (x => true), + vpred: (VertexID, VD) => Boolean = ((v, d) => true)) + : Graph[VD, ED] /** * Restricts the graph to only the vertices and edges that are also in `other`, but keeps the @@ -249,14 +246,12 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] { * Merges multiple edges between two vertices into a single edge. For correct results, the graph * must have been partitioned using [[partitionBy]]. * - * @tparam ED2 the type of the resulting edge data after grouping. - * - * @param f the user-supplied commutative associative function to merge edge attributes for - * duplicate edges. + * @param merge the user-supplied commutative associative function to merge edge attributes + * for duplicate edges. * * @return The resulting graph with a single edge for each (source, dest) vertex pair. */ - def groupEdges(merge: (ED, ED) => ED): Graph[VD,ED] + def groupEdges(merge: (ED, ED) => ED): Graph[VD, ED] /** * Computes statistics about the neighboring edges and vertices of each vertex. The user supplied @@ -270,7 +265,7 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] { * more messages to neighboring vertices * * @param reduceFunc the user defined reduce function which should - * be commutative and assosciative and is used to combine the output + * be commutative and associative and is used to combine the output * of the map phase * * @param activeSetOpt optionally, a set of "active" vertices and a direction of edges to consider @@ -301,21 +296,20 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] { /** * Joins the vertices with entries in the `table` RDD and merges the results using `mapFunc`. The - * input table should contain at most one entry for each vertex. If no entry in `table` is + * input table should contain at most one entry for each vertex. If no entry in `other` is * provided for a particular vertex in the graph, the map function receives `None`. * * @tparam U the type of entry in the table of updates * @tparam VD2 the new vertex value type * - * @param table the table to join with the vertices in the graph. - * The table should contain at most one entry for each vertex. - * - * @param mapFunc the function used to compute the new vertex - * values. The map function is invoked for all vertices, even those - * that do not have a corresponding entry in the table. + * @param other the table to join with the vertices in the graph. + * The table should contain at most one entry for each vertex. + * @param mapFunc the function used to compute the new vertex values. + * The map function is invoked for all vertices, even those + * that do not have a corresponding entry in the table. * * @example This function is used to update the vertices with new values based on external data. - * For example we could add the out-degree to each vertex record: + * For example we could add the out-degree to each vertex record: * * {{{ * val rawGraph: Graph[_, _] = Graph.textFile("webgraph") @@ -324,20 +318,20 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] { * (vid, data, optDeg) => optDeg.getOrElse(0) * } * }}} - * */ - def outerJoinVertices[U: ClassTag, VD2: ClassTag](table: RDD[(VertexID, U)]) + def outerJoinVertices[U: ClassTag, VD2: ClassTag](other: RDD[(VertexID, U)]) (mapFunc: (VertexID, VD, Option[U]) => VD2) : Graph[VD2, ED] + /** + * The associated [[GraphOps]] object. + */ // Save a copy of the GraphOps object so there is always one unique GraphOps object // for a given Graph object, and thus the lazy vals in GraphOps would work as intended. val ops = new GraphOps(this) } // end of Graph - - /** * The Graph object contains a collection of routines used to construct graphs from RDDs. */ @@ -357,7 +351,8 @@ object Graph { def fromEdgeTuples[VD: ClassTag]( rawEdges: RDD[(VertexID, VertexID)], defaultValue: VD, - uniqueEdges: Option[PartitionStrategy] = None): Graph[VD, Int] = { + uniqueEdges: Option[PartitionStrategy] = None): Graph[VD, Int] = + { val edges = rawEdges.map(p => Edge(p._1, p._2, 1)) val graph = GraphImpl(edges, defaultValue) uniqueEdges match { @@ -391,10 +386,8 @@ object Graph { * @tparam ED the edge attribute type * @param vertices the "set" of vertices and their attributes * @param edges the collection of edges in the graph - * @param defaultVertexAttr the default vertex attribute to use for - * vertices that are mentioned in edges but not in vertices - * @param partitionStrategy the partition strategy to use when - * partitioning the edges + * @param defaultVertexAttr the default vertex attribute to use for vertices that are + * mentioned in edges but not in vertices */ def apply[VD: ClassTag, ED: ClassTag]( vertices: RDD[(VertexID, VD)], @@ -406,9 +399,9 @@ object Graph { /** * Implicitly extracts the [[GraphOps]] member from a graph. * - * To improve modularity the Graph type only contains a small set of basic operations. All the - * convenience operations are defined in the [[GraphOps]] class which may be shared across multiple - * graph implementations. + * To improve modularity the Graph type only contains a small set of basic operations. + * All the convenience operations are defined in the [[GraphOps]] class which may be + * shared across multiple graph implementations. */ implicit def graphToGraphOps[VD: ClassTag, ED: ClassTag](g: Graph[VD, ED]) = g.ops } // end of Graph object 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 3c06a403ea..7bdb101efb 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala @@ -1,12 +1,7 @@ package org.apache.spark.graphx -import java.util.{Arrays => JArrays} -import scala.reflect.ClassTag - -import org.apache.spark.graphx.impl.EdgePartitionBuilder import org.apache.spark.{Logging, SparkContext} -import org.apache.spark.graphx.impl.{EdgePartition, GraphImpl} -import org.apache.spark.util.collection.PrimitiveVector +import org.apache.spark.graphx.impl.{EdgePartitionBuilder, GraphImpl} /** * Provides utilities for loading [[Graph]]s from files. @@ -31,19 +26,20 @@ object GraphLoader extends Logging { * 1 8 * }}} * - * @param sc + * @param sc SparkContext * @param path the path to the file (e.g., /home/data/file or hdfs://file) * @param canonicalOrientation whether to orient edges in the positive * direction * @param minEdgePartitions the number of partitions for the * the edge RDD - * @tparam ED */ def edgeListFile( sc: SparkContext, path: String, canonicalOrientation: Boolean = false, - minEdgePartitions: Int = 1): Graph[Int, Int] = { + minEdgePartitions: Int = 1) + : Graph[Int, Int] = + { val startTime = System.currentTimeMillis // Parse the edge data table directly into edge partitions 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 fc7635a033..b9ccd8765e 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala @@ -5,6 +5,7 @@ package org.apache.spark.graphx * vertex IDs. */ sealed trait PartitionStrategy extends Serializable { + /** Returns the partition number for a given edge. */ def getPartition(src: VertexID, dst: VertexID, numParts: PartitionID): PartitionID } 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 83e28d0ab2..ce4eb53829 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala @@ -89,14 +89,16 @@ object Pregel { * */ def apply[VD: ClassTag, ED: ClassTag, A: ClassTag] - (graph: Graph[VD, ED], initialMsg: A, maxIterations: Int = Int.MaxValue, - activeDirection: EdgeDirection = EdgeDirection.Out)( - vprog: (VertexID, VD, A) => VD, + (graph: Graph[VD, ED], + initialMsg: A, + maxIterations: Int = Int.MaxValue, + activeDirection: EdgeDirection = EdgeDirection.Out) + (vprog: (VertexID, VD, A) => VD, sendMsg: EdgeTriplet[VD, ED] => Iterator[(VertexID,A)], mergeMsg: (A, A) => A) - : Graph[VD, ED] = { - - var g = graph.mapVertices( (vid, vdata) => vprog(vid, vdata, initialMsg) ).cache() + : Graph[VD, ED] = + { + var g = graph.mapVertices((vid, vdata) => vprog(vid, vdata, initialMsg)).cache() // compute the messages var messages = g.mapReduceTriplets(sendMsg, mergeMsg) var activeMessages = messages.count() 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 a03e73ee79..d4d71627e1 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala @@ -100,10 +100,9 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) */ def groupEdges(merge: (ED, ED) => ED): EdgePartition[ED] = { val builder = new EdgePartitionBuilder[ED] - var firstIter: Boolean = true - var currSrcId: VertexID = nullValue[VertexID] - var currDstId: VertexID = nullValue[VertexID] - var currAttr: ED = nullValue[ED] + var currSrcId: VertexID = null.asInstanceOf[VertexID] + var currDstId: VertexID = null.asInstanceOf[VertexID] + var currAttr: ED = null.asInstanceOf[ED] var i = 0 while (i < size) { if (i > 0 && currSrcId == srcIds(i) && currDstId == dstIds(i)) { 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 6a2abc71cc..9e39519200 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala @@ -249,8 +249,8 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( // For each vertex, replicate its attribute only to partitions where it is // in the relevant position in an edge. - val mapUsesSrcAttr = accessesVertexAttr[VD, ED](mapFunc, "srcAttr") - val mapUsesDstAttr = accessesVertexAttr[VD, ED](mapFunc, "dstAttr") + val mapUsesSrcAttr = accessesVertexAttr(mapFunc, "srcAttr") + val mapUsesDstAttr = accessesVertexAttr(mapFunc, "dstAttr") val vs = activeSetOpt match { case Some((activeSet, _)) => replicatedVertexView.get(mapUsesSrcAttr, mapUsesDstAttr, activeSet) @@ -308,10 +308,12 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( } // end of mapReduceTriplets override def outerJoinVertices[U: ClassTag, VD2: ClassTag] - (updates: RDD[(VertexID, U)])(updateF: (VertexID, VD, Option[U]) => VD2): Graph[VD2, ED] = { + (other: RDD[(VertexID, U)]) + (updateF: (VertexID, VD, Option[U]) => VD2): Graph[VD2, ED] = + { if (classTag[VD] equals classTag[VD2]) { // updateF preserves type, so we can use incremental replication - val newVerts = vertices.leftJoin(updates)(updateF) + val newVerts = vertices.leftJoin(other)(updateF) val changedVerts = vertices.asInstanceOf[VertexRDD[VD2]].diff(newVerts) val newReplicatedVertexView = new ReplicatedVertexView[VD2]( changedVerts, edges, routingTable, @@ -319,12 +321,13 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( new GraphImpl(newVerts, edges, routingTable, newReplicatedVertexView) } else { // updateF does not preserve type, so we must re-replicate all vertices - val newVerts = vertices.leftJoin(updates)(updateF) + val newVerts = vertices.leftJoin(other)(updateF) GraphImpl(newVerts, edges, routingTable) } } - private def accessesVertexAttr[VD, ED](closure: AnyRef, attrName: String): Boolean = { + /** Test whether the closure accesses the the attribute with name `attrName`. */ + private def accessesVertexAttr(closure: AnyRef, attrName: String): Boolean = { try { BytecodeUtils.invokedMethod(closure, classOf[EdgeTriplet[VD, ED]], attrName) } catch { 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 8c35f4206e..d5e1de1ce0 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/Analytics.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/Analytics.scala @@ -37,14 +37,12 @@ object Analytics extends Logging { case "pagerank" => var tol: Float = 0.001F var outFname = "" - var numVPart = 4 var numEPart = 4 var partitionStrategy: Option[PartitionStrategy] = None options.foreach{ case ("tol", v) => tol = v.toFloat case ("output", v) => outFname = v - case ("numVPart", v) => numVPart = v.toInt case ("numEPart", v) => numEPart = v.toInt case ("partStrategy", v) => partitionStrategy = Some(pickPartitioner(v)) case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) @@ -90,16 +88,12 @@ object Analytics extends Logging { case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) } - if(!isDynamic && numIter == Int.MaxValue) { + if (!isDynamic && numIter == Int.MaxValue) { println("Set number of iterations!") sys.exit(1) } println("======================================") println("| Connected Components |") - println("--------------------------------------") - println(" Using parameters:") - println(" \tDynamic: " + isDynamic) - println(" \tNumIter: " + numIter) println("======================================") val sc = new SparkContext(host, "ConnectedComponents(" + fname + ")", conf) @@ -112,20 +106,18 @@ object Analytics extends Logging { sc.stop() case "triangles" => - var numVPart = 4 var numEPart = 4 // TriangleCount requires the graph to be partitioned var partitionStrategy: PartitionStrategy = RandomVertexCut options.foreach{ case ("numEPart", v) => numEPart = v.toInt - case ("numVPart", v) => numVPart = v.toInt case ("partStrategy", v) => partitionStrategy = pickPartitioner(v) case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) } println("======================================") println("| Triangle Count |") - println("--------------------------------------") + println("======================================") val sc = new SparkContext(host, "TriangleCount(" + fname + ")", conf) val graph = GraphLoader.edgeListFile(sc, fname, canonicalOrientation = true, minEdgePartitions = numEPart).partitionBy(partitionStrategy).cache() diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/ConnectedComponents.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/ConnectedComponents.scala index d078d2acdb..da03d99264 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/ConnectedComponents.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/ConnectedComponents.scala @@ -4,7 +4,7 @@ import scala.reflect.ClassTag import org.apache.spark.graphx._ - +/** Connected components algorithm. */ object ConnectedComponents { /** * Compute the connected component membership of each vertex and return a graph with the vertex diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala index cf95267e77..853ef38712 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala @@ -5,7 +5,7 @@ import scala.reflect.ClassTag import org.apache.spark.Logging import org.apache.spark.graphx._ - +/** PageRank algorithm implementation. */ object PageRank extends Logging { /** diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala index f5570daec1..fa6b1db29b 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala @@ -1,11 +1,12 @@ package org.apache.spark.graphx.lib -import org.apache.spark.rdd._ -import org.apache.spark.graphx._ import scala.util.Random import org.apache.commons.math.linear._ +import org.apache.spark.rdd._ +import org.apache.spark.graphx._ -class SVDPlusPlusConf( // SVDPlusPlus parameters +/** Configuration parameters for SVDPlusPlus. */ +class SVDPlusPlusConf( var rank: Int, var maxIters: Int, var minVal: Double, @@ -15,11 +16,15 @@ class SVDPlusPlusConf( // SVDPlusPlus parameters var gamma6: Double, var gamma7: Double) extends Serializable +/** Implementation of SVD++ algorithm. */ object SVDPlusPlus { /** - * Implement SVD++ based on "Factorization Meets the Neighborhood: a Multifaceted Collaborative Filtering Model", - * paper is available at [[http://public.research.att.com/~volinsky/netflix/kdd08koren.pdf]]. - * The prediction rule is rui = u + bu + bi + qi*(pu + |N(u)|^(-0.5)*sum(y)), see the details on page 6. + * Implement SVD++ based on "Factorization Meets the Neighborhood: + * a Multifaceted Collaborative Filtering Model", + * available at [[http://public.research.att.com/~volinsky/netflix/kdd08koren.pdf]]. + * + * The prediction rule is rui = u + bu + bi + qi*(pu + |N(u)|^(-0.5)*sum(y)), + * see the details on page 6. * * @param edges edges for constructing the graph * @@ -27,16 +32,16 @@ object SVDPlusPlus { * * @return a graph with vertex attributes containing the trained model */ - - def run(edges: RDD[Edge[Double]], conf: SVDPlusPlusConf): (Graph[(RealVector, RealVector, Double, Double), Double], Double) = { - - // generate default vertex attribute + def run(edges: RDD[Edge[Double]], conf: SVDPlusPlusConf) + : (Graph[(RealVector, RealVector, Double, Double), Double], Double) = + { + // Generate default vertex attribute def defaultF(rank: Int): (RealVector, RealVector, Double, Double) = { val v1 = new ArrayRealVector(rank) val v2 = new ArrayRealVector(rank) for (i <- 0 until rank) { - v1.setEntry(i, Random.nextDouble) - v2.setEntry(i, Random.nextDouble) + v1.setEntry(i, Random.nextDouble()) + v2.setEntry(i, Random.nextDouble()) } (v1, v2, 0.0, 0.0) } @@ -49,14 +54,18 @@ object SVDPlusPlus { // construct graph var g = Graph.fromEdges(edges, defaultF(conf.rank)).cache() - // calculate initial bias and norm - var t0 = g.mapReduceTriplets(et => - Iterator((et.srcId, (1L, et.attr)), (et.dstId, (1L, et.attr))), (g1: (Long, Double), g2: (Long, Double)) => (g1._1 + g2._1, g1._2 + g2._2)) - g = g.outerJoinVertices(t0) { (vid: VertexID, vd: (RealVector, RealVector, Double, Double), msg: Option[(Long, Double)]) => - (vd._1, vd._2, msg.get._2 / msg.get._1, 1.0 / scala.math.sqrt(msg.get._1)) + // Calculate initial bias and norm + val t0 = g.mapReduceTriplets( + et => Iterator((et.srcId, (1L, et.attr)), (et.dstId, (1L, et.attr))), + (g1: (Long, Double), g2: (Long, Double)) => (g1._1 + g2._1, g1._2 + g2._2)) + + g = g.outerJoinVertices(t0) { + (vid: VertexID, vd: (RealVector, RealVector, Double, Double), msg: Option[(Long, Double)]) => + (vd._1, vd._2, msg.get._2 / msg.get._1, 1.0 / scala.math.sqrt(msg.get._1)) } - def mapTrainF(conf: SVDPlusPlusConf, u: Double)(et: EdgeTriplet[(RealVector, RealVector, Double, Double), Double]) + def mapTrainF(conf: SVDPlusPlusConf, u: Double) + (et: EdgeTriplet[(RealVector, RealVector, Double, Double), Double]) : Iterator[(VertexID, (RealVector, RealVector, Double))] = { val (usr, itm) = (et.srcAttr, et.dstAttr) val (p, q) = (usr._1, itm._1) @@ -64,31 +73,49 @@ object SVDPlusPlus { pred = math.max(pred, conf.minVal) pred = math.min(pred, conf.maxVal) val err = et.attr - pred - val updateP = ((q.mapMultiply(err)).subtract(p.mapMultiply(conf.gamma7))).mapMultiply(conf.gamma2) - val updateQ = ((usr._2.mapMultiply(err)).subtract(q.mapMultiply(conf.gamma7))).mapMultiply(conf.gamma2) - val updateY = ((q.mapMultiply(err * usr._4)).subtract((itm._2).mapMultiply(conf.gamma7))).mapMultiply(conf.gamma2) + val updateP = q.mapMultiply(err) + .subtract(p.mapMultiply(conf.gamma7)) + .mapMultiply(conf.gamma2) + val updateQ = usr._2.mapMultiply(err) + .subtract(q.mapMultiply(conf.gamma7)) + .mapMultiply(conf.gamma2) + val updateY = q.mapMultiply(err * usr._4) + .subtract(itm._2.mapMultiply(conf.gamma7)) + .mapMultiply(conf.gamma2) Iterator((et.srcId, (updateP, updateY, (err - conf.gamma6 * usr._3) * conf.gamma1)), (et.dstId, (updateQ, updateY, (err - conf.gamma6 * itm._3) * conf.gamma1))) } for (i <- 0 until conf.maxIters) { - // phase 1, calculate pu + |N(u)|^(-0.5)*sum(y) for user nodes + // Phase 1, calculate pu + |N(u)|^(-0.5)*sum(y) for user nodes g.cache() - var t1 = g.mapReduceTriplets(et => Iterator((et.srcId, et.dstAttr._2)), (g1: RealVector, g2: RealVector) => g1.add(g2)) - g = g.outerJoinVertices(t1) { (vid: VertexID, vd: (RealVector, RealVector, Double, Double), msg: Option[RealVector]) => - if (msg.isDefined) (vd._1, vd._1.add(msg.get.mapMultiply(vd._4)), vd._3, vd._4) else vd + val t1 = g.mapReduceTriplets( + et => Iterator((et.srcId, et.dstAttr._2)), + (g1: RealVector, g2: RealVector) => g1.add(g2)) + g = g.outerJoinVertices(t1) { + (vid: VertexID, vd: (RealVector, RealVector, Double, Double), msg: Option[RealVector]) => + if (msg.isDefined) (vd._1, vd._1.add(msg.get.mapMultiply(vd._4)), vd._3, vd._4) else vd } - // phase 2, update p for user nodes and q, y for item nodes + + // Phase 2, update p for user nodes and q, y for item nodes g.cache() - val t2 = g.mapReduceTriplets(mapTrainF(conf, u), (g1: (RealVector, RealVector, Double), g2: (RealVector, RealVector, Double)) => - (g1._1.add(g2._1), g1._2.add(g2._2), g1._3 + g2._3)) - g = g.outerJoinVertices(t2) { (vid: VertexID, vd: (RealVector, RealVector, Double, Double), msg: Option[(RealVector, RealVector, Double)]) => - (vd._1.add(msg.get._1), vd._2.add(msg.get._2), vd._3 + msg.get._3, vd._4) + val t2 = g.mapReduceTriplets( + mapTrainF(conf, u), + (g1: (RealVector, RealVector, Double), g2: (RealVector, RealVector, Double)) => + (g1._1.add(g2._1), g1._2.add(g2._2), g1._3 + g2._3)) + g = g.outerJoinVertices(t2) { + (vid: VertexID, + vd: (RealVector, RealVector, Double, Double), + msg: Option[(RealVector, RealVector, Double)]) => + (vd._1.add(msg.get._1), vd._2.add(msg.get._2), vd._3 + msg.get._3, vd._4) } } // calculate error on training set - def mapTestF(conf: SVDPlusPlusConf, u: Double)(et: EdgeTriplet[(RealVector, RealVector, Double, Double), Double]): Iterator[(VertexID, Double)] = { + def mapTestF(conf: SVDPlusPlusConf, u: Double) + (et: EdgeTriplet[(RealVector, RealVector, Double, Double), Double]) + : Iterator[(VertexID, Double)] = + { val (usr, itm) = (et.srcAttr, et.dstAttr) val (p, q) = (usr._1, itm._1) var pred = u + usr._3 + itm._3 + q.dotProduct(usr._2) @@ -99,9 +126,11 @@ object SVDPlusPlus { } g.cache() val t3 = g.mapReduceTriplets(mapTestF(conf, u), (g1: Double, g2: Double) => g1 + g2) - g = g.outerJoinVertices(t3) { (vid: VertexID, vd: (RealVector, RealVector, Double, Double), msg: Option[Double]) => - if (msg.isDefined) (vd._1, vd._2, vd._3, msg.get) else vd + g = g.outerJoinVertices(t3) { + (vid: VertexID, vd: (RealVector, RealVector, Double, Double), msg: Option[Double]) => + if (msg.isDefined) (vd._1, vd._2, vd._3, msg.get) else vd } + (g, u) } } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/StronglyConnectedComponents.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/StronglyConnectedComponents.scala index 43c4b9cf2d..11847509da 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/StronglyConnectedComponents.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/StronglyConnectedComponents.scala @@ -4,6 +4,7 @@ import scala.reflect.ClassTag import org.apache.spark.graphx._ +/** Strongly connected components algorithm implementation. */ object StronglyConnectedComponents { /** diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/TriangleCount.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/TriangleCount.scala index 58da9e3aed..f87eab9505 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/TriangleCount.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/TriangleCount.scala @@ -4,27 +4,26 @@ import scala.reflect.ClassTag import org.apache.spark.graphx._ - +/** + * Compute the number of triangles passing through each vertex. + * + * The algorithm is relatively straightforward and can be computed in three steps: + * + * 1) Compute the set of neighbors for each vertex + * 2) For each edge compute the intersection of the sets and send the + * count to both vertices. + * 3) Compute the sum at each vertex and divide by two since each + * triangle is counted twice. + * + * Note that the input graph should have its edges in canonical direction + * (i.e. the `sourceId` less than `destId`). Also the graph must have been partitioned + * using [[org.apache.spark.graphx.Graph#partitionBy]]. + */ object TriangleCount { - /** - * Compute the number of triangles passing through each vertex. - * - * The algorithm is relatively straightforward and can be computed in three steps: - * - * 1) Compute the set of neighbors for each vertex - * 2) For each edge compute the intersection of the sets and send the - * count to both vertices. - * 3) Compute the sum at each vertex and divide by two since each - * triangle is counted twice. - * - * - * @param graph a graph with `sourceId` less than `destId`. The graph must have been partitioned - * using [[org.apache.spark.graphx.Graph#partitionBy]], and its edges must be in canonical - * orientation (srcId < dstId). - */ + def run[VD: ClassTag, ED: ClassTag](graph: Graph[VD,ED]): Graph[Int, ED] = { // Remove redundant edges - val g = graph.groupEdges((a, b) => a).cache + val g = graph.groupEdges((a, b) => a).cache() // Construct set representations of the neighborhoods val nbrSets: VertexRDD[VertexSet] = @@ -56,8 +55,10 @@ object TriangleCount { val iter = smallSet.iterator var counter: Int = 0 while (iter.hasNext) { - val vid = iter.next - if (vid != et.srcId && vid != et.dstId && largeSet.contains(vid)) { counter += 1 } + val vid = iter.next() + if (vid != et.srcId && vid != et.dstId && largeSet.contains(vid)) { + counter += 1 + } } Iterator((et.srcId, counter), (et.dstId, counter)) } @@ -71,7 +72,5 @@ object TriangleCount { assert((dblCount & 1) == 0) dblCount / 2 } - } // end of TriangleCount - } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/package.scala b/graphx/src/main/scala/org/apache/spark/graphx/package.scala index e70d2fd09f..60dfc1dc37 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/package.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/package.scala @@ -2,6 +2,7 @@ package org.apache.spark import org.apache.spark.util.collection.OpenHashSet +/** GraphX is a graph processing framework built on top of Spark. */ package object graphx { /** * A 64-bit vertex identifier that uniquely identifies a vertex within a graph. It does not need @@ -9,11 +10,9 @@ package object graphx { */ type VertexID = Long + /** Integer identifer of a graph partition. */ // TODO: Consider using Char. type PartitionID = Int private[graphx] type VertexSet = OpenHashSet[VertexID] - - /** Returns the default null-like value for a data type T. */ - private[graphx] def nullValue[T] = null.asInstanceOf[T] } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala index ec8d534333..1c5b234d74 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala @@ -10,8 +10,11 @@ import org.objectweb.asm.{ClassReader, ClassVisitor, MethodVisitor} import org.objectweb.asm.Opcodes._ - -private[spark] object BytecodeUtils { +/** + * Includes an utility function to test whether a function accesses a specific attribute + * of an object. + */ +private[graphx] object BytecodeUtils { /** * Test whether the given closure invokes the specified method in the specified class. diff --git a/graphx/src/main/scala/org/apache/spark/graphx/util/collection/PrimitiveKeyOpenHashMap.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/collection/PrimitiveKeyOpenHashMap.scala index 1088944cd3..7b02e2ed1a 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/util/collection/PrimitiveKeyOpenHashMap.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/util/collection/PrimitiveKeyOpenHashMap.scala @@ -28,7 +28,7 @@ import scala.reflect._ * * Under the hood, it uses our OpenHashSet implementation. */ -private[spark] +private[graphx] class PrimitiveKeyOpenHashMap[@specialized(Long, Int) K: ClassTag, @specialized(Long, Int, Double) V: ClassTag]( val keySet: OpenHashSet[K], var _values: Array[V]) -- cgit v1.2.3 From 87f335db78221fc250bd64f39a334293db490379 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 13 Jan 2014 18:30:26 -0800 Subject: Made more things private. --- graphx/src/main/scala/org/apache/spark/graphx/Edge.scala | 8 ++++---- graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala | 4 ---- .../scala/org/apache/spark/graphx/impl/EdgePartition.scala | 1 + .../org/apache/spark/graphx/impl/EdgePartitionBuilder.scala | 1 + .../main/scala/org/apache/spark/graphx/impl/GraphImpl.scala | 2 +- .../org/apache/spark/graphx/impl/MessageToPartition.scala | 5 +++++ .../org/apache/spark/graphx/impl/ReplicatedVertexView.scala | 3 ++- .../scala/org/apache/spark/graphx/impl/RoutingTable.scala | 1 + .../main/scala/org/apache/spark/graphx/impl/Serializers.scala | 11 ++++++++++- .../src/main/scala/org/apache/spark/graphx/impl/package.scala | 2 +- 10 files changed, 26 insertions(+), 12 deletions(-) (limited to 'graphx') diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala b/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala index 6c396c3dbe..85463052bc 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala @@ -1,18 +1,18 @@ package org.apache.spark.graphx - /** * A single directed edge consisting of a source id, target id, * and the data associated with the edge. * * @tparam ED type of the edge attribute + * + * @param srcId The vertex id of the source vertex + * @param dstId The vertex id of the target vertex + * @param attr The attribute associated with the edge */ case class Edge[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED] ( - /** The vertex id of the source vertex */ var srcId: VertexID = 0, - /** The vertex id of the target vertex. */ var dstId: VertexID = 0, - /** The attribute associated with the edge. */ var attr: ED = null.asInstanceOf[ED]) extends Serializable { 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 66d5180020..447ef555ca 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala @@ -209,12 +209,8 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) { * This function iterates until there are no remaining messages, or * for `maxIterations` iterations. * - * @tparam VD the vertex data type - * @tparam ED the edge data type * @tparam A the Pregel message type * - * @param graph the input graph. - * * @param initialMsg the message each vertex will receive at the on * the first iteration * 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 d4d71627e1..ee95ead3ad 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala @@ -15,6 +15,7 @@ import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap * @param index a clustered index on source vertex id * @tparam ED the edge attribute type. */ +private[graphx] class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED: ClassTag]( val srcIds: Array[VertexID], val dstIds: Array[VertexID], 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 fbc29409b5..9d072f9335 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala @@ -7,6 +7,7 @@ import org.apache.spark.graphx._ import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap import org.apache.spark.util.collection.PrimitiveVector +private[graphx] class EdgePartitionBuilder[@specialized(Long, Int, Double) ED: ClassTag](size: Int = 64) { var edges = new PrimitiveVector[Edge[ED]](size) 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 97ca642f9b..348490c186 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala @@ -15,7 +15,7 @@ import org.apache.spark.util.ClosureCleaner /** - * A Graph RDD that supports computation on graphs. + * A graph that supports computation on graphs. * * Graphs are represented using two classes of data: vertex-partitioned and * edge-partitioned. `vertices` contains vertex attributes, which are vertex-partitioned. `edges` 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 ad5daf8f6a..05508ff716 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala @@ -7,6 +7,7 @@ import org.apache.spark.graphx.{PartitionID, VertexID} import org.apache.spark.rdd.{ShuffledRDD, RDD} +private[graphx] class VertexBroadcastMsg[@specialized(Int, Long, Double, Boolean) T]( @transient var partition: PartitionID, var vid: VertexID, @@ -26,6 +27,7 @@ class VertexBroadcastMsg[@specialized(Int, Long, Double, Boolean) T]( * @param partition index of the target partition. * @param data value to send */ +private[graphx] class MessageToPartition[@specialized(Int, Long, Double, Char, Boolean/*, AnyRef*/) T]( @transient var partition: PartitionID, var data: T) @@ -39,6 +41,7 @@ class MessageToPartition[@specialized(Int, Long, Double, Char, Boolean/*, AnyRef } +private[graphx] class VertexBroadcastMsgRDDFunctions[T: ClassTag](self: RDD[VertexBroadcastMsg[T]]) { def partitionBy(partitioner: Partitioner): RDD[VertexBroadcastMsg[T]] = { val rdd = new ShuffledRDD[PartitionID, (VertexID, T), VertexBroadcastMsg[T]](self, partitioner) @@ -56,6 +59,7 @@ class VertexBroadcastMsgRDDFunctions[T: ClassTag](self: RDD[VertexBroadcastMsg[T } +private[graphx] class MsgRDDFunctions[T: ClassTag](self: RDD[MessageToPartition[T]]) { /** @@ -68,6 +72,7 @@ class MsgRDDFunctions[T: ClassTag](self: RDD[MessageToPartition[T]]) { } +private[graphx] object MsgRDDFunctions { implicit def rdd2PartitionRDDFunctions[T: ClassTag](rdd: RDD[MessageToPartition[T]]) = { new MsgRDDFunctions(rdd) 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 0e2f5a9dd9..4ebe0b0267 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala @@ -138,7 +138,7 @@ class ReplicatedVertexView[VD: ClassTag]( } } -object ReplicatedVertexView { +private object ReplicatedVertexView { protected def buildBuffer[VD: ClassTag]( pid2vidIter: Iterator[Array[Array[VertexID]]], vertexPartIter: Iterator[VertexPartition[VD]]) = { @@ -187,6 +187,7 @@ object ReplicatedVertexView { } } +private[graphx] class VertexAttributeBlock[VD: ClassTag](val vids: Array[VertexID], val attrs: Array[VD]) extends Serializable { def iterator: Iterator[(VertexID, VD)] = diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTable.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTable.scala index 3bd8b24133..f342fd7437 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTable.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTable.scala @@ -12,6 +12,7 @@ import org.apache.spark.util.collection.PrimitiveVector * may be used multiple times in ReplicatedVertexView -- once to ship the vertex attributes and * (possibly) once to ship the active-set information. */ +private[impl] class RoutingTable(edges: EdgeRDD[_], vertices: VertexRDD[_]) { val bothAttrs: RDD[Array[Array[VertexID]]] = createPid2Vid(true, true) 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 1c3c87f08d..cbd6318f33 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala @@ -7,6 +7,7 @@ import org.apache.spark.SparkConf import org.apache.spark.graphx._ import org.apache.spark.serializer._ +private[graphx] class VertexIDMsgSerializer(conf: SparkConf) extends Serializer { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { @@ -27,6 +28,7 @@ class VertexIDMsgSerializer(conf: SparkConf) extends Serializer { } /** A special shuffle serializer for VertexBroadcastMessage[Int]. */ +private[graphx] class IntVertexBroadcastMsgSerializer(conf: SparkConf) extends Serializer { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { @@ -50,6 +52,7 @@ class IntVertexBroadcastMsgSerializer(conf: SparkConf) extends Serializer { } /** A special shuffle serializer for VertexBroadcastMessage[Long]. */ +private[graphx] class LongVertexBroadcastMsgSerializer(conf: SparkConf) extends Serializer { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { @@ -73,6 +76,7 @@ class LongVertexBroadcastMsgSerializer(conf: SparkConf) extends Serializer { } /** A special shuffle serializer for VertexBroadcastMessage[Double]. */ +private[graphx] class DoubleVertexBroadcastMsgSerializer(conf: SparkConf) extends Serializer { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { @@ -96,6 +100,7 @@ class DoubleVertexBroadcastMsgSerializer(conf: SparkConf) extends Serializer { } /** A special shuffle serializer for AggregationMessage[Int]. */ +private[graphx] class IntAggMsgSerializer(conf: SparkConf) extends Serializer { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { @@ -119,6 +124,7 @@ class IntAggMsgSerializer(conf: SparkConf) extends Serializer { } /** A special shuffle serializer for AggregationMessage[Long]. */ +private[graphx] class LongAggMsgSerializer(conf: SparkConf) extends Serializer { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { @@ -142,6 +148,7 @@ class LongAggMsgSerializer(conf: SparkConf) extends Serializer { } /** A special shuffle serializer for AggregationMessage[Double]. */ +private[graphx] class DoubleAggMsgSerializer(conf: SparkConf) extends Serializer { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { @@ -168,6 +175,7 @@ class DoubleAggMsgSerializer(conf: SparkConf) extends Serializer { // Helper classes to shorten the implementation of those special serializers. //////////////////////////////////////////////////////////////////////////////// +private[graphx] abstract class ShuffleSerializationStream(s: OutputStream) extends SerializationStream { // The implementation should override this one. def writeObject[T](t: T): SerializationStream @@ -281,6 +289,7 @@ abstract class ShuffleSerializationStream(s: OutputStream) extends Serialization override def close(): Unit = s.close() } +private[graphx] abstract class ShuffleDeserializationStream(s: InputStream) extends DeserializationStream { // The implementation should override this one. def readObject[T](): T @@ -371,7 +380,7 @@ abstract class ShuffleDeserializationStream(s: InputStream) extends Deserializat override def close(): Unit = s.close() } -sealed trait ShuffleSerializerInstance extends SerializerInstance { +private[graphx] sealed trait ShuffleSerializerInstance extends SerializerInstance { override def serialize[T](t: T): ByteBuffer = throw new UnsupportedOperationException diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/package.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/package.scala index a6bbf63888..cfc3281b64 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/package.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/package.scala @@ -3,5 +3,5 @@ package org.apache.spark.graphx import org.apache.spark.util.collection.OpenHashSet package object impl { - type VertexIdToIndexMap = OpenHashSet[VertexID] + private[graphx] type VertexIdToIndexMap = OpenHashSet[VertexID] } -- cgit v1.2.3 From ae06d2c22ffb8af3c27c29bc55aadfb73b56e9ff Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 13 Jan 2014 18:31:49 -0800 Subject: Updated GraphGenerator. --- .../apache/spark/graphx/util/GraphGenerators.scala | 60 +++++++++++----------- 1 file changed, 30 insertions(+), 30 deletions(-) (limited to 'graphx') diff --git a/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala index e0fd9b972c..dbea233c34 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala @@ -15,6 +15,7 @@ import org.apache.spark.graphx.Graph import org.apache.spark.graphx.Edge import org.apache.spark.graphx.impl.GraphImpl +/** A collection of graph generating functions. */ object GraphGenerators { val RMATa = 0.45 @@ -24,6 +25,9 @@ object GraphGenerators { // Right now it just generates a bunch of edges where // the edge data is the weight (default 1) + /** + * Generate a graph whose vertex out degree is log normal. + */ def logNormalGraph(sc: SparkContext, numVertices: Int): Graph[Int, Int] = { // based on Pregel settings val mu = 4 @@ -32,13 +36,12 @@ object GraphGenerators { val vertices: RDD[(VertexID, Int)] = sc.parallelize(0 until numVertices).map{ src => (src, sampleLogNormal(mu, sigma, numVertices)) } - val edges = vertices.flatMap{ - v => generateRandomEdges(v._1.toInt, v._2, numVertices) + val edges = vertices.flatMap { v => + generateRandomEdges(v._1.toInt, v._2, numVertices) } Graph(vertices, edges, 0) } - def generateRandomEdges(src: Int, numEdges: Int, maxVertexID: Int): Array[Edge[Int]] = { val rand = new Random() var dsts: Set[Int] = Set() @@ -48,10 +51,9 @@ object GraphGenerators { dsts += nextDst } } - dsts.map {dst => Edge[Int](src, dst, 1) }.toArray + dsts.map(dst => Edge[Int](src, dst, 1)).toArray } - /** * Randomly samples from a log normal distribution whose corresponding normal distribution has the * the given mean and standard deviation. It uses the formula `X = exp(m+s*Z)` where `m`, `s` are @@ -60,9 +62,9 @@ object GraphGenerators { * * @param mu the mean of the normal distribution * @param sigma the standard deviation of the normal distribution - * @param macVal exclusive upper bound on the value of the sample + * @param maxVal exclusive upper bound on the value of the sample */ - def sampleLogNormal(mu: Double, sigma: Double, maxVal: Int): Int = { + private def sampleLogNormal(mu: Double, sigma: Double, maxVal: Int): Int = { val rand = new Random() val m = math.exp(mu+(sigma*sigma)/2.0) val s = math.sqrt((math.exp(sigma*sigma) - 1) * math.exp(2*mu + sigma*sigma)) @@ -76,27 +78,29 @@ object GraphGenerators { math.round(X.toFloat) } - - + /** + * A random graph generator using the R-MAT model, proposed in + * "R-MAT: A Recursive Model for Graph Mining" by Chakrabarti et al. + * + * See [[http://www.cs.cmu.edu/~christos/PUBLICATIONS/siam04.pdf]]. + */ def rmatGraph(sc: SparkContext, requestedNumVertices: Int, numEdges: Int): Graph[Int, Int] = { // let N = requestedNumVertices // the number of vertices is 2^n where n=ceil(log2[N]) // This ensures that the 4 quadrants are the same size at all recursion levels - val numVertices = math.round(math.pow(2.0, math.ceil(math.log(requestedNumVertices)/math.log(2.0)))).toInt + val numVertices = math.round( + math.pow(2.0, math.ceil(math.log(requestedNumVertices) / math.log(2.0)))).toInt var edges: Set[Edge[Int]] = Set() while (edges.size < numEdges) { if (edges.size % 100 == 0) { println(edges.size + " edges") } edges += addEdge(numVertices) - } - val graph = outDegreeFromEdges(sc.parallelize(edges.toList)) - graph - + outDegreeFromEdges(sc.parallelize(edges.toList)) } - def outDegreeFromEdges[ED: ClassTag](edges: RDD[Edge[ED]]): Graph[Int, ED] = { + private def outDegreeFromEdges[ED: ClassTag](edges: RDD[Edge[ED]]): Graph[Int, ED] = { val vertices = edges.flatMap { edge => List((edge.srcId, 1)) } .reduceByKey(_ + _) .map{ case (vid, degree) => (vid, degree) } @@ -107,7 +111,7 @@ object GraphGenerators { * @param numVertices Specifies the total number of vertices in the graph (used to get * the dimensions of the adjacency matrix */ - def addEdge(numVertices: Int): Edge[Int] = { + private def addEdge(numVertices: Int): Edge[Int] = { //val (src, dst) = chooseCell(numVertices/2.0, numVertices/2.0, numVertices/2.0) val v = math.round(numVertices.toFloat/2.0).toInt @@ -115,7 +119,6 @@ object GraphGenerators { Edge[Int](src, dst, 1) } - /** * This method recursively subdivides the the adjacency matrix into quadrants * until it picks a single cell. The naming conventions in this paper match @@ -149,10 +152,10 @@ object GraphGenerators { * }}} */ @tailrec - def chooseCell(x: Int, y: Int, t: Int): (Int, Int) = { - if (t <= 1) - (x,y) - else { + private def chooseCell(x: Int, y: Int, t: Int): (Int, Int) = { + if (t <= 1) { + (x, y) + } else { val newT = math.round(t.toFloat/2.0).toInt pickQuadrant(RMATa, RMATb, RMATc, RMATd) match { case 0 => chooseCell(x, y, newT) @@ -164,22 +167,21 @@ object GraphGenerators { } // TODO(crankshaw) turn result into an enum (or case class for pattern matching} - def pickQuadrant(a: Double, b: Double, c: Double, d: Double): Int = { - if (a+b+c+d != 1.0) { - throw new IllegalArgumentException("R-MAT probability parameters sum to " + (a+b+c+d) + ", should sum to 1.0") + private def pickQuadrant(a: Double, b: Double, c: Double, d: Double): Int = { + if (a + b + c + d != 1.0) { + throw new IllegalArgumentException( + "R-MAT probability parameters sum to " + (a+b+c+d) + ", should sum to 1.0") } val rand = new Random() val result = rand.nextDouble() result match { case x if x < a => 0 // 0 corresponds to quadrant a - case x if (x >= a && x < a+b) => 1 // 1 corresponds to b - case x if (x >= a+b && x < a+b+c) => 2 // 2 corresponds to c + case x if (x >= a && x < a + b) => 1 // 1 corresponds to b + case x if (x >= a + b && x < a + b + c) => 2 // 2 corresponds to c case _ => 3 // 3 corresponds to d } } - - /** * Create `rows` by `cols` grid graph with each vertex connected to its * row+1 and col+1 neighbors. Vertex ids are assigned in row major @@ -220,6 +222,4 @@ object GraphGenerators { Graph.fromEdgeTuples(edges, 1) } // end of starGraph - - } // end of Graph Generators -- cgit v1.2.3 From 1dce9ce446dd248755cd65b7a6a0729a4dca2d62 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 13 Jan 2014 18:32:04 -0800 Subject: Moved PartitionStrategy's into an object. --- .../apache/spark/graphx/PartitionStrategy.scala | 158 +++++++++++---------- .../org/apache/spark/graphx/lib/Analytics.scala | 2 + .../scala/org/apache/spark/graphx/GraphSuite.scala | 1 + .../spark/graphx/lib/TriangleCountSuite.scala | 5 +- 4 files changed, 85 insertions(+), 81 deletions(-) (limited to 'graphx') 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 b9ccd8765e..6d2990a3f6 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala @@ -4,96 +4,100 @@ package org.apache.spark.graphx * Represents the way edges are assigned to edge partitions based on their source and destination * vertex IDs. */ -sealed trait PartitionStrategy extends Serializable { +trait PartitionStrategy extends Serializable { /** Returns the partition number for a given edge. */ def getPartition(src: VertexID, dst: VertexID, numParts: PartitionID): PartitionID } - /** - * Assigns edges to partitions using a 2D partitioning of the sparse edge adjacency matrix, - * guaranteeing a `2 * sqrt(numParts)` bound on vertex replication. - * - * Suppose we have a graph with 11 vertices that we want to partition - * over 9 machines. We can use the following sparse matrix representation: - * - *
- *       __________________________________
- *  v0   | P0 *     | P1       | P2    *  |
- *  v1   |  ****    |  *       |          |
- *  v2   |  ******* |      **  |  ****    |
- *  v3   |  *****   |  *  *    |       *  |
- *       ----------------------------------
- *  v4   | P3 *     | P4 ***   | P5 **  * |
- *  v5   |  *  *    |  *       |          |
- *  v6   |       *  |      **  |  ****    |
- *  v7   |  * * *   |  *  *    |       *  |
- *       ----------------------------------
- *  v8   | P6   *   | P7    *  | P8  *   *|
- *  v9   |     *    |  *    *  |          |
- *  v10  |       *  |      **  |  *  *    |
- *  v11  | * <-E    |  ***     |       ** |
- *       ----------------------------------
- * 
- * - * The edge denoted by `E` connects `v11` with `v1` and is assigned to processor `P6`. To get the - * processor number we divide the matrix into `sqrt(numParts)` by `sqrt(numParts)` blocks. Notice - * that edges adjacent to `v11` can only be in the first column of blocks `(P0, P3, P6)` or the last - * row of blocks `(P6, P7, P8)`. As a consequence we can guarantee that `v11` will need to be - * replicated to at most `2 * sqrt(numParts)` machines. - * - * Notice that `P0` has many edges and as a consequence this partitioning would lead to poor work - * balance. To improve balance we first multiply each vertex id by a large prime to shuffle the - * vertex locations. - * - * One of the limitations of this approach is that the number of machines must either be a perfect - * square. We partially address this limitation by computing the machine assignment to the next - * largest perfect square and then mapping back down to the actual number of machines. - * Unfortunately, this can also lead to work imbalance and so it is suggested that a perfect square - * is used. + * Collection of built-in [[PartitionStrategy]] implementations. */ -case object EdgePartition2D extends PartitionStrategy { - override def getPartition(src: VertexID, dst: VertexID, numParts: PartitionID): PartitionID = { - val ceilSqrtNumParts: PartitionID = math.ceil(math.sqrt(numParts)).toInt - val mixingPrime: VertexID = 1125899906842597L - val col: PartitionID = ((math.abs(src) * mixingPrime) % ceilSqrtNumParts).toInt - val row: PartitionID = ((math.abs(dst) * mixingPrime) % ceilSqrtNumParts).toInt - (col * ceilSqrtNumParts + row) % numParts +object PartitionStrategy { + /** + * Assigns edges to partitions using a 2D partitioning of the sparse edge adjacency matrix, + * guaranteeing a `2 * sqrt(numParts)` bound on vertex replication. + * + * Suppose we have a graph with 11 vertices that we want to partition + * over 9 machines. We can use the following sparse matrix representation: + * + *
+   *       __________________________________
+   *  v0   | P0 *     | P1       | P2    *  |
+   *  v1   |  ****    |  *       |          |
+   *  v2   |  ******* |      **  |  ****    |
+   *  v3   |  *****   |  *  *    |       *  |
+   *       ----------------------------------
+   *  v4   | P3 *     | P4 ***   | P5 **  * |
+   *  v5   |  *  *    |  *       |          |
+   *  v6   |       *  |      **  |  ****    |
+   *  v7   |  * * *   |  *  *    |       *  |
+   *       ----------------------------------
+   *  v8   | P6   *   | P7    *  | P8  *   *|
+   *  v9   |     *    |  *    *  |          |
+   *  v10  |       *  |      **  |  *  *    |
+   *  v11  | * <-E    |  ***     |       ** |
+   *       ----------------------------------
+   * 
+ * + * The edge denoted by `E` connects `v11` with `v1` and is assigned to processor `P6`. To get the + * processor number we divide the matrix into `sqrt(numParts)` by `sqrt(numParts)` blocks. Notice + * that edges adjacent to `v11` can only be in the first column of blocks `(P0, P3, P6)` or the last + * row of blocks `(P6, P7, P8)`. As a consequence we can guarantee that `v11` will need to be + * replicated to at most `2 * sqrt(numParts)` machines. + * + * Notice that `P0` has many edges and as a consequence this partitioning would lead to poor work + * balance. To improve balance we first multiply each vertex id by a large prime to shuffle the + * vertex locations. + * + * One of the limitations of this approach is that the number of machines must either be a perfect + * square. We partially address this limitation by computing the machine assignment to the next + * largest perfect square and then mapping back down to the actual number of machines. + * Unfortunately, this can also lead to work imbalance and so it is suggested that a perfect square + * is used. + */ + case object EdgePartition2D extends PartitionStrategy { + override def getPartition(src: VertexID, dst: VertexID, numParts: PartitionID): PartitionID = { + val ceilSqrtNumParts: PartitionID = math.ceil(math.sqrt(numParts)).toInt + val mixingPrime: VertexID = 1125899906842597L + val col: PartitionID = ((math.abs(src) * mixingPrime) % ceilSqrtNumParts).toInt + val row: PartitionID = ((math.abs(dst) * mixingPrime) % ceilSqrtNumParts).toInt + (col * ceilSqrtNumParts + row) % numParts + } } -} -/** - * Assigns edges to partitions using only the source vertex ID, colocating edges with the same - * source. - */ -case object EdgePartition1D extends PartitionStrategy { - override def getPartition(src: VertexID, dst: VertexID, numParts: PartitionID): PartitionID = { - val mixingPrime: VertexID = 1125899906842597L - (math.abs(src) * mixingPrime).toInt % numParts + /** + * Assigns edges to partitions using only the source vertex ID, colocating edges with the same + * source. + */ + case object EdgePartition1D extends PartitionStrategy { + override def getPartition(src: VertexID, dst: VertexID, numParts: PartitionID): PartitionID = { + val mixingPrime: VertexID = 1125899906842597L + (math.abs(src) * mixingPrime).toInt % numParts + } } -} -/** - * Assigns edges to partitions by hashing the source and destination vertex IDs, resulting in a - * random vertex cut that colocates all same-direction edges between two vertices. - */ -case object RandomVertexCut extends PartitionStrategy { - override def getPartition(src: VertexID, dst: VertexID, numParts: PartitionID): PartitionID = { - math.abs((src, dst).hashCode()) % numParts + /** + * Assigns edges to partitions by hashing the source and destination vertex IDs, resulting in a + * random vertex cut that colocates all same-direction edges between two vertices. + */ + case object RandomVertexCut extends PartitionStrategy { + override def getPartition(src: VertexID, dst: VertexID, numParts: PartitionID): PartitionID = { + math.abs((src, dst).hashCode()) % numParts + } } -} -/** - * Assigns edges to partitions by hashing the source and destination vertex IDs in a canonical - * direction, resulting in a random vertex cut that colocates all edges between two vertices, - * regardless of direction. - */ -case object CanonicalRandomVertexCut extends PartitionStrategy { - override def getPartition(src: VertexID, dst: VertexID, numParts: PartitionID): PartitionID = { - val lower = math.min(src, dst) - val higher = math.max(src, dst) - math.abs((lower, higher).hashCode()) % numParts + /** + * Assigns edges to partitions by hashing the source and destination vertex IDs in a canonical + * direction, resulting in a random vertex cut that colocates all edges between two vertices, + * regardless of direction. + */ + case object CanonicalRandomVertexCut extends PartitionStrategy { + override def getPartition(src: VertexID, dst: VertexID, numParts: PartitionID): PartitionID = { + val lower = math.min(src, dst) + val higher = math.max(src, dst) + math.abs((lower, higher).hashCode()) % numParts + } } } 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 d5e1de1ce0..e0aff5644e 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/Analytics.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/Analytics.scala @@ -2,6 +2,7 @@ package org.apache.spark.graphx.lib import org.apache.spark._ import org.apache.spark.graphx._ +import org.apache.spark.graphx.PartitionStrategy._ /** * Driver program for running graph algorithms. @@ -20,6 +21,7 @@ object Analytics extends Logging { } def pickPartitioner(v: String): PartitionStrategy = { + // TODO: Use reflection rather than listing all the partitioning strategies here. v match { case "RandomVertexCut" => RandomVertexCut case "EdgePartition1D" => EdgePartition1D 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 c32a6cbb81..9587f04c3e 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala @@ -4,6 +4,7 @@ import org.scalatest.FunSuite import org.apache.spark.SparkContext import org.apache.spark.graphx.Graph._ +import org.apache.spark.graphx.PartitionStrategy._ import org.apache.spark.rdd._ class GraphSuite extends FunSuite with LocalSparkContext { diff --git a/graphx/src/test/scala/org/apache/spark/graphx/lib/TriangleCountSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/lib/TriangleCountSuite.scala index a286b7d03b..3452ce9764 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/lib/TriangleCountSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/lib/TriangleCountSuite.scala @@ -2,11 +2,8 @@ package org.apache.spark.graphx.lib import org.scalatest.FunSuite -import org.apache.spark.SparkContext -import org.apache.spark.SparkContext._ import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.GraphGenerators -import org.apache.spark.rdd._ +import org.apache.spark.graphx.PartitionStrategy.RandomVertexCut class TriangleCountSuite extends FunSuite with LocalSparkContext { -- cgit v1.2.3 From 8e5c7324303ee9a9a61ad35e94ada5638ca0cf70 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 13 Jan 2014 18:45:20 -0800 Subject: Moved SVDPlusPlusConf into SVDPlusPlus object itself. --- .../org/apache/spark/graphx/lib/SVDPlusPlus.scala | 30 ++++++++++++---------- .../apache/spark/graphx/lib/SVDPlusPlusSuite.scala | 2 +- 2 files changed, 17 insertions(+), 15 deletions(-) (limited to 'graphx') diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala index fa6b1db29b..ba6517e012 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala @@ -5,19 +5,21 @@ import org.apache.commons.math.linear._ import org.apache.spark.rdd._ import org.apache.spark.graphx._ -/** Configuration parameters for SVDPlusPlus. */ -class SVDPlusPlusConf( - var rank: Int, - var maxIters: Int, - var minVal: Double, - var maxVal: Double, - var gamma1: Double, - var gamma2: Double, - var gamma6: Double, - var gamma7: Double) extends Serializable - /** Implementation of SVD++ algorithm. */ object SVDPlusPlus { + + /** Configuration parameters for SVDPlusPlus. */ + class Conf( + var rank: Int, + var maxIters: Int, + var minVal: Double, + var maxVal: Double, + var gamma1: Double, + var gamma2: Double, + var gamma6: Double, + var gamma7: Double) + extends Serializable + /** * Implement SVD++ based on "Factorization Meets the Neighborhood: * a Multifaceted Collaborative Filtering Model", @@ -32,7 +34,7 @@ object SVDPlusPlus { * * @return a graph with vertex attributes containing the trained model */ - def run(edges: RDD[Edge[Double]], conf: SVDPlusPlusConf) + def run(edges: RDD[Edge[Double]], conf: Conf) : (Graph[(RealVector, RealVector, Double, Double), Double], Double) = { // Generate default vertex attribute @@ -64,7 +66,7 @@ object SVDPlusPlus { (vd._1, vd._2, msg.get._2 / msg.get._1, 1.0 / scala.math.sqrt(msg.get._1)) } - def mapTrainF(conf: SVDPlusPlusConf, u: Double) + def mapTrainF(conf: Conf, u: Double) (et: EdgeTriplet[(RealVector, RealVector, Double, Double), Double]) : Iterator[(VertexID, (RealVector, RealVector, Double))] = { val (usr, itm) = (et.srcAttr, et.dstAttr) @@ -112,7 +114,7 @@ object SVDPlusPlus { } // calculate error on training set - def mapTestF(conf: SVDPlusPlusConf, u: Double) + def mapTestF(conf: Conf, u: Double) (et: EdgeTriplet[(RealVector, RealVector, Double, Double), Double]) : Iterator[(VertexID, Double)] = { diff --git a/graphx/src/test/scala/org/apache/spark/graphx/lib/SVDPlusPlusSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/lib/SVDPlusPlusSuite.scala index a4a1cdab18..e173c652a5 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/lib/SVDPlusPlusSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/lib/SVDPlusPlusSuite.scala @@ -18,7 +18,7 @@ class SVDPlusPlusSuite extends FunSuite with LocalSparkContext { val fields = line.split(",") Edge(fields(0).toLong * 2, fields(1).toLong * 2 + 1, fields(2).toDouble) } - val conf = new SVDPlusPlusConf(10, 2, 0.0, 5.0, 0.007, 0.007, 0.005, 0.015) // 2 iterations + val conf = new SVDPlusPlus.Conf(10, 2, 0.0, 5.0, 0.007, 0.007, 0.005, 0.015) // 2 iterations var (graph, u) = SVDPlusPlus.run(edges, conf) graph.cache() val err = graph.vertices.collect.map{ case (vid, vd) => -- cgit v1.2.3 From 9317286b72ec8bb065b0422c344c267cc49189e3 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 13 Jan 2014 18:45:35 -0800 Subject: More cleanup. --- graphx/src/main/scala/org/apache/spark/graphx/Edge.scala | 2 +- .../src/main/scala/org/apache/spark/graphx/EdgeDirection.scala | 3 +++ .../org/apache/spark/graphx/lib/ConnectedComponents.scala | 4 +--- .../main/scala/org/apache/spark/graphx/lib/TriangleCount.scala | 10 +++++----- 4 files changed, 10 insertions(+), 9 deletions(-) (limited to 'graphx') diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala b/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala index 85463052bc..21be58e740 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala @@ -37,7 +37,7 @@ case class Edge[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED] if (vid == srcId) EdgeDirection.Out else { assert(vid == dstId); EdgeDirection.In } } -object Edge { +private[graphx] object Edge { def lexicographicOrdering[ED] = new Ordering[Edge[ED]] { override def compare(a: Edge[ED], b: Edge[ED]): Int = (if (a.srcId != b.srcId) a.srcId - b.srcId else a.dstId - b.dstId).toInt diff --git a/graphx/src/main/scala/org/apache/spark/graphx/EdgeDirection.scala b/graphx/src/main/scala/org/apache/spark/graphx/EdgeDirection.scala index 5b58a61bbd..f265764006 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/EdgeDirection.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeDirection.scala @@ -26,6 +26,9 @@ class EdgeDirection private (private val name: String) extends Serializable { } +/** + * A set of [[EdgeDirection]]s. + */ object EdgeDirection { /** Edges arriving at a vertex. */ final val In = new EdgeDirection("In") diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/ConnectedComponents.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/ConnectedComponents.scala index 121df5ad67..4d1f5e74df 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/ConnectedComponents.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/ConnectedComponents.scala @@ -14,13 +14,11 @@ object ConnectedComponents { * @tparam ED the edge attribute type (preserved in the computation) * * @param graph the graph for which to compute the connected components - * @param undirected compute reachability ignoring edge direction. * * @return a graph with vertex attributes containing the smallest vertex in each * connected component */ - def run[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]): - Graph[VertexID, ED] = { + def run[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]): Graph[VertexID, ED] = { val ccGraph = graph.mapVertices { case (vid, _) => vid } def sendMessage(edge: EdgeTriplet[VertexID, ED]) = { if (edge.srcAttr < edge.dstAttr) { diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/TriangleCount.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/TriangleCount.scala index 4b04557bc2..23c9c40594 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/TriangleCount.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/TriangleCount.scala @@ -9,11 +9,11 @@ import org.apache.spark.graphx._ * * The algorithm is relatively straightforward and can be computed in three steps: * - * 1) Compute the set of neighbors for each vertex - * 2) For each edge compute the intersection of the sets and send the - * count to both vertices. - * 3) Compute the sum at each vertex and divide by two since each - * triangle is counted twice. + *
    + *
  • Compute the set of neighbors for each vertex + *
  • For each edge compute the intersection of the sets and send the count to both vertices. + *
  • Compute the sum at each vertex and divide by two since each triangle is counted twice. + *
* * Note that the input graph should have its edges in canonical direction * (i.e. the `sourceId` less than `destId`). Also the graph must have been partitioned -- cgit v1.2.3 From 0b18bfba1aba60c2a1f576f10d9ab2fa316ebfa0 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 13 Jan 2014 18:51:04 -0800 Subject: Updated doc for PageRank. --- .../org/apache/spark/graphx/lib/PageRank.scala | 86 ++++++++++------------ 1 file changed, 39 insertions(+), 47 deletions(-) (limited to 'graphx') diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala index b2056699aa..08256dccb2 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala @@ -5,7 +5,42 @@ import scala.reflect.ClassTag import org.apache.spark.Logging import org.apache.spark.graphx._ -/** PageRank algorithm implementation. */ +/** + * PageRank algorithm implementation. There are two implementations of PageRank implemented. + * + * The first implementation uses the [[Pregel]] interface and runs PageRank for a fixed number + * of iterations: + * {{{ + * var PR = Array.fill(n)( 1.0 ) + * val oldPR = Array.fill(n)( 1.0 ) + * for( iter <- 0 until numIter ) { + * swap(oldPR, PR) + * for( i <- 0 until n ) { + * PR[i] = alpha + (1 - alpha) * inNbrs[i].map(j => oldPR[j] / outDeg[j]).sum + * } + * } + * }}} + * + * The second implementation uses the standalone [[Graph]] interface and runs PageRank until + * convergence: + * + * {{{ + * var PR = Array.fill(n)( 1.0 ) + * val oldPR = Array.fill(n)( 0.0 ) + * while( max(abs(PR - oldPr)) > tol ) { + * swap(oldPR, PR) + * for( i <- 0 until n if abs(PR[i] - oldPR[i]) > tol ) { + * PR[i] = alpha + (1 - \alpha) * inNbrs[i].map(j => oldPR[j] / outDeg[j]).sum + * } + * } + * }}} + * + * `alpha` is the random reset probability (typically 0.15), `inNbrs[i]` is the set of + * neighbors whick link to `i` and `outDeg[j]` is the out degree of vertex `j`. + * + * Note that this is not the "normalized" PageRank and as a consequence pages that have no + * inlinks will have a PageRank of alpha. + */ object PageRank extends Logging { /** @@ -13,26 +48,6 @@ object PageRank extends Logging { * with vertex attributes containing the PageRank and edge * attributes the normalized edge weight. * - * The following PageRank fixed point is computed for each vertex. - * - * {{{ - * var PR = Array.fill(n)( 1.0 ) - * val oldPR = Array.fill(n)( 1.0 ) - * for( iter <- 0 until numIter ) { - * swap(oldPR, PR) - * for( i <- 0 until n ) { - * PR[i] = alpha + (1 - alpha) * inNbrs[i].map(j => oldPR[j] / outDeg[j]).sum - * } - * } - * }}} - * - * where `alpha` is the random reset probability (typically 0.15), - * `inNbrs[i]` is the set of neighbors whick link to `i` and - * `outDeg[j]` is the out degree of vertex `j`. - * - * Note that this is not the "normalized" PageRank and as a consequence pages that have no - * inlinks will have a PageRank of alpha. - * * @tparam VD the original vertex attribute (not used) * @tparam ED the original edge attribute (not used) * @@ -47,16 +62,11 @@ object PageRank extends Logging { def run[VD: ClassTag, ED: ClassTag]( graph: Graph[VD, ED], numIter: Int, resetProb: Double = 0.15): Graph[Double, Double] = { - - /** - * Initialize the pagerankGraph with each edge attribute having - * weight 1/outDegree and each vertex with attribute 1.0. - */ + // Initialize the pagerankGraph with each edge attribute having + // weight 1/outDegree and each vertex with attribute 1.0. val pagerankGraph: Graph[Double, Double] = graph // Associate the degree with each vertex - .outerJoinVertices(graph.outDegrees){ - (vid, vdata, deg) => deg.getOrElse(0) - } + .outerJoinVertices(graph.outDegrees) { (vid, vdata, deg) => deg.getOrElse(0) } // Set the weight on the edges based on the degree .mapTriplets( e => 1.0 / e.srcAttr ) // Set the vertex attributes to the initial pagerank values @@ -85,23 +95,6 @@ object PageRank extends Logging { * Run a dynamic version of PageRank returning a graph with vertex attributes containing the * PageRank and edge attributes containing the normalized edge weight. * - * {{{ - * var PR = Array.fill(n)( 1.0 ) - * val oldPR = Array.fill(n)( 0.0 ) - * while( max(abs(PR - oldPr)) > tol ) { - * swap(oldPR, PR) - * for( i <- 0 until n if abs(PR[i] - oldPR[i]) > tol ) { - * PR[i] = alpha + (1 - \alpha) * inNbrs[i].map(j => oldPR[j] / outDeg[j]).sum - * } - * } - * }}} - * - * where `alpha` is the random reset probability (typically 0.15), `inNbrs[i]` is the set of - * neighbors whick link to `i` and `outDeg[j]` is the out degree of vertex `j`. - * - * Note that this is not the "normalized" PageRank and as a consequence pages that have no - * inlinks will have a PageRank of alpha. - * * @tparam VD the original vertex attribute (not used) * @tparam ED the original edge attribute (not used) * @@ -157,5 +150,4 @@ object PageRank extends Logging { vertexProgram, sendMessage, messageCombiner) .mapVertices((vid, attr) => attr._1) } // end of deltaPageRank - } -- cgit v1.2.3 From d4d9ece1af258ccdc83afbb8ba26345d7af16422 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Mon, 13 Jan 2014 18:45:46 -0800 Subject: Remove Graph.statistics and GraphImpl.printLineage --- .../main/scala/org/apache/spark/graphx/Graph.scala | 7 +-- .../org/apache/spark/graphx/impl/GraphImpl.scala | 65 ---------------------- .../org/apache/spark/graphx/lib/PageRank.scala | 6 -- 3 files changed, 1 insertion(+), 77 deletions(-) (limited to 'graphx') 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 1e3f3895de..7e99276d25 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala @@ -90,11 +90,6 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] { */ def partitionBy(partitionStrategy: PartitionStrategy): Graph[VD, ED] - /** - * Computes statistics describing the graph representation. - */ - def statistics: Map[String, Any] - /** * Transforms each vertex attribute in the graph using the map function. * @@ -254,7 +249,7 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] { def groupEdges(merge: (ED, ED) => ED): Graph[VD, ED] /** - * Computes statistics about the neighboring edges and vertices of each vertex. The user supplied + * Aggregates values from the neighboring edges and vertices of each vertex. The user supplied * `mapFunc` function is invoked on each edge of the graph, generating 0 or more "messages" to be * "sent" to either vertex in the edge. The `reduceFunc` is then used to combine the output of * the map phase destined to each vertex. 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 348490c186..12d46a809c 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala @@ -83,71 +83,6 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( GraphImpl(vertices, newEdges) } - override def statistics: Map[String, Any] = { - // Get the total number of vertices after replication, used to compute the replication ratio. - def numReplicatedVertices(vid2pids: RDD[Array[Array[VertexID]]]): Double = { - vid2pids.map(_.map(_.size).sum.toLong).reduce(_ + _).toDouble - } - - val numVertices = this.ops.numVertices - val numEdges = this.ops.numEdges - val replicationRatioBoth = numReplicatedVertices(routingTable.bothAttrs) / numVertices - val replicationRatioSrcOnly = numReplicatedVertices(routingTable.srcAttrOnly) / numVertices - val replicationRatioDstOnly = numReplicatedVertices(routingTable.dstAttrOnly) / numVertices - // One entry for each partition, indicate the total number of edges on that partition. - val loadArray = edges.partitionsRDD.map(_._2.size).collect().map(_.toDouble / numEdges) - val minLoad = loadArray.min - val maxLoad = loadArray.max - Map( - "Num Vertices" -> numVertices, - "Num Edges" -> numEdges, - "Replication (both)" -> replicationRatioBoth, - "Replication (src only)" -> replicationRatioSrcOnly, - "Replication (dest only)" -> replicationRatioDstOnly, - "Load Array" -> loadArray, - "Min Load" -> minLoad, - "Max Load" -> maxLoad) - } - - /** - * Display the lineage information for this graph. - */ - def printLineage() = { - def traverseLineage( - rdd: RDD[_], - indent: String = "", - visited: Map[Int, String] = Map.empty[Int, String]) { - if (visited.contains(rdd.id)) { - println(indent + visited(rdd.id)) - println(indent) - } else { - val locs = rdd.partitions.map( p => rdd.preferredLocations(p) ) - val cacheLevel = rdd.getStorageLevel - val name = rdd.id - val deps = rdd.dependencies - val partitioner = rdd.partitioner - val numparts = partitioner match { case Some(p) => p.numPartitions; case None => 0} - println(indent + name + ": " + cacheLevel.description + " (partitioner: " + partitioner + - ", " + numparts +")") - println(indent + " |---> Deps: " + deps.map(d => (d, d.rdd.id) ).toString) - println(indent + " |---> PrefLoc: " + locs.map(x=> x.toString).mkString(", ")) - deps.foreach(d => traverseLineage(d.rdd, indent + " | ", visited)) - } - } - println("edges ------------------------------------------") - traverseLineage(edges, " ") - var visited = Map(edges.id -> "edges") - println("\n\nvertices ------------------------------------------") - traverseLineage(vertices, " ", visited) - visited += (vertices.id -> "vertices") - println("\n\nroutingTable.bothAttrs -------------------------------") - traverseLineage(routingTable.bothAttrs, " ", visited) - visited += (routingTable.bothAttrs.id -> "routingTable.bothAttrs") - println("\n\ntriplets ----------------------------------------") - traverseLineage(triplets, " ", visited) - println(visited) - } // end of printLineage - override def reverse: Graph[VD, ED] = { val newETable = edges.mapEdgePartitions((pid, part) => part.reverse) new GraphImpl(vertices, newETable, routingTable, replicatedVertexView) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala index 08256dccb2..2f4d6d6864 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala @@ -73,9 +73,6 @@ object PageRank extends Logging { .mapVertices( (id, attr) => 1.0 ) .cache() - // Display statistics about pagerank - logInfo(pagerankGraph.statistics.toString) - // Define the three functions needed to implement PageRank in the GraphX // version of Pregel def vertexProgram(id: VertexID, attr: Double, msgSum: Double): Double = @@ -121,9 +118,6 @@ object PageRank extends Logging { .mapVertices( (id, attr) => (0.0, 0.0) ) .cache() - // Display statistics about pagerank - logInfo(pagerankGraph.statistics.toString) - // Define the three functions needed to implement PageRank in the GraphX // version of Pregel def vertexProgram(id: VertexID, attr: (Double, Double), msgSum: Double): (Double, Double) = { -- cgit v1.2.3 From 84d6af80211bbe15777a8d48eb37052b0b6631c4 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Mon, 13 Jan 2014 20:59:45 -0800 Subject: Make Graph{,Impl,Ops} serializable to work around capture --- graphx/src/main/scala/org/apache/spark/graphx/Graph.scala | 2 +- graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala | 2 +- graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) (limited to 'graphx') 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 7e99276d25..9dd05ade0a 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala @@ -20,7 +20,7 @@ import org.apache.spark.storage.StorageLevel * @tparam VD the vertex attribute type * @tparam ED the edge attribute type */ -abstract class Graph[VD: ClassTag, ED: ClassTag] { +abstract class Graph[VD: ClassTag, ED: ClassTag] protected () extends Serializable { /** * An RDD containing the vertices and their associated attributes. 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 447ef555ca..233adb9b87 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala @@ -14,7 +14,7 @@ import org.apache.spark.rdd.RDD * @tparam VD the vertex attribute type * @tparam ED the edge attribute type */ -class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) { +class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Serializable { /** The number of edges in the graph. */ lazy val numEdges: Long = graph.edges.count() 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 12d46a809c..56d1d9efea 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala @@ -32,7 +32,7 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( @transient val replicatedVertexView: ReplicatedVertexView[VD]) extends Graph[VD, ED] with Serializable { - /** Default construct is provided to support serialization */ + /** Default constructor is provided to support serialization */ protected def this() = this(null, null, null, null) /** Return a RDD that brings edges together with their source and destination vertices. */ -- cgit v1.2.3 From c6023bee60cee06b3dd31bb8253da6e07862c13d Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Mon, 13 Jan 2014 21:00:25 -0800 Subject: Fix infinite loop in GraphGenerators.generateRandomEdges The loop occurred when numEdges < numVertices. This commit fixes it by allowing generateRandomEdges to generate a multigraph. --- .../scala/org/apache/spark/graphx/util/GraphGenerators.scala | 9 +-------- 1 file changed, 1 insertion(+), 8 deletions(-) (limited to 'graphx') diff --git a/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala index dbea233c34..57422ce3f1 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala @@ -44,14 +44,7 @@ object GraphGenerators { def generateRandomEdges(src: Int, numEdges: Int, maxVertexID: Int): Array[Edge[Int]] = { val rand = new Random() - var dsts: Set[Int] = Set() - while (dsts.size < numEdges) { - val nextDst = rand.nextInt(maxVertexID) - if (nextDst != src) { - dsts += nextDst - } - } - dsts.map(dst => Edge[Int](src, dst, 1)).toArray + Array.fill(maxVertexID) { Edge[Int](src, rand.nextInt(maxVertexID), 1) } } /** -- cgit v1.2.3 From c28e5a08eea4f295676ea591c877c648f9f29847 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Mon, 13 Jan 2014 21:11:39 -0800 Subject: Improve scaladoc links --- graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala | 2 +- graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala | 10 +++++----- 2 files changed, 6 insertions(+), 6 deletions(-) (limited to 'graphx') 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 0269ed3cf3..c0a23d1986 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala @@ -9,7 +9,7 @@ import org.apache.spark.storage.StorageLevel /** * `EdgeRDD[ED]` extends `RDD[Edge[ED]]` by storing the edges in columnar format on each partition - * for performance. It is constructed using [[impl.EdgePartitionBuilder]]. + * for performance. */ class EdgeRDD[@specialized ED: ClassTag]( val partitionsRDD: RDD[(PartitionID, EdgePartition[ED])]) 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 233adb9b87..f10e63f059 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala @@ -254,7 +254,7 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Seriali * Run a dynamic version of PageRank returning a graph with vertex attributes containing the * PageRank and edge attributes containing the normalized edge weight. * - * @see [[org.apache.spark.graphx.lib.PageRank]], method `runUntilConvergence`. + * @see [[org.apache.spark.graphx.lib.PageRank$#runUntilConvergence]] */ def pageRank(tol: Double, resetProb: Double = 0.15): Graph[Double, Double] = { PageRank.runUntilConvergence(graph, tol, resetProb) @@ -264,7 +264,7 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Seriali * Run PageRank for a fixed number of iterations returning a graph with vertex attributes * containing the PageRank and edge attributes the normalized edge weight. * - * @see [[org.apache.spark.graphx.lib.PageRank]], method `run`. + * @see [[org.apache.spark.graphx.lib.PageRank$#run]] */ def staticPageRank(numIter: Int, resetProb: Double = 0.15): Graph[Double, Double] = { PageRank.run(graph, numIter, resetProb) @@ -274,7 +274,7 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Seriali * Compute the connected component membership of each vertex and return a graph with the vertex * value containing the lowest vertex id in the connected component containing that vertex. * - * @see [[org.apache.spark.graphx.lib.ConnectedComponents]] + * @see [[org.apache.spark.graphx.lib.ConnectedComponents$#run]] */ def connectedComponents(): Graph[VertexID, ED] = { ConnectedComponents.run(graph) @@ -283,7 +283,7 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Seriali /** * Compute the number of triangles passing through each vertex. * - * @see [[org.apache.spark.graphx.lib.TriangleCount]] + * @see [[org.apache.spark.graphx.lib.TriangleCount$#run]] */ def triangleCount(): Graph[Int, ED] = { TriangleCount.run(graph) @@ -293,7 +293,7 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Seriali * Compute the strongly connected component (SCC) of each vertex and return a graph with the * vertex value containing the lowest vertex id in the SCC containing that vertex. * - * @see [[org.apache.spark.graphx.lib.StronglyConnectedComponents]] + * @see [[org.apache.spark.graphx.lib.StronglyConnectedComponents$#run]] */ def stronglyConnectedComponents(numIter: Int): Graph[VertexID, ED] = { StronglyConnectedComponents.run(graph, numIter) -- cgit v1.2.3 From c6dbfd16942f1dc0673071580ca968dd4f8428da Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Mon, 13 Jan 2014 22:08:44 -0800 Subject: Edge object must be public for Edge case class --- graphx/src/main/scala/org/apache/spark/graphx/Edge.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) (limited to 'graphx') diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala b/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala index 21be58e740..738a38b27f 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala @@ -37,8 +37,8 @@ case class Edge[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED] if (vid == srcId) EdgeDirection.Out else { assert(vid == dstId); EdgeDirection.In } } -private[graphx] object Edge { - def lexicographicOrdering[ED] = new Ordering[Edge[ED]] { +object Edge { + private[graphx] def lexicographicOrdering[ED] = new Ordering[Edge[ED]] { override def compare(a: Edge[ED], b: Edge[ED]): Int = (if (a.srcId != b.srcId) a.srcId - b.srcId else a.dstId - b.dstId).toInt } -- cgit v1.2.3 From 76ebdae79866f4721ef39aaccfa89a255d3cea6a Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Mon, 13 Jan 2014 22:20:45 -0800 Subject: Fix bug in GraphLoader.edgeListFile that caused srcId > dstId --- graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) (limited to 'graphx') 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 7bdb101efb..5904aa3a28 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala @@ -53,7 +53,7 @@ object GraphLoader extends Logging { } val srcId = lineArray(0).toLong val dstId = lineArray(1).toLong - if (canonicalOrientation && dstId > srcId) { + if (canonicalOrientation && srcId > dstId) { builder.add(dstId, srcId, 1) } else { builder.add(srcId, dstId, 1) -- cgit v1.2.3 From af645be5b8d41d5a0fd4a529956c5ab438198db4 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Mon, 13 Jan 2014 22:29:45 -0800 Subject: Fix all code examples in guide --- docs/graphx-programming-guide.md | 46 ++++++++++++++++++++-------------------- graphx/data/users.txt | 13 ++++++------ 2 files changed, 30 insertions(+), 29 deletions(-) (limited to 'graphx') diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md index 226299a759..a7ab00306e 100644 --- a/docs/graphx-programming-guide.md +++ b/docs/graphx-programming-guide.md @@ -357,7 +357,7 @@ val relationships: RDD[Edge[String]] = val defaultUser = ("John Doe", "Missing") // Build the initial Graph val graph = Graph(users, relationships, defaultUser) -// Notice that there is a user 0 (for which we have no information) connecting users +// Notice that there is a user 0 (for which we have no information) connected to users // 4 (peter) and 5 (franklin). graph.triplets.map( triplet => triplet.srcAttr._1 + " is the " + triplet.attr + " of " + triplet.dstAttr._1 @@ -858,11 +858,11 @@ val graph = GraphLoader.edgeListFile(sc, "graphx/data/followers.txt") val ranks = graph.pageRank(0.0001).vertices // Join the ranks with the usernames val users = sc.textFile("graphx/data/users.txt").map { line => - val fields = line.split("\\s+") + val fields = line.split(",") (fields(0).toLong, fields(1)) } -val ranksByUsername = users.leftOuterJoin(ranks).map { - case (id, (username, rankOpt)) => (username, rankOpt.getOrElse(0.0)) +val ranksByUsername = users.join(ranks).map { + case (id, (username, rank)) => (username, rank) } // Print the result println(ranksByUsername.collect().mkString("\n")) @@ -881,11 +881,11 @@ val graph = GraphLoader.edgeListFile(sc, "graphx/data/followers.txt") val cc = graph.connectedComponents().vertices // Join the connected components with the usernames val users = sc.textFile("graphx/data/users.txt").map { line => - val fields = line.split("\\s+") + val fields = line.split(",") (fields(0).toLong, fields(1)) } -val ccByUsername = users.join(cc).map { case (id, (username, cc)) => - (username, cc) +val ccByUsername = users.join(cc).map { + case (id, (username, cc)) => (username, cc) } // Print the result println(ccByUsername.collect().mkString("\n")) @@ -900,12 +900,12 @@ A vertex is part of a triangle when it has two adjacent vertices with an edge be {% highlight scala %} // Load the edges in canonical order and partition the graph for triangle count -val graph = GraphLoader.edgeListFile(sc, "graphx/data/followers.txt", true).partitionBy(RandomVertexCut) +val graph = GraphLoader.edgeListFile(sc, "graphx/data/followers.txt", true).partitionBy(PartitionStrategy.RandomVertexCut) // Find the triangle count for each vertex val triCounts = graph.triangleCount().vertices // Join the triangle counts with the usernames val users = sc.textFile("graphx/data/users.txt").map { line => - val fields = line.split("\\s+") + val fields = line.split(",") (fields(0).toLong, fields(1)) } val triCountByUsername = users.join(triCounts).map { case (id, (username, tc)) => @@ -934,32 +934,32 @@ all of this in just a few lines with GraphX: // Connect to the Spark cluster val sc = new SparkContext("spark://master.amplab.org", "research") -// Load my user data and prase into tuples of user id and attribute list -val users = sc.textFile("hdfs://user_attributes.tsv") - .map(line => line.split).map( parts => (parts.head, parts.tail) ) +// Load my user data and parse into tuples of user id and attribute list +val users = (sc.textFile("graphx/data/users.txt") + .map(line => line.split(",")).map( parts => (parts.head.toLong, parts.tail) )) // Parse the edge data which is already in userId -> userId format -val followerGraph = Graph.textFile(sc, "hdfs://followers.tsv") +val followerGraph = GraphLoader.edgeListFile(sc, "graphx/data/followers.txt") // Attach the user attributes -val graph = followerGraph.outerJoinVertices(users){ +val graph = followerGraph.outerJoinVertices(users) { case (uid, deg, Some(attrList)) => attrList // Some users may not have attributes so we set them as empty case (uid, deg, None) => Array.empty[String] - } +} -// Restrict the graph to users which have exactly two attributes -val subgraph = graph.subgraph((vid, attr) => attr.size == 2) +// Restrict the graph to users with usernames and names +val subgraph = graph.subgraph(vpred = (vid, attr) => attr.size == 2) // Compute the PageRank -val pagerankGraph = Analytics.pagerank(subgraph) +val pagerankGraph = subgraph.pageRank(0.001) // Get the attributes of the top pagerank users -val userInfoWithPageRank = subgraph.outerJoinVertices(pagerankGraph.vertices){ - case (uid, attrList, Some(pr)) => (pr, attrList) - case (uid, attrList, None) => (pr, attrList) - } +val userInfoWithPageRank = subgraph.outerJoinVertices(pagerankGraph.vertices) { + case (uid, attrList, Some(pr)) => (pr, attrList.toList) + case (uid, attrList, None) => (0.0, attrList.toList) +} -println(userInfoWithPageRank.top(5)) +println(userInfoWithPageRank.vertices.top(5)(Ordering.by(_._2._1)).mkString("\n")) {% endhighlight %} diff --git a/graphx/data/users.txt b/graphx/data/users.txt index 26e3b3bb4d..982d19d50b 100644 --- a/graphx/data/users.txt +++ b/graphx/data/users.txt @@ -1,6 +1,7 @@ -1 BarackObama -2 ladygaga -3 jeresig -4 justinbieber -6 matei_zaharia -7 odersky +1,BarackObama,Barack Obama +2,ladygaga,Goddess of Love +3,jeresig,John Resig +4,justinbieber,Justin Bieber +6,matei_zaharia,Matei Zaharia +7,odersky,Martin Odersky +8,anonsys -- cgit v1.2.3 From 80e73ed0004cceb47a450c79aa4faa598502fa45 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Mon, 13 Jan 2014 22:56:57 -0800 Subject: Adding minimal additional functionality to EdgeRDD --- .../main/scala/org/apache/spark/graphx/EdgeRDD.scala | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) (limited to 'graphx') 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 c0a23d1986..832b7816fe 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala @@ -57,6 +57,23 @@ class EdgeRDD[@specialized ED: ClassTag]( }, preservesPartitioning = true)) } + /** + * Map the values in an edge partitioning preserving the structure but changing the values. + * + * @tparam ED2 the new edge value type + * @param f the function from an edge to a new edge value + * @return a new EdgeRDD containing the new edge values + */ + def mapValues[ED2: ClassTag](f: Edge[ED] => ED2): EdgeRDD[ED2] = + mapEdgePartitions((pid, part) => part.map(f)) + + /** + * Reverse all the edges in this RDD. + * + * @return a new EdgeRDD containing all the edges reversed + */ + def reverse: EdgeRDD[ED] = mapEdgePartitions((pid, part) => part.reverse) + /** * Inner joins this EdgeRDD with another EdgeRDD, assuming both are partitioned using the same * [[PartitionStrategy]]. -- cgit v1.2.3