Skip to content
Snippets Groups Projects
Commit 4c4f7691 authored by Matei Zaharia's avatar Matei Zaharia
Browse files

Optimize Scala PageRank to use reduceByKey

parent 06e4f2a8
No related branches found
No related tags found
No related merge requests found
......@@ -20,9 +20,10 @@ object SparkPageRank {
System.exit(1)
}
var iters = args(2).toInt
val ctx = new SparkContext(args(0), "PageRank", System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
val ctx = new SparkContext(args(0), "PageRank",
System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
val lines = ctx.textFile(args(1), 1)
val links = lines.map{s =>
val links = lines.map{ s =>
val parts = s.split("\\s+")
(parts(0), parts(1))
}.distinct().groupByKey().cache()
......@@ -33,17 +34,12 @@ object SparkPageRank {
val size = urls.size
urls.map(url => (url, rank / size))
}
ranks = contribs.groupByKey().mapValues{ranks =>
val sumRanks = ranks.foldLeft(0.0)(_ + _)
0.15 + sumRanks * 0.85
}
ranks = contribs.reduceByKey(_ + _).mapValues(0.15 + 0.85 * _)
}
val output = ranks.collect()
output.foreach(tup => println(tup._1 + " has rank: " + tup._2 + "."))
ctx.stop()
System.exit(0)
}
}
......
0% Loading or .
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment