aboutsummaryrefslogtreecommitdiff
path: root/core/src/test/scala
diff options
context:
space:
mode:
authorReynold Xin <reynoldx@gmail.com>2013-08-17 21:43:29 -0700
committerReynold Xin <reynoldx@gmail.com>2013-08-17 21:43:29 -0700
commit2c00ea3efc7d9a23af8ba11352460294e1865942 (patch)
treea72c858cf44b872fd6d14a31e015d3e02c627727 /core/src/test/scala
parent0e84fee76b529089fb52f15151202e9a7b847ed5 (diff)
downloadspark-2c00ea3efc7d9a23af8ba11352460294e1865942.tar.gz
spark-2c00ea3efc7d9a23af8ba11352460294e1865942.tar.bz2
spark-2c00ea3efc7d9a23af8ba11352460294e1865942.zip
Moved shuffle serializer setting from a constructor parameter to a setSerializer method in various RDDs that involve shuffle operations.
Diffstat (limited to 'core/src/test/scala')
-rw-r--r--core/src/test/scala/spark/ShuffleSuite.scala19
1 files changed, 6 insertions, 13 deletions
diff --git a/core/src/test/scala/spark/ShuffleSuite.scala b/core/src/test/scala/spark/ShuffleSuite.scala
index 752e4b85e6..c686b8cc5a 100644
--- a/core/src/test/scala/spark/ShuffleSuite.scala
+++ b/core/src/test/scala/spark/ShuffleSuite.scala
@@ -17,17 +17,8 @@
package spark
-import scala.collection.mutable.ArrayBuffer
-import scala.collection.mutable.HashSet
-
import org.scalatest.FunSuite
import org.scalatest.matchers.ShouldMatchers
-import org.scalatest.prop.Checkers
-import org.scalacheck.Arbitrary._
-import org.scalacheck.Gen
-import org.scalacheck.Prop._
-
-import com.google.common.io.Files
import spark.rdd.ShuffledRDD
import spark.SparkContext._
@@ -59,8 +50,8 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext {
}
// If the Kryo serializer is not used correctly, the shuffle would fail because the
// default Java serializer cannot handle the non serializable class.
- val c = new ShuffledRDD(b, new HashPartitioner(NUM_BLOCKS),
- classOf[spark.KryoSerializer].getName)
+ val c = new ShuffledRDD(b, new HashPartitioner(NUM_BLOCKS))
+ .setSerializer(classOf[spark.KryoSerializer].getName)
val shuffleId = c.dependencies.head.asInstanceOf[ShuffleDependency[Int, Int]].shuffleId
assert(c.count === 10)
@@ -81,7 +72,8 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext {
}
// If the Kryo serializer is not used correctly, the shuffle would fail because the
// default Java serializer cannot handle the non serializable class.
- val c = new ShuffledRDD(b, new HashPartitioner(3), classOf[spark.KryoSerializer].getName)
+ val c = new ShuffledRDD(b, new HashPartitioner(3))
+ .setSerializer(classOf[spark.KryoSerializer].getName)
assert(c.count === 10)
}
@@ -96,7 +88,8 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext {
// NOTE: The default Java serializer doesn't create zero-sized blocks.
// So, use Kryo
- val c = new ShuffledRDD(b, new HashPartitioner(10), classOf[spark.KryoSerializer].getName)
+ val c = new ShuffledRDD(b, new HashPartitioner(10))
+ .setSerializer(classOf[spark.KryoSerializer].getName)
val shuffleId = c.dependencies.head.asInstanceOf[ShuffleDependency[Int, Int]].shuffleId
assert(c.count === 4)