|
17 | 17 |
|
18 | 18 | package org.apache.spark |
19 | 19 |
|
| 20 | +import org.apache.spark.util.ResetSystemProperties |
20 | 21 | import org.scalatest.FunSuite |
21 | 22 |
|
22 | 23 | import org.apache.hadoop.io.BytesWritable |
23 | 24 |
|
24 | | -class SparkContextSuite extends FunSuite with LocalSparkContext { |
| 25 | +class SparkContextSuite extends FunSuite with ResetSystemProperties with LocalSparkContext { |
25 | 26 |
|
26 | 27 | test("Only one SparkContext may be active at a time") { |
27 | 28 | // Regression test for SPARK-4180 |
28 | | - TestUtils.withSystemProperty("spark.driver.allowMultipleContexts", "false") { |
29 | | - val conf = new SparkConf().setAppName("test").setMaster("local") |
30 | | - sc = new SparkContext(conf) |
31 | | - // A SparkContext is already running, so we shouldn't be able to create a second one |
32 | | - intercept[SparkException] { new SparkContext(conf) } |
33 | | - // After stopping the running context, we should be able to create a new one |
34 | | - resetSparkContext() |
35 | | - sc = new SparkContext(conf) |
36 | | - } |
| 29 | + System.setProperty("spark.driver.allowMultipleContexts", "false") |
| 30 | + val conf = new SparkConf().setAppName("test").setMaster("local") |
| 31 | + sc = new SparkContext(conf) |
| 32 | + // A SparkContext is already running, so we shouldn't be able to create a second one |
| 33 | + intercept[SparkException] { new SparkContext(conf) } |
| 34 | + // After stopping the running context, we should be able to create a new one |
| 35 | + resetSparkContext() |
| 36 | + sc = new SparkContext(conf) |
37 | 37 | } |
38 | 38 |
|
39 | 39 | test("Can still construct a new SparkContext after failing to construct a previous one") { |
40 | | - TestUtils.withSystemProperty("spark.driver.allowMultipleContexts", "false") { |
41 | | - // This is an invalid configuration (no app name or master URL) |
42 | | - intercept[SparkException] { |
43 | | - new SparkContext(new SparkConf()) |
44 | | - } |
45 | | - // Even though those earlier calls failed, we should still be able to create a new context |
46 | | - sc = new SparkContext(new SparkConf().setMaster("local").setAppName("test")) |
| 40 | + System.setProperty("spark.driver.allowMultipleContexts", "false") |
| 41 | + // This is an invalid configuration (no app name or master URL) |
| 42 | + intercept[SparkException] { |
| 43 | + new SparkContext(new SparkConf()) |
47 | 44 | } |
| 45 | + // Even though those earlier calls failed, we should still be able to create a new context |
| 46 | + sc = new SparkContext(new SparkConf().setMaster("local").setAppName("test")) |
48 | 47 | } |
49 | 48 |
|
50 | 49 | test("Check for multiple SparkContexts can be disabled via undocumented debug option") { |
51 | | - TestUtils.withSystemProperty("spark.driver.allowMultipleContexts", "true") { |
52 | | - var secondSparkContext: SparkContext = null |
53 | | - try { |
54 | | - val conf = new SparkConf().setAppName("test").setMaster("local") |
55 | | - sc = new SparkContext(conf) |
56 | | - secondSparkContext = new SparkContext(conf) |
57 | | - } finally { |
58 | | - Option(secondSparkContext).foreach(_.stop()) |
59 | | - } |
| 50 | + System.setProperty("spark.driver.allowMultipleContexts", "true") |
| 51 | + var secondSparkContext: SparkContext = null |
| 52 | + try { |
| 53 | + val conf = new SparkConf().setAppName("test").setMaster("local") |
| 54 | + sc = new SparkContext(conf) |
| 55 | + secondSparkContext = new SparkContext(conf) |
| 56 | + } finally { |
| 57 | + Option(secondSparkContext).foreach(_.stop()) |
60 | 58 | } |
61 | 59 | } |
62 | 60 |
|
|
0 commit comments