aboutsummaryrefslogtreecommitdiff
path: root/examples
diff options
context:
space:
mode:
authorNick Pentreath <nick.pentreath@gmail.com>2013-08-07 16:38:52 +0200
committerNick Pentreath <nick.pentreath@gmail.com>2013-08-07 16:38:52 +0200
commitcce758b8938afb24c6d61a02f697201c41801fb6 (patch)
tree76590ceb47fbc22ddc431fd3e20817769ecc11c0 /examples
parent3c8478e1fbe3ec85a83f2822ad8a8d4cca580487 (diff)
downloadspark-cce758b8938afb24c6d61a02f697201c41801fb6.tar.gz
spark-cce758b8938afb24c6d61a02f697201c41801fb6.tar.bz2
spark-cce758b8938afb24c6d61a02f697201c41801fb6.zip
Adding Scala version of PageRank example
Diffstat (limited to 'examples')
-rw-r--r--examples/src/main/scala/spark/examples/SparkPageRank.scala51
1 files changed, 51 insertions, 0 deletions
diff --git a/examples/src/main/scala/spark/examples/SparkPageRank.scala b/examples/src/main/scala/spark/examples/SparkPageRank.scala
new file mode 100644
index 0000000000..6d2be34956
--- /dev/null
+++ b/examples/src/main/scala/spark/examples/SparkPageRank.scala
@@ -0,0 +1,51 @@
+package spark.examples
+
+import spark.SparkContext._
+import spark.SparkContext
+
+
+/**
+ * Computes the PageRank of URLs from an input file. Input file should
+ * be in format of:
+ * URL neighbor URL
+ * URL neighbor URL
+ * URL neighbor URL
+ * ...
+ * where URL and their neighbors are separated by space(s).
+ */
+object SparkPageRank {
+
+ def main(args: Array[String]) {
+ if (args.length < 3) {
+ System.err.println("Usage: PageRank <master> <file> <number_of_iterations>")
+ 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 lines = ctx.textFile(args(1), 1)
+ val links = lines.map(s => {
+ val parts = s.split("\\s+")
+ (parts(0), parts(1))
+ }).distinct().groupByKey().cache()
+ var ranks = links.mapValues(v => 1.0)
+
+ for (i <- 1 to iters) {
+ val contribs = links.join(ranks).values.flatMap{ case (urls, rank) => {
+ 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
+ })
+ }
+
+ val output = ranks.collect()
+ output.foreach(tup => println(tup._1 + " has rank: " + tup._2 + "."))
+
+ ctx.stop()
+ System.exit(0)
+ }
+}
+