aboutsummaryrefslogtreecommitdiff
path: root/examples
diff options
context:
space:
mode:
authorReza Zadeh <rizlar@gmail.com>2014-01-13 23:52:34 -0800
committerReza Zadeh <rizlar@gmail.com>2014-01-13 23:52:34 -0800
commit845e568fada0550e632e7381748c5a9ebbe53e16 (patch)
tree3a4fa34894df649b5ef429cd794b73cf4b3e99b1 /examples
parentf324d5355514b1c7ae85019b476046bb64b5593e (diff)
parentfdaabdc67387524ffb84354f87985f48bd31cf60 (diff)
downloadspark-845e568fada0550e632e7381748c5a9ebbe53e16.tar.gz
spark-845e568fada0550e632e7381748c5a9ebbe53e16.tar.bz2
spark-845e568fada0550e632e7381748c5a9ebbe53e16.zip
Merge remote-tracking branch 'upstream/master' into sparsesvd
Diffstat (limited to 'examples')
-rw-r--r--examples/src/main/scala/org/apache/spark/examples/LocalALS.scala8
-rw-r--r--examples/src/main/scala/org/apache/spark/examples/LocalFileLR.scala2
-rw-r--r--examples/src/main/scala/org/apache/spark/examples/LocalKMeans.scala2
-rw-r--r--examples/src/main/scala/org/apache/spark/examples/SparkALS.scala6
-rw-r--r--examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala2
-rw-r--r--examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala12
-rw-r--r--examples/src/main/scala/org/apache/spark/examples/graphx/LiveJournalPageRank.scala49
-rw-r--r--examples/src/main/scala/org/apache/spark/streaming/examples/RawNetworkGrep.scala2
-rw-r--r--examples/src/main/scala/org/apache/spark/streaming/examples/RecoverableNetworkWordCount.scala2
-rw-r--r--examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala4
-rw-r--r--examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala4
-rw-r--r--examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala4
-rw-r--r--examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala2
-rw-r--r--examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala2
14 files changed, 75 insertions, 26 deletions
diff --git a/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala b/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala
index 83db8b9e26..c8ecbb8e41 100644
--- a/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala
@@ -43,7 +43,7 @@ object LocalALS {
def generateR(): DoubleMatrix2D = {
val mh = factory2D.random(M, F)
val uh = factory2D.random(U, F)
- return algebra.mult(mh, algebra.transpose(uh))
+ algebra.mult(mh, algebra.transpose(uh))
}
def rmse(targetR: DoubleMatrix2D, ms: Array[DoubleMatrix1D],
@@ -56,7 +56,7 @@ object LocalALS {
//println("R: " + r)
blas.daxpy(-1, targetR, r)
val sumSqs = r.aggregate(Functions.plus, Functions.square)
- return sqrt(sumSqs / (M * U))
+ sqrt(sumSqs / (M * U))
}
def updateMovie(i: Int, m: DoubleMatrix1D, us: Array[DoubleMatrix1D],
@@ -80,7 +80,7 @@ object LocalALS {
val ch = new CholeskyDecomposition(XtX)
val Xty2D = factory2D.make(Xty.toArray, F)
val solved2D = ch.solve(Xty2D)
- return solved2D.viewColumn(0)
+ solved2D.viewColumn(0)
}
def updateUser(j: Int, u: DoubleMatrix1D, ms: Array[DoubleMatrix1D],
@@ -104,7 +104,7 @@ object LocalALS {
val ch = new CholeskyDecomposition(XtX)
val Xty2D = factory2D.make(Xty.toArray, F)
val solved2D = ch.solve(Xty2D)
- return solved2D.viewColumn(0)
+ solved2D.viewColumn(0)
}
def main(args: Array[String]) {
diff --git a/examples/src/main/scala/org/apache/spark/examples/LocalFileLR.scala b/examples/src/main/scala/org/apache/spark/examples/LocalFileLR.scala
index fb130ea198..9ab5f5a486 100644
--- a/examples/src/main/scala/org/apache/spark/examples/LocalFileLR.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/LocalFileLR.scala
@@ -28,7 +28,7 @@ object LocalFileLR {
def parsePoint(line: String): DataPoint = {
val nums = line.split(' ').map(_.toDouble)
- return DataPoint(new Vector(nums.slice(1, D+1)), nums(0))
+ DataPoint(new Vector(nums.slice(1, D+1)), nums(0))
}
def main(args: Array[String]) {
diff --git a/examples/src/main/scala/org/apache/spark/examples/LocalKMeans.scala b/examples/src/main/scala/org/apache/spark/examples/LocalKMeans.scala
index f90ea35cd4..a730464ea1 100644
--- a/examples/src/main/scala/org/apache/spark/examples/LocalKMeans.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/LocalKMeans.scala
@@ -55,7 +55,7 @@ object LocalKMeans {
}
}
- return bestIndex
+ bestIndex
}
def main(args: Array[String]) {
diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala b/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala
index 30c86d83e6..17bafc2218 100644
--- a/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala
@@ -44,7 +44,7 @@ object SparkALS {
def generateR(): DoubleMatrix2D = {
val mh = factory2D.random(M, F)
val uh = factory2D.random(U, F)
- return algebra.mult(mh, algebra.transpose(uh))
+ algebra.mult(mh, algebra.transpose(uh))
}
def rmse(targetR: DoubleMatrix2D, ms: Array[DoubleMatrix1D],
@@ -57,7 +57,7 @@ object SparkALS {
//println("R: " + r)
blas.daxpy(-1, targetR, r)
val sumSqs = r.aggregate(Functions.plus, Functions.square)
- return sqrt(sumSqs / (M * U))
+ sqrt(sumSqs / (M * U))
}
def update(i: Int, m: DoubleMatrix1D, us: Array[DoubleMatrix1D],
@@ -83,7 +83,7 @@ object SparkALS {
val ch = new CholeskyDecomposition(XtX)
val Xty2D = factory2D.make(Xty.toArray, F)
val solved2D = ch.solve(Xty2D)
- return solved2D.viewColumn(0)
+ solved2D.viewColumn(0)
}
def main(args: Array[String]) {
diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala
index ff72532db1..39819064ed 100644
--- a/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala
@@ -43,7 +43,7 @@ object SparkHdfsLR {
while (i < D) {
x(i) = tok.nextToken.toDouble; i += 1
}
- return DataPoint(new Vector(x), y)
+ DataPoint(new Vector(x), y)
}
def main(args: Array[String]) {
diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala b/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala
index 8c99025eaa..9fe2465235 100644
--- a/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala
@@ -30,7 +30,7 @@ object SparkKMeans {
val rand = new Random(42)
def parseVector(line: String): Vector = {
- return new Vector(line.split(' ').map(_.toDouble))
+ new Vector(line.split(' ').map(_.toDouble))
}
def closestPoint(p: Vector, centers: Array[Vector]): Int = {
@@ -46,7 +46,7 @@ object SparkKMeans {
}
}
- return bestIndex
+ bestIndex
}
def main(args: Array[String]) {
@@ -61,15 +61,15 @@ object SparkKMeans {
val K = args(2).toInt
val convergeDist = args(3).toDouble
- var kPoints = data.takeSample(false, K, 42).toArray
+ val kPoints = data.takeSample(withReplacement = false, K, 42).toArray
var tempDist = 1.0
while(tempDist > convergeDist) {
- var closest = data.map (p => (closestPoint(p, kPoints), (p, 1)))
+ val closest = data.map (p => (closestPoint(p, kPoints), (p, 1)))
- var pointStats = closest.reduceByKey{case ((x1, y1), (x2, y2)) => (x1 + x2, y1 + y2)}
+ val pointStats = closest.reduceByKey{case ((x1, y1), (x2, y2)) => (x1 + x2, y1 + y2)}
- var newPoints = pointStats.map {pair => (pair._1, pair._2._1 / pair._2._2)}.collectAsMap()
+ val newPoints = pointStats.map {pair => (pair._1, pair._2._1 / pair._2._2)}.collectAsMap()
tempDist = 0.0
for (i <- 0 until K) {
diff --git a/examples/src/main/scala/org/apache/spark/examples/graphx/LiveJournalPageRank.scala b/examples/src/main/scala/org/apache/spark/examples/graphx/LiveJournalPageRank.scala
new file mode 100644
index 0000000000..d58fddff2b
--- /dev/null
+++ b/examples/src/main/scala/org/apache/spark/examples/graphx/LiveJournalPageRank.scala
@@ -0,0 +1,49 @@
+/*
+ * 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.examples.graphx
+
+import org.apache.spark.SparkContext._
+import org.apache.spark._
+import org.apache.spark.graphx._
+import org.apache.spark.graphx.lib.Analytics
+
+/**
+ * Uses GraphX to run PageRank on a LiveJournal social network graph. Download the dataset from
+ * http://snap.stanford.edu/data/soc-LiveJournal1.html.
+ */
+object LiveJournalPageRank {
+ def main(args: Array[String]) {
+ if (args.length < 2) {
+ System.err.println(
+ "Usage: LiveJournalPageRank <master> <edge_list_file>\n" +
+ " [--tol=<tolerance>]\n" +
+ " The tolerance allowed at convergence (smaller => more accurate). Default is " +
+ "0.001.\n" +
+ " [--output=<output_file>]\n" +
+ " If specified, the file to write the ranks to.\n" +
+ " [--numEPart=<num_edge_partitions>]\n" +
+ " The number of partitions for the graph's edge RDD. Default is 4.\n" +
+ " [--partStrategy=RandomVertexCut | EdgePartition1D | EdgePartition2D | " +
+ "CanonicalRandomVertexCut]\n" +
+ " The way edges are assigned to edge partitions. Default is RandomVertexCut.")
+ System.exit(-1)
+ }
+
+ Analytics.main(args.patch(1, List("pagerank"), 0))
+ }
+}
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/RawNetworkGrep.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/RawNetworkGrep.scala
index 3d08d86567..99b79c3949 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/RawNetworkGrep.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/RawNetworkGrep.scala
@@ -58,7 +58,7 @@ object RawNetworkGrep {
val rawStreams = (1 to numStreams).map(_ =>
ssc.rawSocketStream[String](host, port, StorageLevel.MEMORY_ONLY_SER_2)).toArray
val union = ssc.union(rawStreams)
- union.filter(_.contains("the")).count().foreach(r =>
+ union.filter(_.contains("the")).count().foreachRDD(r =>
println("Grep count: " + r.collect().mkString))
ssc.start()
}
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/RecoverableNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/RecoverableNetworkWordCount.scala
index d51e6e9418..8c5d0bd568 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/RecoverableNetworkWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/RecoverableNetworkWordCount.scala
@@ -82,7 +82,7 @@ object RecoverableNetworkWordCount {
val lines = ssc.socketTextStream(ip, port)
val words = lines.flatMap(_.split(" "))
val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _)
- wordCounts.foreach((rdd: RDD[(String, Int)], time: Time) => {
+ wordCounts.foreachRDD((rdd: RDD[(String, Int)], time: Time) => {
val counts = "Counts at time " + time + " " + rdd.collect().mkString("[", ", ", "]")
println(counts)
println("Appending to " + outputFile.getAbsolutePath)
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala
index 80b5a98b14..483c4d3118 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala
@@ -81,7 +81,7 @@ object TwitterAlgebirdCMS {
val exactTopUsers = users.map(id => (id, 1))
.reduceByKey((a, b) => a + b)
- approxTopUsers.foreach(rdd => {
+ approxTopUsers.foreachRDD(rdd => {
if (rdd.count() != 0) {
val partial = rdd.first()
val partialTopK = partial.heavyHitters.map(id =>
@@ -96,7 +96,7 @@ object TwitterAlgebirdCMS {
}
})
- exactTopUsers.foreach(rdd => {
+ exactTopUsers.foreachRDD(rdd => {
if (rdd.count() != 0) {
val partialMap = rdd.collect().toMap
val partialTopK = rdd.map(
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala
index cb2f2c51a0..94c2bf29ac 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala
@@ -67,7 +67,7 @@ object TwitterAlgebirdHLL {
val exactUsers = users.map(id => Set(id)).reduce(_ ++ _)
- approxUsers.foreach(rdd => {
+ approxUsers.foreachRDD(rdd => {
if (rdd.count() != 0) {
val partial = rdd.first()
globalHll += partial
@@ -76,7 +76,7 @@ object TwitterAlgebirdHLL {
}
})
- exactUsers.foreach(rdd => {
+ exactUsers.foreachRDD(rdd => {
if (rdd.count() != 0) {
val partial = rdd.first()
userSet ++= partial
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala
index 16c10feaba..8a70d4a978 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala
@@ -56,13 +56,13 @@ object TwitterPopularTags {
// Print popular hashtags
- topCounts60.foreach(rdd => {
+ topCounts60.foreachRDD(rdd => {
val topList = rdd.take(5)
println("\nPopular topics in last 60 seconds (%s total):".format(rdd.count()))
topList.foreach{case (count, tag) => println("%s (%s tweets)".format(tag, count))}
})
- topCounts10.foreach(rdd => {
+ topCounts10.foreachRDD(rdd => {
val topList = rdd.take(5)
println("\nPopular topics in last 10 seconds (%s total):".format(rdd.count()))
topList.foreach{case (count, tag) => println("%s (%s tweets)".format(tag, count))}
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala
index 4fe57de4a4..a2600989ca 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala
@@ -65,7 +65,7 @@ object PageViewGenerator {
return item
}
}
- return inputMap.take(1).head._1 // Shouldn't get here if probabilities add up to 1.0
+ inputMap.take(1).head._1 // Shouldn't get here if probabilities add up to 1.0
}
def getNextClickEvent() : String = {
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala
index da6b67bcce..bb44bc3d06 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala
@@ -91,7 +91,7 @@ object PageViewStream {
case "popularUsersSeen" =>
// Look for users in our existing dataset and print it out if we have a match
pageViews.map(view => (view.userID, 1))
- .foreach((rdd, time) => rdd.join(userList)
+ .foreachRDD((rdd, time) => rdd.join(userList)
.map(_._2._2)
.take(10)
.foreach(u => println("Saw user %s at time %s".format(u, time))))