@@ -23,14 +23,12 @@ import org.scalatest.Matchers
2323import org .scalatest .time .{Millis , Span }
2424
2525import org .apache .spark .storage .{RDDBlockId , StorageLevel }
26- import org .apache .spark .util .ResetSystemProperties
2726
2827class NotSerializableClass
2928class NotSerializableExn (val notSer : NotSerializableClass ) extends Throwable () {}
3029
3130
32- class DistributedSuite extends FunSuite with Matchers with ResetSystemProperties
33- with LocalSparkContext {
31+ class DistributedSuite extends FunSuite with Matchers with LocalSparkContext {
3432
3533 val clusterUrl = " local-cluster[2,1,512]"
3634
@@ -79,8 +77,8 @@ class DistributedSuite extends FunSuite with Matchers with ResetSystemProperties
7977 }
8078
8179 test(" groupByKey where map output sizes exceed maxMbInFlight" ) {
82- System .setProperty (" spark.reducer.maxMbInFlight" , " 1" )
83- sc = new SparkContext (clusterUrl, " test" )
80+ val conf = new SparkConf ().set (" spark.reducer.maxMbInFlight" , " 1" )
81+ sc = new SparkContext (clusterUrl, " test" , conf )
8482 // This data should be around 20 MB, so even with 4 mappers and 2 reducers, each map output
8583 // file should be about 2.5 MB
8684 val pairs = sc.parallelize(1 to 2000 , 4 ).map(x => (x % 16 , new Array [Byte ](10000 )))
@@ -214,8 +212,8 @@ class DistributedSuite extends FunSuite with Matchers with ResetSystemProperties
214212 }
215213
216214 test(" compute when only some partitions fit in memory" ) {
217- System .setProperty (" spark.storage.memoryFraction" , " 0.01" )
218- sc = new SparkContext (clusterUrl, " test" )
215+ val conf = new SparkConf ().set (" spark.storage.memoryFraction" , " 0.01" )
216+ sc = new SparkContext (clusterUrl, " test" , conf )
219217 // data will be 4 million * 4 bytes = 16 MB in size, but our memoryFraction set the cache
220218 // to only 5 MB (0.01 of 512 MB), so not all of it will fit in memory; we use 20 partitions
221219 // to make sure that *some* of them do fit though
0 commit comments