diff options
author | Ankur Dave <ankurdave@gmail.com> | 2014-01-10 15:23:32 -0800 |
---|---|---|
committer | Ankur Dave <ankurdave@gmail.com> | 2014-01-10 15:23:32 -0800 |
commit | 11dd35c28be9542d00ddb75660873525b2e22b43 (patch) | |
tree | cbb2d7693a5aec331803dd37176c36daf8b93e94 /graphx | |
parent | 9e48af6dba5817b7ac35a5bfa5ffe8825960c4c8 (diff) | |
download | spark-11dd35c28be9542d00ddb75660873525b2e22b43.tar.gz spark-11dd35c28be9542d00ddb75660873525b2e22b43.tar.bz2 spark-11dd35c28be9542d00ddb75660873525b2e22b43.zip |
Clean up GraphGenerators
Diffstat (limited to 'graphx')
-rw-r--r-- | graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala | 75 |
1 files changed, 9 insertions, 66 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 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: + * <pre> * * dst -> * (x,y) *************** _ @@ -191,6 +133,7 @@ object GraphGenerators { * | c | d | | * | | | | * *************** - + * </pre> * * 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) = { |