aboutsummaryrefslogtreecommitdiff
path: root/core/src/test
diff options
context:
space:
mode:
authorStephen Haberman <stephen@exigencecorp.com>2013-03-23 11:45:45 -0500
committerStephen Haberman <stephen@exigencecorp.com>2013-03-23 11:45:45 -0500
commit4ca273edc4b19894675e96fb62b3afb04ce77a37 (patch)
tree4e0c0e38def464f18106cd24fc2eed90ef123bc5 /core/src/test
parent00170eb0b9c30c1ab4f470c694b434385b8cbb7a (diff)
parentb8949cab889da4ba0f613a17b2eef52a32476410 (diff)
downloadspark-4ca273edc4b19894675e96fb62b3afb04ce77a37.tar.gz
spark-4ca273edc4b19894675e96fb62b3afb04ce77a37.tar.bz2
spark-4ca273edc4b19894675e96fb62b3afb04ce77a37.zip
Merge branch 'master' into shufflecoalesce
Conflicts: core/src/test/scala/spark/RDDSuite.scala
Diffstat (limited to 'core/src/test')
-rw-r--r--core/src/test/scala/spark/RDDSuite.scala92
1 files changed, 91 insertions, 1 deletions
diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala
index bcbb472f6c..a6178867bc 100644
--- a/core/src/test/scala/spark/RDDSuite.scala
+++ b/core/src/test/scala/spark/RDDSuite.scala
@@ -3,7 +3,7 @@ package spark
import scala.collection.mutable.HashMap
import org.scalatest.FunSuite
import spark.SparkContext._
-import spark.rdd.{CoalescedRDD, PartitionPruningRDD, ShuffledRDD}
+import spark.rdd.{CoalescedRDD, CoGroupedRDD, PartitionPruningRDD, ShuffledRDD}
class RDDSuite extends FunSuite with LocalSparkContext {
@@ -123,6 +123,36 @@ class RDDSuite extends FunSuite with LocalSparkContext {
assert(rdd.collect().toList === List(1, 2, 3, 4))
}
+ test("cogrouped RDDs") {
+ sc = new SparkContext("local", "test")
+ val rdd1 = sc.makeRDD(Array((1, "one"), (1, "another one"), (2, "two"), (3, "three")), 2)
+ val rdd2 = sc.makeRDD(Array((1, "one1"), (1, "another one1"), (2, "two1")), 2)
+
+ // Use cogroup function
+ val cogrouped = rdd1.cogroup(rdd2).collectAsMap()
+ assert(cogrouped(1) === (Seq("one", "another one"), Seq("one1", "another one1")))
+ assert(cogrouped(2) === (Seq("two"), Seq("two1")))
+ assert(cogrouped(3) === (Seq("three"), Seq()))
+
+ // Construct CoGroupedRDD directly, with map side combine enabled
+ val cogrouped1 = new CoGroupedRDD[Int](
+ Seq(rdd1.asInstanceOf[RDD[(Int, Any)]], rdd2.asInstanceOf[RDD[(Int, Any)]]),
+ new HashPartitioner(3),
+ true).collectAsMap()
+ assert(cogrouped1(1).toSeq === Seq(Seq("one", "another one"), Seq("one1", "another one1")))
+ assert(cogrouped1(2).toSeq === Seq(Seq("two"), Seq("two1")))
+ assert(cogrouped1(3).toSeq === Seq(Seq("three"), Seq()))
+
+ // Construct CoGroupedRDD directly, with map side combine disabled
+ val cogrouped2 = new CoGroupedRDD[Int](
+ Seq(rdd1.asInstanceOf[RDD[(Int, Any)]], rdd2.asInstanceOf[RDD[(Int, Any)]]),
+ new HashPartitioner(3),
+ false).collectAsMap()
+ assert(cogrouped2(1).toSeq === Seq(Seq("one", "another one"), Seq("one1", "another one1")))
+ assert(cogrouped2(2).toSeq === Seq(Seq("two"), Seq("two1")))
+ assert(cogrouped2(3).toSeq === Seq(Seq("three"), Seq()))
+ }
+
test("coalesced RDDs") {
sc = new SparkContext("local", "test")
val data = sc.parallelize(1 to 10, 10)
@@ -182,4 +212,64 @@ class RDDSuite extends FunSuite with LocalSparkContext {
assert(prunedData.size === 1)
assert(prunedData(0) === 10)
}
+
+ test("mapWith") {
+ import java.util.Random
+ sc = new SparkContext("local", "test")
+ val ones = sc.makeRDD(Array(1, 1, 1, 1, 1, 1), 2)
+ val randoms = ones.mapWith(
+ (index: Int) => new Random(index + 42))
+ {(t: Int, prng: Random) => prng.nextDouble * t}.collect()
+ val prn42_3 = {
+ val prng42 = new Random(42)
+ prng42.nextDouble(); prng42.nextDouble(); prng42.nextDouble()
+ }
+ val prn43_3 = {
+ val prng43 = new Random(43)
+ prng43.nextDouble(); prng43.nextDouble(); prng43.nextDouble()
+ }
+ assert(randoms(2) === prn42_3)
+ assert(randoms(5) === prn43_3)
+ }
+
+ test("flatMapWith") {
+ import java.util.Random
+ sc = new SparkContext("local", "test")
+ val ones = sc.makeRDD(Array(1, 1, 1, 1, 1, 1), 2)
+ val randoms = ones.flatMapWith(
+ (index: Int) => new Random(index + 42))
+ {(t: Int, prng: Random) =>
+ val random = prng.nextDouble()
+ Seq(random * t, random * t * 10)}.
+ collect()
+ val prn42_3 = {
+ val prng42 = new Random(42)
+ prng42.nextDouble(); prng42.nextDouble(); prng42.nextDouble()
+ }
+ val prn43_3 = {
+ val prng43 = new Random(43)
+ prng43.nextDouble(); prng43.nextDouble(); prng43.nextDouble()
+ }
+ assert(randoms(5) === prn42_3 * 10)
+ assert(randoms(11) === prn43_3 * 10)
+ }
+
+ test("filterWith") {
+ import java.util.Random
+ sc = new SparkContext("local", "test")
+ val ints = sc.makeRDD(Array(1, 2, 3, 4, 5, 6), 2)
+ val sample = ints.filterWith(
+ (index: Int) => new Random(index + 42))
+ {(t: Int, prng: Random) => prng.nextInt(3) == 0}.
+ collect()
+ val checkSample = {
+ val prng42 = new Random(42)
+ val prng43 = new Random(43)
+ Array(1, 2, 3, 4, 5, 6).filter{i =>
+ if (i < 4) 0 == prng42.nextInt(3)
+ else 0 == prng43.nextInt(3)}
+ }
+ assert(sample.size === checkSample.size)
+ for (i <- 0 until sample.size) assert(sample(i) === checkSample(i))
+ }
}