[SPARK-6753] Clone SparkConf in ShuffleSuite tests

Prior to this change, the unit test for SPARK-3426 did not clone the
original SparkConf, which meant that that test did not use the options
set by suites that subclass ShuffleSuite.scala. This commit fixes that
problem.

JoshRosen would be great if you could take a look at this, since you wrote this
test originally.

Author: Kay Ousterhout <kayousterhout@gmail.com>

Closes #5401 from kayousterhout/SPARK-6753 and squashes the following commits:

368c540 [Kay Ousterhout] [SPARK-6753] Clone SparkConf in ShuffleSuite tests
This commit is contained in:
Kay Ousterhout 2015-04-08 10:26:45 -07:00 committed by Josh Rosen
parent f7e21dd1ec
commit 9d44ddce1d

View file

@ -242,14 +242,14 @@ abstract class ShuffleSuite extends FunSuite with Matchers with LocalSparkContex
shuffleSpillCompress <- Set(true, false);
shuffleCompress <- Set(true, false)
) {
val conf = new SparkConf()
val myConf = conf.clone()
.setAppName("test")
.setMaster("local")
.set("spark.shuffle.spill.compress", shuffleSpillCompress.toString)
.set("spark.shuffle.compress", shuffleCompress.toString)
.set("spark.shuffle.memoryFraction", "0.001")
resetSparkContext()
sc = new SparkContext(conf)
sc = new SparkContext(myConf)
try {
sc.parallelize(0 until 100000).map(i => (i / 4, i)).groupByKey().collect()
} catch {