Skip to content

Commit 352ed6b

Browse files
committed
[SPARK-1010] Clean up uses of System.setProperty in unit tests
Several of our tests call System.setProperty (or test code which implicitly sets system properties) and don't always reset/clear the modified properties, which can create ordering dependencies between tests and cause hard-to-diagnose failures. This patch removes most uses of System.setProperty from our tests, since in most cases we can use SparkConf to set these configurations (there are a few exceptions, including the tests of SparkConf itself). For the cases where we continue to use System.setProperty, this patch introduces a `ResetSystemProperties` ScalaTest mixin class which snapshots the system properties before individual tests and to automatically restores them on test completion / failure. See the block comment at the top of the ResetSystemProperties class for more details. Author: Josh Rosen <[email protected]> Closes #3739 from JoshRosen/cleanup-system-properties-in-tests and squashes the following commits: 0236d66 [Josh Rosen] Replace setProperty uses in two example programs / tools 3888fe3 [Josh Rosen] Remove setProperty use in LocalJavaStreamingContext 4f4031d [Josh Rosen] Add note on why SparkSubmitSuite needs ResetSystemProperties 4742a5b [Josh Rosen] Clarify ResetSystemProperties trait inheritance ordering. 0eaf0b6 [Josh Rosen] Remove setProperty call in TaskResultGetterSuite. 7a3d224 [Josh Rosen] Fix trait ordering 3fdb554 [Josh Rosen] Remove setProperty call in TaskSchedulerImplSuite bee20df [Josh Rosen] Remove setProperty calls in SparkContextSchedulerCreationSuite 655587c [Josh Rosen] Remove setProperty calls in JobCancellationSuite 3f2f955 [Josh Rosen] Remove System.setProperty calls in DistributedSuite cfe9cce [Josh Rosen] Remove use of system properties in SparkContextSuite 8783ab0 [Josh Rosen] Remove TestUtils.setSystemProperty, since it is subsumed by the ResetSystemProperties trait. 633a84a [Josh Rosen] Remove use of system properties in FileServerSuite 25bfce2 [Josh Rosen] Use ResetSystemProperties in UtilsSuite 1d1aa5a [Josh Rosen] Use ResetSystemProperties in SizeEstimatorSuite dd9492b [Josh Rosen] Use ResetSystemProperties in AkkaUtilsSuite b0daff2 [Josh Rosen] Use ResetSystemProperties in BlockManagerSuite e9ded62 [Josh Rosen] Use ResetSystemProperties in TaskSchedulerImplSuite 5b3cb54 [Josh Rosen] Use ResetSystemProperties in SparkListenerSuite 0995c4b [Josh Rosen] Use ResetSystemProperties in SparkContextSchedulerCreationSuite c83ded8 [Josh Rosen] Use ResetSystemProperties in SparkConfSuite 51aa870 [Josh Rosen] Use withSystemProperty in ShuffleSuite 60a63a1 [Josh Rosen] Use ResetSystemProperties in JobCancellationSuite 14a92e4 [Josh Rosen] Use withSystemProperty in FileServerSuite 628f46c [Josh Rosen] Use ResetSystemProperties in DistributedSuite 9e3e0dd [Josh Rosen] Add ResetSystemProperties test fixture mixin; use it in SparkSubmitSuite. 4dcea38 [Josh Rosen] Move withSystemProperty to TestUtils class.
1 parent 035bac8 commit 352ed6b

File tree

23 files changed

+216
-232
lines changed

23 files changed

+216
-232
lines changed

core/src/test/scala/org/apache/spark/DistributedSuite.scala

Lines changed: 5 additions & 16 deletions
Original file line numberDiff line numberDiff line change
@@ -17,7 +17,6 @@
1717

1818
package org.apache.spark
1919

20-
import org.scalatest.BeforeAndAfter
2120
import org.scalatest.FunSuite
2221
import org.scalatest.concurrent.Timeouts._
2322
import org.scalatest.Matchers
@@ -29,16 +28,10 @@ class NotSerializableClass
2928
class NotSerializableExn(val notSer: NotSerializableClass) extends Throwable() {}
3029

