aboutsummaryrefslogtreecommitdiff
path: root/examples
diff options
context:
space:
mode:
authorHenry Saputra <henry@platfora.com>2013-11-19 16:57:57 -0800
committerHenry Saputra <henry@platfora.com>2013-11-19 16:57:57 -0800
commit43dfac51324778d3f6df3de2cbd6fb223251774c (patch)
tree6b56d4800b7ebc6affbac1883f59ae7ec346d8f8 /examples
parent10be58f251b5e883295bd46383c0a9758555f8fc (diff)
parentf568912f85f58ae152db90f199c1f3a002f270c1 (diff)
downloadspark-43dfac51324778d3f6df3de2cbd6fb223251774c.tar.gz
spark-43dfac51324778d3f6df3de2cbd6fb223251774c.tar.bz2
spark-43dfac51324778d3f6df3de2cbd6fb223251774c.zip
Merge branch 'master' into removesemicolonscala
Diffstat (limited to 'examples')
-rw-r--r--examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala10
-rw-r--r--examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala15
2 files changed, 14 insertions, 11 deletions
diff --git a/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala b/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala
index 529709c2f9..a119980992 100644
--- a/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala
@@ -32,13 +32,13 @@ object BroadcastTest {
System.setProperty("spark.broadcast.factory", "org.apache.spark.broadcast." + bcName + "BroadcastFactory")
System.setProperty("spark.broadcast.blockSize", blockSize)
- val sc = new SparkContext(args(0), "Broadcast Test 2",
+ val sc = new SparkContext(args(0), "Broadcast Test",
System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
val slices = if (args.length > 1) args(1).toInt else 2
val num = if (args.length > 2) args(2).toInt else 1000000
- var arr1 = new Array[Int](num)
+ val arr1 = new Array[Int](num)
for (i <- 0 until arr1.length) {
arr1(i) = i
}
@@ -48,9 +48,9 @@ object BroadcastTest {
println("===========")
val startTime = System.nanoTime
val barr1 = sc.broadcast(arr1)
- sc.parallelize(1 to 10, slices).foreach {
- i => println(barr1.value.size)
- }
+ val observedSizes = sc.parallelize(1 to 10, slices).map(_ => barr1.value.size)
+ // Collect the small RDD so we can print the observed sizes locally.
+ observedSizes.collect().foreach(i => println(i))
println("Iteration %d took %.0f milliseconds".format(i, (System.nanoTime - startTime) / 1E6))
}
diff --git a/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala b/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala
index f79f0142b8..e1afc29f9a 100644
--- a/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala
@@ -18,35 +18,38 @@
package org.apache.spark.examples
import org.apache.spark.SparkContext
+import org.apache.spark.rdd.RDD
object MultiBroadcastTest {
def main(args: Array[String]) {
if (args.length == 0) {
- System.err.println("Usage: BroadcastTest <master> [<slices>] [numElem]")
+ System.err.println("Usage: MultiBroadcastTest <master> [<slices>] [numElem]")
System.exit(1)
}
- val sc = new SparkContext(args(0), "Broadcast Test",
+ val sc = new SparkContext(args(0), "Multi-Broadcast Test",
System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
val slices = if (args.length > 1) args(1).toInt else 2
val num = if (args.length > 2) args(2).toInt else 1000000
- var arr1 = new Array[Int](num)
+ val arr1 = new Array[Int](num)
for (i <- 0 until arr1.length) {
arr1(i) = i
}
- var arr2 = new Array[Int](num)
+ val arr2 = new Array[Int](num)
for (i <- 0 until arr2.length) {
arr2(i) = i
}
val barr1 = sc.broadcast(arr1)
val barr2 = sc.broadcast(arr2)
- sc.parallelize(1 to 10, slices).foreach {
- i => println(barr1.value.size + barr2.value.size)
+ val observedSizes: RDD[(Int, Int)] = sc.parallelize(1 to 10, slices).map { _ =>
+ (barr1.value.size, barr2.value.size)
}
+ // Collect the small RDD so we can print the observed sizes locally.
+ observedSizes.collect().foreach(i => println(i))
System.exit(0)
}