From 015010c4213594be5c9086f82743adebfd88a936 Mon Sep 17 00:00:00 2001 From: RJ Nowling Date: Tue, 26 Aug 2014 16:17:59 -0400 Subject: [PATCH 01/19] Fixed GraphGenerator logNormalGraph API to make backward-incompatible change in commit 894ecde04 --- .../org/apache/spark/graphx/util/GraphGenerators.scala | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) 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 60149548ab852..f16882c506ff3 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 @@ -50,13 +50,17 @@ object GraphGenerators { * * @param sc * @param numVertices + * @param numEParts * @param mu * @param sigma * @return */ - def logNormalGraph(sc: SparkContext, numVertices: Int, numEParts: Int, + def logNormalGraph(sc: SparkContext, numVertices: Int, numEParts: Int = 0, mu: Double = 4.0, sigma: Double = 1.3): Graph[Long, Int] = { - val vertices = sc.parallelize(0 until numVertices, numEParts).map { src => + + val evalNumEParts = if (numEParts == 0) sc.defaultParallelism else numEParts + + val vertices = sc.parallelize(0 until numVertices, evalNumEParts).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) From c1831368ce36bbac48c2733ffa3077be453aedae Mon Sep 17 00:00:00 2001 From: RJ Nowling Date: Wed, 27 Aug 2014 16:03:45 -0400 Subject: [PATCH 02/19] Fix to deterministic GraphGenerators.logNormalGraph that allows generating graphs randomly using optional seed. --- .../spark/graphx/util/GraphGenerators.scala | 19 ++++++++++++++----- 1 file changed, 14 insertions(+), 5 deletions(-) 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 f16882c506ff3..0ae9c1ca5fc05 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 @@ -48,28 +48,37 @@ object GraphGenerators { * Ilan Horn, Naty Leiser, and Grzegorz Czajkowski. 2010. * Pregel: a system for large-scale graph processing. SIGMOD '10. * + * If the seed is set to 0 (default), the seeds are set deterministically + * based on source vertex IDs. If the seed is -1, a random seed is chosen. + * Otherwise, use the user-specified seed. + * * @param sc * @param numVertices * @param numEParts * @param mu * @param sigma + * @param seed * @return */ def logNormalGraph(sc: SparkContext, numVertices: Int, numEParts: Int = 0, - mu: Double = 4.0, sigma: Double = 1.3): Graph[Long, Int] = { + mu: Double = 4.0, sigma: Double = 1.3, seed: Int = 0): Graph[Long, Int] = { - val evalNumEParts = if (numEParts == 0) sc.defaultParallelism else numEParts + val evalNumEParts = if (numEParts == 0) sc.defaultParallelism else numEParts + val evalSeed = if (seed == -1) (new Random().nextInt()) else seed + val vertices = sc.parallelize(0 until numVertices, evalNumEParts).map { src => // Initialize the random number generator with the source vertex id - val rand = new Random(src) + // if seed == 0 + val rand = new Random(evalSeed ^ src) val degree = math.min(numVertices.toLong, math.exp(rand.nextGaussian() * sigma + mu).toLong) (src.toLong, degree) } 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) + // if seed == 0 + val rand = new Random(evalSeed ^ src) var i = 0 override def hasNext(): Boolean = { i < degree } override def next(): Edge[Int] = { @@ -86,7 +95,7 @@ object GraphGenerators { // the edge data is the weight (default 1) val RMATc = 0.15 - def generateRandomEdges(src: Int, numEdges: Int, maxVertexId: Int): Array[Edge[Int]] = { + def generateRandomEdges(src: Int, numEdges: Int, maxVertexId: Int, seed: Int = 0): Array[Edge[Int]] = { val rand = new Random() Array.fill(maxVertexId) { Edge[Int](src, rand.nextInt(maxVertexId), 1) } } From 684804d4c5c33bd4d50a179b15e3b1b0375ff1d8 Mon Sep 17 00:00:00 2001 From: RJ Nowling Date: Wed, 27 Aug 2014 17:06:51 -0400 Subject: [PATCH 03/19] revert PR #720 which introduce errors in logNormalGraph and messed up seeding of RNGs. Add user-defined optional seed for deterministic behavior --- .../spark/graphx/util/GraphGenerators.scala | 42 ++++++++----------- 1 file changed, 17 insertions(+), 25 deletions(-) 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 0ae9c1ca5fc05..6fdd3391dba6a 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 @@ -61,33 +61,23 @@ object GraphGenerators { * @return */ def logNormalGraph(sc: SparkContext, numVertices: Int, numEParts: Int = 0, - mu: Double = 4.0, sigma: Double = 1.3, seed: Int = 0): Graph[Long, Int] = { + mu: Double = 4.0, sigma: Double = 1.3, seed: Long = -1): Graph[Long, Int] = { val evalNumEParts = if (numEParts == 0) sc.defaultParallelism else numEParts - val evalSeed = if (seed == -1) (new Random().nextInt()) else seed - - val vertices = sc.parallelize(0 until numVertices, evalNumEParts).map { src => - // Initialize the random number generator with the source vertex id - // if seed == 0 - val rand = new Random(evalSeed ^ src) - val degree = math.min(numVertices.toLong, math.exp(rand.nextGaussian() * sigma + mu).toLong) - (src.toLong, degree) + // Enable deterministic seeding + val seedRand = if (seed == -1) new Random() else new Random(seed) + val seed1 = seedRand.nextInt() + val seed2 = seedRand.nextInt() + + val vertices : RDD[(VertexId, Long)] = sc.parallelize(0 until numVertices, evalNumEParts).map { + src => (src, sampleLogNormal(mu, sigma, numVertices, seed=(seed1 ^ src))) } + val edges = vertices.flatMap { case (src, degree) => - new Iterator[Edge[Int]] { - // Initialize the random number generator with the source vertex id - // if seed == 0 - val rand = new Random(evalSeed ^ 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 - } - } + generateRandomEdges(src.toInt, degree.toInt, numVertices, seed=(seed2 ^ src)) } + Graph(vertices, edges, 0) } @@ -95,8 +85,8 @@ object GraphGenerators { // the edge data is the weight (default 1) val RMATc = 0.15 - def generateRandomEdges(src: Int, numEdges: Int, maxVertexId: Int, seed: Int = 0): Array[Edge[Int]] = { - val rand = new Random() + def generateRandomEdges(src: Int, numEdges: Int, maxVertexId: Int, seed: Long = -1): Array[Edge[Int]] = { + val rand = if (seed == -1) new Random() else new Random(seed) Array.fill(maxVertexId) { Edge[Int](src, rand.nextInt(maxVertexId), 1) } } @@ -110,9 +100,11 @@ object GraphGenerators { * @param mu the mean of the normal distribution * @param sigma the standard deviation of the normal distribution * @param maxVal exclusive upper bound on the value of the sample + * @param seed optional seed */ - private def sampleLogNormal(mu: Double, sigma: Double, maxVal: Int): Int = { - val rand = new Random() + private def sampleLogNormal(mu: Double, sigma: Double, maxVal: Int, seed: Long = -1): Int = { + val rand = if (seed == -1) new Random() else new Random(seed) + val sigmaSq = sigma * sigma val m = math.exp(mu + sigmaSq / 2.0) // expm1 is exp(m)-1 with better accuracy for tiny m From d40141aff30b51e6341e8165a336f9060ffa9c8a Mon Sep 17 00:00:00 2001 From: RJ Nowling Date: Wed, 27 Aug 2014 17:20:21 -0400 Subject: [PATCH 04/19] Fix style error --- .../scala/org/apache/spark/graphx/util/GraphGenerators.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 6fdd3391dba6a..8db2298b1f0e8 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 @@ -85,7 +85,8 @@ object GraphGenerators { // the edge data is the weight (default 1) val RMATc = 0.15 - def generateRandomEdges(src: Int, numEdges: Int, maxVertexId: Int, seed: Long = -1): Array[Edge[Int]] = { + def generateRandomEdges(src: Int, numEdges: Int, maxVertexId: Int, + seed: Long = -1): Array[Edge[Int]] = { val rand = if (seed == -1) new Random() else new Random(seed) Array.fill(maxVertexId) { Edge[Int](src, rand.nextInt(maxVertexId), 1) } } From 98bb73ca0fac1739ad748725bcd35a729c626381 Mon Sep 17 00:00:00 2001 From: RJ Nowling Date: Thu, 28 Aug 2014 09:17:26 -0400 Subject: [PATCH 05/19] Add documentation for logNormalGraph parameters --- .../spark/graphx/util/GraphGenerators.scala | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) 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 8db2298b1f0e8..bddbbadbbb5c5 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 @@ -40,7 +40,7 @@ object GraphGenerators { val RMATd = 0.25 /** - * Generate a graph whose vertex out degree is log normal. + * Generate a graph whose vertex out degree distribution is log normal. * * The default values for mu and sigma are taken from the Pregel paper: * @@ -52,13 +52,13 @@ object GraphGenerators { * based on source vertex IDs. If the seed is -1, a random seed is chosen. * Otherwise, use the user-specified seed. * - * @param sc - * @param numVertices - * @param numEParts - * @param mu - * @param sigma - * @param seed - * @return + * @param sc Spark Context + * @param numVertices number of vertices in generated graph + * @param numEParts (optional) number of partitions + * @param mu (optional, default: 4.0) mean of out-degree distribution + * @param sigma (optional, default: 1.3) standard deviation of out-degree distribution + * @param seed (optional, default: -1) seed for RNGs, -1 causes a random seed to be chosen + * @return Graph object */ def logNormalGraph(sc: SparkContext, numVertices: Int, numEParts: Int = 0, mu: Double = 4.0, sigma: Double = 1.3, seed: Long = -1): Graph[Long, Int] = { From 1ff8d3049f779a9da594e7efd610e9189a328c7d Mon Sep 17 00:00:00 2001 From: RJ Nowling Date: Thu, 28 Aug 2014 09:36:28 -0400 Subject: [PATCH 06/19] Fix bug in generateRandomEdges where numVertices instead of numEdges was used to control number of edges to generate --- .../scala/org/apache/spark/graphx/util/GraphGenerators.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 bddbbadbbb5c5..aab95135ef11a 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 @@ -88,7 +88,7 @@ object GraphGenerators { def generateRandomEdges(src: Int, numEdges: Int, maxVertexId: Int, seed: Long = -1): Array[Edge[Int]] = { val rand = if (seed == -1) new Random() else new Random(seed) - Array.fill(maxVertexId) { Edge[Int](src, rand.nextInt(maxVertexId), 1) } + Array.fill(numEdges) { Edge[Int](src, rand.nextInt(maxVertexId), 1) } } /** From b5eeb80c58f15a7a87c2c03697bc2922b298a0d1 Mon Sep 17 00:00:00 2001 From: RJ Nowling Date: Thu, 28 Aug 2014 09:39:26 -0400 Subject: [PATCH 07/19] Add optional seed parameter to SynthBenchmark and set default to randomly generate a seed --- .../org/apache/spark/examples/graphx/SynthBenchmark.scala | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/graphx/SynthBenchmark.scala b/examples/src/main/scala/org/apache/spark/examples/graphx/SynthBenchmark.scala index 551c339b19523..b60ba861d8829 100644 --- a/examples/src/main/scala/org/apache/spark/examples/graphx/SynthBenchmark.scala +++ b/examples/src/main/scala/org/apache/spark/examples/graphx/SynthBenchmark.scala @@ -44,6 +44,7 @@ object SynthBenchmark { * -mu the mean parameter for the log-normal graph (Default: 4.0) * -sigma the stdev parameter for the log-normal graph (Default: 1.3) * -degFile the local file to save the degree information (Default: Empty) + * -seed seed to use for RNGs (Default: -1, picks seed randomly) */ def main(args: Array[String]) { val options = args.map { @@ -62,6 +63,7 @@ object SynthBenchmark { var mu: Double = 4.0 var sigma: Double = 1.3 var degFile: String = "" + var seed: Int = -1 options.foreach { case ("app", v) => app = v @@ -72,6 +74,7 @@ object SynthBenchmark { case ("mu", v) => mu = v.toDouble case ("sigma", v) => sigma = v.toDouble case ("degFile", v) => degFile = v + case ("seed", v) => seed = v.toInt case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) } @@ -85,7 +88,7 @@ object SynthBenchmark { // Create the graph println(s"Creating graph...") val unpartitionedGraph = GraphGenerators.logNormalGraph(sc, numVertices, - numEPart.getOrElse(sc.defaultParallelism), mu, sigma) + numEPart.getOrElse(sc.defaultParallelism), mu, sigma, seed) // Repartition the graph val graph = partitionStrategy.foldLeft(unpartitionedGraph)(_.partitionBy(_)).cache() From dfbb6dd86256ba176a304d103c9b0b9fe00d302b Mon Sep 17 00:00:00 2001 From: RJ Nowling Date: Thu, 28 Aug 2014 09:41:20 -0400 Subject: [PATCH 08/19] Fix parameter name in SynthBenchmark docs --- .../scala/org/apache/spark/examples/graphx/SynthBenchmark.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/graphx/SynthBenchmark.scala b/examples/src/main/scala/org/apache/spark/examples/graphx/SynthBenchmark.scala index b60ba861d8829..2b72afee49c9e 100644 --- a/examples/src/main/scala/org/apache/spark/examples/graphx/SynthBenchmark.scala +++ b/examples/src/main/scala/org/apache/spark/examples/graphx/SynthBenchmark.scala @@ -38,7 +38,7 @@ object SynthBenchmark { * Options: * -app "pagerank" or "cc" for pagerank or connected components. (Default: pagerank) * -niters the number of iterations of pagerank to use (Default: 10) - * -numVertices the number of vertices in the graph (Default: 1000000) + * -nverts the number of vertices in the graph (Default: 1000000) * -numEPart the number of edge partitions in the graph (Default: number of cores) * -partStrategy the graph partitioning strategy to use * -mu the mean parameter for the log-normal graph (Default: 4.0) From 1c8fc4407cabc8c592bde4b696bd09f5b3766da5 Mon Sep 17 00:00:00 2001 From: RJ Nowling Date: Thu, 28 Aug 2014 09:46:01 -0400 Subject: [PATCH 09/19] Connected components part of SynthBenchmark was failing to call count on RDD before printing --- .../scala/org/apache/spark/examples/graphx/SynthBenchmark.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/graphx/SynthBenchmark.scala b/examples/src/main/scala/org/apache/spark/examples/graphx/SynthBenchmark.scala index 2b72afee49c9e..5f35a5836462e 100644 --- a/examples/src/main/scala/org/apache/spark/examples/graphx/SynthBenchmark.scala +++ b/examples/src/main/scala/org/apache/spark/examples/graphx/SynthBenchmark.scala @@ -116,7 +116,7 @@ object SynthBenchmark { println(s"Total PageRank = $totalPR") } else if (app == "cc") { println("Running Connected Components") - val numComponents = graph.connectedComponents.vertices.map(_._2).distinct() + val numComponents = graph.connectedComponents.vertices.map(_._2).distinct().count() println(s"Number of components = $numComponents") } val runTime = System.currentTimeMillis() - startTime From 6803da168c9ba8f990d568b2a9a17e480031626a Mon Sep 17 00:00:00 2001 From: RJ Nowling Date: Thu, 28 Aug 2014 10:14:25 -0400 Subject: [PATCH 10/19] Add GraphGeneratorsSuite with test for generateRandomEdges --- .../graphx/util/GraphGeneratorsSuite.scala | 55 +++++++++++++++++++ 1 file changed, 55 insertions(+) create mode 100644 graphx/src/test/scala/org/apache/spark/graphx/util/GraphGeneratorsSuite.scala diff --git a/graphx/src/test/scala/org/apache/spark/graphx/util/GraphGeneratorsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/util/GraphGeneratorsSuite.scala new file mode 100644 index 0000000000000..3356110e51147 --- /dev/null +++ b/graphx/src/test/scala/org/apache/spark/graphx/util/GraphGeneratorsSuite.scala @@ -0,0 +1,55 @@ +/* + * 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 org.scalatest.FunSuite + +import org.apache.spark.SparkContext +import org.apache.spark.graphx.util.GraphGenerators +import org.apache.spark.rdd._ + +class GraphGeneratorsSuite extends FunSuite with LocalSparkContext { + + test("GraphGenerators.generateRandomEdges") { + val src = 5 + val numEdges10 = 10 + val numEdges20 = 20 + val maxVertexId = 100 + + val edges10 = GraphGenerators.generateRandomEdges(src, numEdges10, maxVertexId) + assert(edges10.length == numEdges10) + + val correctSrc = edges10.forall( e => e.srcId == src ) + assert(correctSrc) + + val correctWeight = edges10.forall( e => e.attr == 1 ) + assert(correctWeight) + + val correctRange = edges10.forall( e => e.dstId >= 0 && e.dstId <= maxVertexId ) + assert(correctRange) + + val edges20 = GraphGenerators.generateRandomEdges(src, numEdges20, maxVertexId) + assert(edges20.length == numEdges20) + + val edges10_round1 = GraphGenerators.generateRandomEdges(src, numEdges10, maxVertexId, seed=12345) + val edges10_round2 = GraphGenerators.generateRandomEdges(src, numEdges10, maxVertexId, seed=12345) + assert(edges10_round1 == edges10_round2) + + } + +} From b99cba9847adc8e6e1c3639c8fcbee7be1748fc1 Mon Sep 17 00:00:00 2001 From: RJ Nowling Date: Thu, 28 Aug 2014 10:21:31 -0400 Subject: [PATCH 11/19] Make sampleLogNormal private to Spark (vs private) for unit testing --- .../scala/org/apache/spark/graphx/util/GraphGenerators.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 aab95135ef11a..dcbb954e5c2bc 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 @@ -103,7 +103,7 @@ object GraphGenerators { * @param maxVal exclusive upper bound on the value of the sample * @param seed optional seed */ - private def sampleLogNormal(mu: Double, sigma: Double, maxVal: Int, seed: Long = -1): Int = { + private[spark] def sampleLogNormal(mu: Double, sigma: Double, maxVal: Int, seed: Long = -1): Int = { val rand = if (seed == -1) new Random() else new Random(seed) val sigmaSq = sigma * sigma From 82f223972ad38ab071c7b624ad529b1d2b3ed0e8 Mon Sep 17 00:00:00 2001 From: RJ Nowling Date: Thu, 28 Aug 2014 10:21:50 -0400 Subject: [PATCH 12/19] Add unit test for sampleLogNormal --- .../spark/graphx/util/GraphGeneratorsSuite.scala | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/graphx/src/test/scala/org/apache/spark/graphx/util/GraphGeneratorsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/util/GraphGeneratorsSuite.scala index 3356110e51147..200524293fe17 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/util/GraphGeneratorsSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/util/GraphGeneratorsSuite.scala @@ -49,7 +49,19 @@ class GraphGeneratorsSuite extends FunSuite with LocalSparkContext { val edges10_round1 = GraphGenerators.generateRandomEdges(src, numEdges10, maxVertexId, seed=12345) val edges10_round2 = GraphGenerators.generateRandomEdges(src, numEdges10, maxVertexId, seed=12345) assert(edges10_round1 == edges10_round2) + } + + test("GraphGenerators.sampleLogNormal") { + val mu = 4.0 + val sigma = 1.3 + val maxVal = 100 + + val dstId = GraphGenerators.sampleLogNormal(mu, sigma, maxVal) + assert(dstId < maxVal) + val dstId_round1 = GraphGenerators.sampleLogNormal(mu, sigma, maxVal, 12345) + val dstId_round2 = GraphGenerators.sampleLogNormal(mu, sigma, maxVal, 12345) + assert(dstId_round1 == dstId_round2) } } From 2faf75f01d6dbebc56bfea9509945bc21501d451 Mon Sep 17 00:00:00 2001 From: RJ Nowling Date: Thu, 28 Aug 2014 10:28:09 -0400 Subject: [PATCH 13/19] Added unit test for logNormalGraph --- .../graphx/util/GraphGeneratorsSuite.scala | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/graphx/src/test/scala/org/apache/spark/graphx/util/GraphGeneratorsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/util/GraphGeneratorsSuite.scala index 200524293fe17..60266a4474fe7 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/util/GraphGeneratorsSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/util/GraphGeneratorsSuite.scala @@ -64,4 +64,23 @@ class GraphGeneratorsSuite extends FunSuite with LocalSparkContext { assert(dstId_round1 == dstId_round2) } + test("GraphGenerators.logNormalGraph") { + withSpark { sc => + val mu = 4.0 + val sigma = 1.3 + val numVertices100 = 100 + + val graph = GraphGenerators.logNormalGraph(sc, numVertices100, mu=mu, sigma=sigma) + assert(graph.vertices.count() == numVertices100) + + val graph_round1 = GraphGenerators.logNormalGraph(sc, numVertices100, mu=mu, sigma=sigma, seed=12345) + val graph_round2 = GraphGenerators.logNormalGraph(sc, numVertices100, mu=mu, sigma=sigma, seed=12345) + + assert(graph_round1.edges == graph_round2.edges) + + val graph_round3 = GraphGenerators.logNormalGraph(sc, numVertices100, mu=mu, sigma=sigma, seed=567) + assert(graph_round1.edges != graph_round3.edges) + } + } + } From 43949ad1c06823ed6385d64ff40810d370d28e70 Mon Sep 17 00:00:00 2001 From: RJ Nowling Date: Thu, 28 Aug 2014 10:28:53 -0400 Subject: [PATCH 14/19] Added test for different seeds for generateRandomEdges --- .../org/apache/spark/graphx/util/GraphGeneratorsSuite.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/graphx/src/test/scala/org/apache/spark/graphx/util/GraphGeneratorsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/util/GraphGeneratorsSuite.scala index 60266a4474fe7..60425f41efc9e 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/util/GraphGeneratorsSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/util/GraphGeneratorsSuite.scala @@ -49,6 +49,9 @@ class GraphGeneratorsSuite extends FunSuite with LocalSparkContext { val edges10_round1 = GraphGenerators.generateRandomEdges(src, numEdges10, maxVertexId, seed=12345) val edges10_round2 = GraphGenerators.generateRandomEdges(src, numEdges10, maxVertexId, seed=12345) assert(edges10_round1 == edges10_round2) + + val edges10_round3 = GraphGenerators.generateRandomEdges(src, numEdges10, maxVertexId, seed=3467) + assert(edges10_round1 != edges10_round3) } test("GraphGenerators.sampleLogNormal") { From 799f00203dc30f070f1725bca697977a54b03b84 Mon Sep 17 00:00:00 2001 From: RJ Nowling Date: Thu, 28 Aug 2014 10:33:02 -0400 Subject: [PATCH 15/19] Added test for different seeds for sampleLogNormal --- .../org/apache/spark/graphx/util/GraphGeneratorsSuite.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/graphx/src/test/scala/org/apache/spark/graphx/util/GraphGeneratorsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/util/GraphGeneratorsSuite.scala index 60425f41efc9e..854a3651802e7 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/util/GraphGeneratorsSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/util/GraphGeneratorsSuite.scala @@ -65,6 +65,9 @@ class GraphGeneratorsSuite extends FunSuite with LocalSparkContext { val dstId_round1 = GraphGenerators.sampleLogNormal(mu, sigma, maxVal, 12345) val dstId_round2 = GraphGenerators.sampleLogNormal(mu, sigma, maxVal, 12345) assert(dstId_round1 == dstId_round2) + + val dstId_round3 = GraphGenerators.sampleLogNormal(mu, sigma, maxVal, 789) + assert(dstId_round1 != dstId_round3) } test("GraphGenerators.logNormalGraph") { From 41fd1f80e9d0f89518775e3ffe08959e8eedc87a Mon Sep 17 00:00:00 2001 From: RJ Nowling Date: Thu, 28 Aug 2014 10:39:04 -0400 Subject: [PATCH 16/19] Fix logNormalGraph scala doc for seed --- .../scala/org/apache/spark/graphx/util/GraphGenerators.scala | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) 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 dcbb954e5c2bc..3c16f0b5ca523 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 @@ -48,9 +48,8 @@ object GraphGenerators { * Ilan Horn, Naty Leiser, and Grzegorz Czajkowski. 2010. * Pregel: a system for large-scale graph processing. SIGMOD '10. * - * If the seed is set to 0 (default), the seeds are set deterministically - * based on source vertex IDs. If the seed is -1, a random seed is chosen. - * Otherwise, use the user-specified seed. + * If the seed is -1, a random seed is chosen. Otherwise, use the + * user-specified seed. * * @param sc Spark Context * @param numVertices number of vertices in generated graph From c70868dd991f375c73015587fc48f0139e7bc52f Mon Sep 17 00:00:00 2001 From: RJ Nowling Date: Thu, 28 Aug 2014 10:39:29 -0400 Subject: [PATCH 17/19] Fix logNormalGraph scala doc for seed --- .../scala/org/apache/spark/graphx/util/GraphGenerators.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 3c16f0b5ca523..0c5df507de35f 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 @@ -48,8 +48,8 @@ object GraphGenerators { * Ilan Horn, Naty Leiser, and Grzegorz Czajkowski. 2010. * Pregel: a system for large-scale graph processing. SIGMOD '10. * - * If the seed is -1, a random seed is chosen. Otherwise, use the - * user-specified seed. + * If the seed is -1 (defaut), a random seed is chosen. Otherwise, use + * the user-specified seed. * * @param sc Spark Context * @param numVertices number of vertices in generated graph From 785ac7026a65e49786c955fd8a4ccc2ed89c1fbf Mon Sep 17 00:00:00 2001 From: RJ Nowling Date: Thu, 28 Aug 2014 10:59:18 -0400 Subject: [PATCH 18/19] Fix style error --- .../scala/org/apache/spark/graphx/util/GraphGenerators.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 0c5df507de35f..7f7a1e9dcf744 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 @@ -102,7 +102,8 @@ object GraphGenerators { * @param maxVal exclusive upper bound on the value of the sample * @param seed optional seed */ - private[spark] def sampleLogNormal(mu: Double, sigma: Double, maxVal: Int, seed: Long = -1): Int = { + private[spark] def sampleLogNormal(mu: Double, sigma: Double, maxVal: Int, + seed: Long = -1): Int = { val rand = if (seed == -1) new Random() else new Random(seed) val sigmaSq = sigma * sigma From e11918e257be8526867da302c61f4b973b32e41f Mon Sep 17 00:00:00 2001 From: RJ Nowling Date: Thu, 28 Aug 2014 13:31:17 -0400 Subject: [PATCH 19/19] Fix bad comparisons in unit tests --- .../graphx/util/GraphGeneratorsSuite.scala | 22 +++++++++++++++---- 1 file changed, 18 insertions(+), 4 deletions(-) diff --git a/graphx/src/test/scala/org/apache/spark/graphx/util/GraphGeneratorsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/util/GraphGeneratorsSuite.scala index 854a3651802e7..cfb1603900d26 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/util/GraphGeneratorsSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/util/GraphGeneratorsSuite.scala @@ -48,10 +48,14 @@ class GraphGeneratorsSuite extends FunSuite with LocalSparkContext { val edges10_round1 = GraphGenerators.generateRandomEdges(src, numEdges10, maxVertexId, seed=12345) val edges10_round2 = GraphGenerators.generateRandomEdges(src, numEdges10, maxVertexId, seed=12345) - assert(edges10_round1 == edges10_round2) + assert(edges10_round1.zip(edges10_round2).forall { case(e1, e2) => + e1.srcId == e2.srcId && e1.dstId == e2.dstId && e1.attr == e2.attr + }) val edges10_round3 = GraphGenerators.generateRandomEdges(src, numEdges10, maxVertexId, seed=3467) - assert(edges10_round1 != edges10_round3) + assert(!edges10_round1.zip(edges10_round3).forall { case(e1, e2) => + e1.srcId == e2.srcId && e1.dstId == e2.dstId && e1.attr == e2.attr + }) } test("GraphGenerators.sampleLogNormal") { @@ -82,10 +86,20 @@ class GraphGeneratorsSuite extends FunSuite with LocalSparkContext { val graph_round1 = GraphGenerators.logNormalGraph(sc, numVertices100, mu=mu, sigma=sigma, seed=12345) val graph_round2 = GraphGenerators.logNormalGraph(sc, numVertices100, mu=mu, sigma=sigma, seed=12345) - assert(graph_round1.edges == graph_round2.edges) + val graph_round1_edges = graph_round1.edges.collect() + val graph_round2_edges = graph_round2.edges.collect() + + assert(graph_round1_edges.zip(graph_round2_edges).forall { case(e1, e2) => + e1.srcId == e2.srcId && e1.dstId == e2.dstId && e1.attr == e2.attr + }) val graph_round3 = GraphGenerators.logNormalGraph(sc, numVertices100, mu=mu, sigma=sigma, seed=567) - assert(graph_round1.edges != graph_round3.edges) + + val graph_round3_edges = graph_round3.edges.collect() + + assert(!graph_round1_edges.zip(graph_round3_edges).forall { case(e1, e2) => + e1.srcId == e2.srcId && e1.dstId == e2.dstId && e1.attr == e2.attr + }) } }