3130

32-
class DistributedSuite extends FunSuite with Matchers with BeforeAndAfter
33-
with LocalSparkContext {
31+
class DistributedSuite extends FunSuite with Matchers with LocalSparkContext {
3432

3533
val clusterUrl = "local-cluster[2,1,512]"
3634

37-
after {
38-
System.clearProperty("spark.reducer.maxMbInFlight")
39-
System.clearProperty("spark.storage.memoryFraction")
40-
}
41-
4235
test("task throws not serializable exception") {
4336
// Ensures that executors do not crash when an exn is not serializable. If executors crash,
4437
// this test will hang. Correct behavior is that executors don't crash but fail tasks
@@ -84,15 +77,14 @@ class DistributedSuite extends FunSuite with Matchers with BeforeAndAfter
8477
}
8578

8679
test("groupByKey where map output sizes exceed maxMbInFlight") {
87-
System.setProperty("spark.reducer.maxMbInFlight", "1")
88-
sc = new SparkContext(clusterUrl, "test")
80+
val conf = new SparkConf().set("spark.reducer.maxMbInFlight", "1")
81+
sc = new SparkContext(clusterUrl, "test", conf)
8982
// This data should be around 20 MB, so even with 4 mappers and 2 reducers, each map output
9083
// file should be about 2.5 MB
9184
val pairs = sc.parallelize(1 to 2000, 4).map(x => (x % 16, new Array[Byte](10000)))
9285
val groups = pairs.groupByKey(2).map(x => (x._1, x._2.size)).collect()
9386
assert(groups.length === 16)
9487
assert(groups.map(_._2).sum === 2000)
95-
// Note that spark.reducer.maxMbInFlight will be cleared in the test suite's after{} block
9688
}
9789

9890
test("accumulators") {
@@ -210,28 +202,25 @@ class DistributedSuite extends FunSuite with Matchers with BeforeAndAfter
210202
}
211203

212204
test("compute without caching when no partitions fit in memory") {
213-
System.setProperty("spark.storage.memoryFraction", "0.0001")
214205
sc = new SparkContext(clusterUrl, "test")
215206
// data will be 4 million * 4 bytes = 16 MB in size, but our memoryFraction set the cache
216207
// to only 50 KB (0.0001 of 512 MB), so no partitions should fit in memory
217208
val data = sc.parallelize(1 to 4000000, 2).persist(StorageLevel.MEMORY_ONLY_SER)
218209
assert(data.count() === 4000000)
219210
assert(data.count() === 4000000)
220211
assert(data.count() === 4000000)
221-
System.clearProperty("spark.storage.memoryFraction")
222212
}
223213

224214
test("compute when only some partitions fit in memory") {
225-
System.setProperty("spark.storage.memoryFraction", "0.01")
226-
sc = new SparkContext(clusterUrl, "test")
215+
val conf = new SparkConf().set("spark.storage.memoryFraction", "0.01")
216+
sc = new SparkContext(clusterUrl, "test", conf)
227217
// data will be 4 million * 4 bytes = 16 MB in size, but our memoryFraction set the cache
228218
// to only 5 MB (0.01 of 512 MB), so not all of it will fit in memory; we use 20 partitions
229219
// to make sure that *some* of them do fit though
230220
val data = sc.parallelize(1 to 4000000, 20).persist(StorageLevel.MEMORY_ONLY_SER)
231221
assert(data.count() === 4000000)
232222
assert(data.count() === 4000000)
233223
assert(data.count() === 4000000)
234-
System.clearProperty("spark.storage.memoryFraction")
235224
}
236225

237226
test("passing environment variables to cluster") {

core/src/test/scala/org/apache/spark/FileServerSuite.scala

Lines changed: 8 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -31,10 +31,11 @@ class FileServerSuite extends FunSuite with LocalSparkContext {
3131
@transient var tmpFile: File = _
3232
@transient var tmpJarUrl: String = _
3333

34+
def newConf: SparkConf = new SparkConf(loadDefaults = false).set("spark.authenticate", "false")
35+
3436
override def beforeEach() {
3537
super.beforeEach()
3638
resetSparkContext()
37-
System.setProperty("spark.authenticate", "false")
3839
}
3940

4041
override def beforeAll() {
@@ -52,7 +53,6 @@ class FileServerSuite extends FunSuite with LocalSparkContext {
5253
val jarFile = new File(testTempDir, "test.jar")
5354
val jarStream = new FileOutputStream(jarFile)
5455
val jar = new JarOutputStream(jarStream, new java.util.jar.Manifest())
55-
System.setProperty("spark.authenticate", "false")
5656

5757
val jarEntry = new JarEntry(textFile.getName)
5858
jar.putNextEntry(jarEntry)
@@ -74,7 +74,7 @@ class FileServerSuite extends FunSuite with LocalSparkContext {
7474
}
7575

7676
test("Distributing files locally") {
77-
sc = new SparkContext("local[4]", "test")
77+
sc = new SparkContext("local[4]", "test", newConf)
7878
sc.addFile(tmpFile.toString)
7979
val testData = Array((1,1), (1,1), (2,1), (3,5), (2,2), (3,0))
8080
val result = sc.parallelize(testData).reduceByKey {
@@ -108,7 +108,7 @@ class FileServerSuite extends FunSuite with LocalSparkContext {
108108

109109
test("Distributing files locally using URL as input") {
110110
// addFile("file:///....")
111-
sc = new SparkContext("local[4]", "test")
111+
sc = new SparkContext("local[4]", "test", newConf)
112112
sc.addFile(new File(tmpFile.toString).toURI.toString)
113113
val testData = Array((1,1), (1,1), (2,1), (3,5), (2,2), (3,0))
114114
val result = sc.parallelize(testData).reduceByKey {
@@ -122,7 +122,7 @@ class FileServerSuite extends FunSuite with LocalSparkContext {
122122
}
123123

124124
test ("Dynamically adding JARS locally") {
125-
sc = new SparkContext("local[4]", "test")
125+
sc = new SparkContext("local[4]", "test", newConf)
126126
sc.addJar(tmpJarUrl)
127127
val testData = Array((1, 1))
128128
sc.parallelize(testData).foreach { x =>
@@ -133,7 +133,7 @@ class FileServerSuite extends FunSuite with LocalSparkContext {
133133
}
134134

135135
test("Distributing files on a standalone cluster") {
136-
sc = new SparkContext("local-cluster[1,1,512]", "test")
136+
sc = new SparkContext("local-cluster[1,1,512]", "test", newConf)
137137
sc.addFile(tmpFile.toString)
138138
val testData = Array((1,1), (1,1), (2,1), (3,5), (2,2), (3,0))
139139
val result = sc.parallelize(testData).reduceByKey {
@@ -147,7 +147,7 @@ class FileServerSuite extends FunSuite with LocalSparkContext {
147147
}
148148

149149
test ("Dynamically adding JARS on a standalone cluster") {
150-
sc = new SparkContext("local-cluster[1,1,512]", "test")
150+
sc = new SparkContext("local-cluster[1,1,512]", "test", newConf)
151151
sc.addJar(tmpJarUrl)
152152
val testData = Array((1,1))
153153
sc.parallelize(testData).foreach { x =>
@@ -158,7 +158,7 @@ class FileServerSuite extends FunSuite with LocalSparkContext {
158158
}
159159

160160
test ("Dynamically adding JARS on a standalone cluster using local: URL") {
161-
sc = new SparkContext("local-cluster[1,1,512]", "test")
161+
sc = new SparkContext("local-cluster[1,1,512]", "test", newConf)
162162
sc.addJar(tmpJarUrl.replace("file", "local"))
163163
val testData = Array((1,1))
164164
sc.parallelize(testData).foreach { x =>

core/src/test/scala/org/apache/spark/JobCancellationSuite.scala

Lines changed: 10 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -40,43 +40,42 @@ class JobCancellationSuite extends FunSuite with Matchers with BeforeAndAfter
4040
override def afterEach() {
4141
super.afterEach()
4242
resetSparkContext()
43-
System.clearProperty("spark.scheduler.mode")
4443
}
4544

4645
test("local mode, FIFO scheduler") {
47-
System.setProperty("spark.scheduler.mode", "FIFO")
48-
sc = new SparkContext("local[2]", "test")
46+
val conf = new SparkConf().set("spark.scheduler.mode", "FIFO")
47+
sc = new SparkContext("local[2]", "test", conf)
4948
testCount()
5049
testTake()
5150
// Make sure we can still launch tasks.
5251
assert(sc.parallelize(1 to 10, 2).count === 10)
5352
}
5453

5554
test("local mode, fair scheduler") {
56-
System.setProperty("spark.scheduler.mode", "FAIR")
55+
val conf = new SparkConf().set("spark.scheduler.mode", "FAIR")
5756
val xmlPath = getClass.getClassLoader.getResource("fairscheduler.xml").getFile()
58-
System.setProperty("spark.scheduler.allocation.file", xmlPath)
59-
sc = new SparkContext("local[2]", "test")
57+
conf.set("spark.scheduler.allocation.file", xmlPath)
58+
sc = new SparkContext("local[2]", "test", conf)
6059
testCount()
6160
testTake()
6261
// Make sure we can still launch tasks.
6362
assert(sc.parallelize(1 to 10, 2).count === 10)
6463
}
6564

6665
test("cluster mode, FIFO scheduler") {
67-
System.setProperty("spark.scheduler.mode", "FIFO")
68-
sc = new SparkContext("local-cluster[2,1,512]", "test")
66+
val conf = new SparkConf().set("spark.scheduler.mode", "FIFO")
67+
sc = new SparkContext("local-cluster[2,1,512]", "test", conf)
6968
testCount()
7069
testTake()
7170
// Make sure we can still launch tasks.
7271
assert(sc.parallelize(1 to 10, 2).count === 10)
7372
}
7473

7574
test("cluster mode, fair scheduler") {
76-
System.setProperty("spark.scheduler.mode", "FAIR")
75+
val conf = new SparkConf().set("spark.scheduler.mode", "FAIR")
7776
val xmlPath = getClass.getClassLoader.getResource("fairscheduler.xml").getFile()
78-
System.setProperty("spark.scheduler.allocation.file", xmlPath)
79-
sc = new SparkContext("local-cluster[2,1,512]", "test")
77+
conf.set("spark.scheduler.allocation.file", xmlPath)
78+
sc = new SparkContext("local-cluster[2,1,512]", "test", conf)
8079
testCount()
8180
testTake()
8281
// Make sure we can still launch tasks.

core/src/test/scala/org/apache/spark/ShuffleSuite.scala

Lines changed: 9 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -35,19 +35,15 @@ abstract class ShuffleSuite extends FunSuite with Matchers with LocalSparkContex
3535
conf.set("spark.test.noStageRetry", "true")
3636

3737
test("groupByKey without compression") {
38-
try {
39-
System.setProperty("spark.shuffle.compress", "false")
40-
sc = new SparkContext("local", "test", conf)
41-
val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1)), 4)
42-
val groups = pairs.groupByKey(4).collect()
43-
assert(groups.size === 2)
44-
val valuesFor1 = groups.find(_._1 == 1).get._2
45-
assert(valuesFor1.toList.sorted === List(1, 2, 3))
46-
val valuesFor2 = groups.find(_._1 == 2).get._2
47-
assert(valuesFor2.toList.sorted === List(1))
48-
} finally {
49-
System.setProperty("spark.shuffle.compress", "true")
50-
}
38+
val myConf = conf.clone().set("spark.shuffle.compress", "false")
39+
sc = new SparkContext("local", "test", myConf)
40+
val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1)), 4)
41+
val groups = pairs.groupByKey(4).collect()
42+
assert(groups.size === 2)
43+
val valuesFor1 = groups.find(_._1 == 1).get._2
44+
assert(valuesFor1.toList.sorted === List(1, 2, 3))
45+
val valuesFor2 = groups.find(_._1 == 2).get._2
46+
assert(valuesFor2.toList.sorted === List(1))
5147
}
5248

5349
test("shuffle non-zero block size") {

core/src/test/scala/org/apache/spark/SparkConfSuite.scala

Lines changed: 19 additions & 32 deletions
Original file line numberDiff line numberDiff line change
@@ -19,27 +19,20 @@ package org.apache.spark
1919

2020
import org.scalatest.FunSuite
2121
import org.apache.spark.serializer.{KryoRegistrator, KryoSerializer}
22+
import org.apache.spark.util.ResetSystemProperties
2223
import com.esotericsoftware.kryo.Kryo
2324

24-
class SparkConfSuite extends FunSuite with LocalSparkContext {
25+
class SparkConfSuite extends FunSuite with LocalSparkContext with ResetSystemProperties {
2526
test("loading from system properties") {
26-
try {
27-
System.setProperty("spark.test.testProperty", "2")
28-
val conf = new SparkConf()
29-
assert(conf.get("spark.test.testProperty") === "2")
30-
} finally {
31-
System.clearProperty("spark.test.testProperty")
32-
}
27+
System.setProperty("spark.test.testProperty", "2")
28+
val conf = new SparkConf()
29+
assert(conf.get("spark.test.testProperty") === "2")
3330
}
3431

3532
test("initializing without loading defaults") {
36-
try {
37-
System.setProperty("spark.test.testProperty", "2")
38-
val conf = new SparkConf(false)
39-
assert(!conf.contains("spark.test.testProperty"))
40-
} finally {
41-
System.clearProperty("spark.test.testProperty")
42-
}
33+
System.setProperty("spark.test.testProperty", "2")
34+
val conf = new SparkConf(false)
35+
assert(!conf.contains("spark.test.testProperty"))
4336
}
4437

4538
test("named set methods") {
@@ -117,23 +110,17 @@ class SparkConfSuite extends FunSuite with LocalSparkContext {
117110

118111
test("nested property names") {
119112
// This wasn't supported by some external conf parsing libraries
120-
try {
121-
System.setProperty("spark.test.a", "a")
122-
System.setProperty("spark.test.a.b", "a.b")
123-
System.setProperty("spark.test.a.b.c", "a.b.c")
124-
val conf = new SparkConf()
125-
assert(conf.get("spark.test.a") === "a")
126-
assert(conf.get("spark.test.a.b") === "a.b")
127-
assert(conf.get("spark.test.a.b.c") === "a.b.c")
128-
conf.set("spark.test.a.b", "A.B")
129-
assert(conf.get("spark.test.a") === "a")
130-
assert(conf.get("spark.test.a.b") === "A.B")
131-
assert(conf.get("spark.test.a.b.c") === "a.b.c")
132-
} finally {
133-
System.clearProperty("spark.test.a")
134-
System.clearProperty("spark.test.a.b")
135-
System.clearProperty("spark.test.a.b.c")
136-
}
113+
System.setProperty("spark.test.a", "a")
114+
System.setProperty("spark.test.a.b", "a.b")
115+
System.setProperty("spark.test.a.b.c", "a.b.c")
116+
val conf = new SparkConf()
117+
assert(conf.get("spark.test.a") === "a")
118+
assert(conf.get("spark.test.a.b") === "a.b")
119+
assert(conf.get("spark.test.a.b.c") === "a.b.c")
120+
conf.set("spark.test.a.b", "A.B")
121+
assert(conf.get("spark.test.a") === "a")
122+
assert(conf.get("spark.test.a.b") === "A.B")
123+
assert(conf.get("spark.test.a.b.c") === "a.b.c")
137124
}
138125

139126
test("register kryo classes through registerKryoClasses") {

core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala

Lines changed: 13 additions & 18 deletions
Original file line numberDiff line numberDiff line change
@@ -27,10 +27,13 @@ import org.apache.spark.scheduler.local.LocalBackend
2727
class SparkContextSchedulerCreationSuite
2828
extends FunSuite with LocalSparkContext with PrivateMethodTester with Logging {
2929

30-
def createTaskScheduler(master: String): TaskSchedulerImpl = {
30+
def createTaskScheduler(master: String): TaskSchedulerImpl =
31+
createTaskScheduler(master, new SparkConf())
32+
33+
def createTaskScheduler(master: String, conf: SparkConf): TaskSchedulerImpl = {
3134
// Create local SparkContext to setup a SparkEnv. We don't actually want to start() the
3235
// real schedulers, so we don't want to create a full SparkContext with the desired scheduler.
33-
sc = new SparkContext("local", "test")
36+
sc = new SparkContext("local", "test", conf)
3437
val createTaskSchedulerMethod =
3538
PrivateMethod[Tuple2[SchedulerBackend, TaskScheduler]]('createTaskScheduler)
3639
val (_, sched) = SparkContext invokePrivate createTaskSchedulerMethod(sc, master)
@@ -102,19 +105,13 @@ class SparkContextSchedulerCreationSuite
102105
}
103106

104107
test("local-default-parallelism") {
105-
val defaultParallelism = System.getProperty("spark.default.parallelism")
106-
System.setProperty("spark.default.parallelism", "16")
107-
val sched = createTaskScheduler("local")
108+
val conf = new SparkConf().set("spark.default.parallelism", "16")
109+
val sched = createTaskScheduler("local", conf)
108110

109111
sched.backend match {
110112
case s: LocalBackend => assert(s.defaultParallelism() === 16)
111113
case _ => fail()
112114
}
113-
114-
Option(defaultParallelism) match {
115-
case Some(v) => System.setProperty("spark.default.parallelism", v)
116-
case _ => System.clearProperty("spark.default.parallelism")
117-
}
118115
}
119116

120117
test("simr") {
@@ -155,9 +152,10 @@ class SparkContextSchedulerCreationSuite
155152
testYarn("yarn-client", "org.apache.spark.scheduler.cluster.YarnClientClusterScheduler")
156153
}
157154

158-
def testMesos(master: String, expectedClass: Class[_]) {
155+
def testMesos(master: String, expectedClass: Class[_], coarse: Boolean) {
156+
val conf = new SparkConf().set("spark.mesos.coarse", coarse.toString)
159157
try {
160-
val sched = createTaskScheduler(master)
158+
val sched = createTaskScheduler(master, conf)
161159
assert(sched.backend.getClass === expectedClass)
162160
} catch {
163161
case e: UnsatisfiedLinkError =>
@@ -168,17 +166,14 @@ class SparkContextSchedulerCreationSuite
168166
}
169167

170168
test("mesos fine-grained") {
171-
System.setProperty("spark.mesos.coarse", "false")
172-
testMesos("mesos://localhost:1234", classOf[MesosSchedulerBackend])
169+
testMesos("mesos://localhost:1234", classOf[MesosSchedulerBackend], coarse = false)
173170
}
174171

175172
test("mesos coarse-grained") {
176-
System.setProperty("spark.mesos.coarse", "true")
177-
testMesos("mesos://localhost:1234", classOf[CoarseMesosSchedulerBackend])
173+
testMesos("mesos://localhost:1234", classOf[CoarseMesosSchedulerBackend], coarse = true)
178174
}
179175

180176
test("mesos with zookeeper") {
181-
System.setProperty("spark.mesos.coarse", "false")
182-
testMesos("zk://localhost:1234,localhost:2345", classOf[MesosSchedulerBackend])
177+
testMesos("zk://localhost:1234,localhost:2345", classOf[MesosSchedulerBackend], coarse = false)
183178
}
184179
}

0 commit comments

Comments
 (0)