diff options
Diffstat (limited to 'graphx/src/main/scala/org')
-rw-r--r-- | graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala | 9 | ||||
-rw-r--r-- | graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala | 41 |
2 files changed, 41 insertions, 9 deletions
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala b/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala index 1526ccef06..ef412cfd4e 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala @@ -119,4 +119,13 @@ object PartitionStrategy { math.abs((lower, higher).hashCode()) % numParts } } + + /** Returns the PartitionStrategy with the specified name. */ + def fromString(s: String): PartitionStrategy = s match { + case "RandomVertexCut" => RandomVertexCut + case "EdgePartition1D" => EdgePartition1D + case "EdgePartition2D" => EdgePartition2D + case "CanonicalRandomVertexCut" => CanonicalRandomVertexCut + case _ => throw new IllegalArgumentException("Invalid PartitionStrategy: " + s) + } } 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 a3c8de3f90..635514f09e 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 @@ -38,19 +38,42 @@ object GraphGenerators { val RMATa = 0.45 val RMATb = 0.15 val RMATd = 0.25 + /** * Generate a graph whose vertex out degree is log normal. + * + * The default values for mu and sigma are taken from the Pregel paper: + * + * Grzegorz Malewicz, Matthew H. Austern, Aart J.C Bik, James C. Dehnert, + * Ilan Horn, Naty Leiser, and Grzegorz Czajkowski. 2010. + * Pregel: a system for large-scale graph processing. SIGMOD '10. + * + * @param sc + * @param numVertices + * @param mu + * @param sigma + * @return */ - def logNormalGraph(sc: SparkContext, numVertices: Int): Graph[Int, Int] = { - // based on Pregel settings - val mu = 4 - val sigma = 1.3 - - val vertices: RDD[(VertexId, Int)] = sc.parallelize(0 until numVertices).map{ - src => (src, sampleLogNormal(mu, sigma, numVertices)) + def logNormalGraph(sc: SparkContext, numVertices: Int, numEParts: Int, + mu: Double = 4.0, sigma: Double = 1.3): Graph[Long, Int] = { + val vertices = sc.parallelize(0 until numVertices, numEParts).map { src => + // Initialize the random number generator with the source vertex id + val rand = new Random(src) + val degree = math.min(numVertices.toLong, math.exp(rand.nextGaussian() * sigma + mu).toLong) + (src.toLong, degree) } - val edges = vertices.flatMap { v => - generateRandomEdges(v._1.toInt, v._2, numVertices) + val edges = vertices.flatMap { case (src, degree) => + new Iterator[Edge[Int]] { + // Initialize the random number generator with the source vertex id + val rand = new Random(src) + var i = 0 + override def hasNext(): Boolean = { i < degree } + override def next(): Edge[Int] = { + val nextEdge = Edge[Int](src, rand.nextInt(numVertices), i) + i += 1 + nextEdge + } + } } Graph(vertices, edges, 0) } |