Skip to content

Commit

Permalink
Combine tests into single test of all combinations
Browse files Browse the repository at this point in the history
  • Loading branch information
JoshRosen committed Oct 22, 2014
1 parent 76ca65e commit 91e7e40
Showing 1 changed file with 22 additions and 20 deletions.
42 changes: 22 additions & 20 deletions core/src/test/scala/org/apache/spark/ShuffleSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -243,26 +243,28 @@ abstract class ShuffleSuite extends FunSuite with Matchers with LocalSparkContex
assert(thrown.getMessage.toLowerCase.contains("serializable"))
}

test("shuffle with shuffle.spill.compress=true, shuffle.compress=false (SPARK-3426)") {
val conf = new SparkConf()
.setAppName("test")
.setMaster("local")
.set("spark.shuffle.spill.compress", "true")
.set("spark.shuffle.compress", "false")
.set("spark.shuffle.memoryFraction", "0.001")
sc = new SparkContext(conf)
sc.parallelize(0 until 100000).map(i => (i/4, i)).groupByKey().collect()
}

test("shuffle with shuffle.spill.compress=false, shuffle.compress=true (SPARK-3426)") {
val conf = new SparkConf()
.setAppName("test")
.setMaster("local")
.set("spark.shuffle.spill.compress", "false")
.set("spark.shuffle.compress", "true")
.set("spark.shuffle.memoryFraction", "0.001")
sc = new SparkContext(conf)
sc.parallelize(0 until 100000).map(i => (i/4, i)).groupByKey().collect()
test("shuffle with different compression settings (SPARK-3426)") {
for (
shuffleSpillCompress <- Set(true, false);
shuffleCompress <- Set(true, false)
) {
val conf = new SparkConf()
.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)
try {
sc.parallelize(0 until 100000).map(i => (i / 4, i)).groupByKey().collect()
} catch {
case e: Exception =>
val errMsg = s"Failed with spark.shuffle.spill.compress=$shuffleSpillCompress," +
s" spark.shuffle.compress=$shuffleCompress"
throw new Exception(errMsg, e)
}
}
}
}

Expand Down

0 comments on commit 91e7e40

Please sign in to comment.