diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index 10dd8cc4d572a..8b3c6871a7b39 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -17,17 +17,16 @@ package org.apache.spark -import org.apache.spark.util.ResetSystemProperties import org.scalatest.FunSuite import org.apache.hadoop.io.BytesWritable -class SparkContextSuite extends FunSuite with ResetSystemProperties with LocalSparkContext { +class SparkContextSuite extends FunSuite with LocalSparkContext { test("Only one SparkContext may be active at a time") { // Regression test for SPARK-4180 - System.setProperty("spark.driver.allowMultipleContexts", "false") val conf = new SparkConf().setAppName("test").setMaster("local") + .set("spark.driver.allowMultipleContexts", "false") sc = new SparkContext(conf) // A SparkContext is already running, so we shouldn't be able to create a second one intercept[SparkException] { new SparkContext(conf) } @@ -37,20 +36,20 @@ class SparkContextSuite extends FunSuite with ResetSystemProperties with LocalSp } test("Can still construct a new SparkContext after failing to construct a previous one") { - System.setProperty("spark.driver.allowMultipleContexts", "false") + val conf = new SparkConf().set("spark.driver.allowMultipleContexts", "false") // This is an invalid configuration (no app name or master URL) intercept[SparkException] { - new SparkContext(new SparkConf()) + new SparkContext(conf) } // Even though those earlier calls failed, we should still be able to create a new context - sc = new SparkContext(new SparkConf().setMaster("local").setAppName("test")) + sc = new SparkContext(conf.setMaster("local").setAppName("test")) } test("Check for multiple SparkContexts can be disabled via undocumented debug option") { - System.setProperty("spark.driver.allowMultipleContexts", "true") var secondSparkContext: SparkContext = null try { val conf = new SparkConf().setAppName("test").setMaster("local") + .set("spark.driver.allowMultipleContexts", "true") sc = new SparkContext(conf) secondSparkContext = new SparkContext(conf) } finally {