Skip to content

Commit

Permalink
Add utility to set system properties in tests.
Browse files Browse the repository at this point in the history
  • Loading branch information
JoshRosen committed Nov 7, 2014
1 parent 4629d5c commit 7ba6db8
Showing 1 changed file with 46 additions and 31 deletions.
77 changes: 46 additions & 31 deletions core/src/test/scala/org/apache/spark/SparkContextSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -23,45 +23,60 @@ import org.apache.hadoop.io.BytesWritable

class SparkContextSuite extends FunSuite with LocalSparkContext {

/** Allows system properties to be changed in tests */
private def withSystemProperty[T](property: String, value: String)(block: => T): T = {
val originalValue = System.getProperty(property)
try {
System.setProperty(property, value)
block
} finally {
if (originalValue == null) {
System.clearProperty(property)
} else {
System.setProperty(property, originalValue)
}
}
}

test("Only one SparkContext may be active at a time") {
// Regression test for SPARK-4180
val conf = new SparkConf().setAppName("test").setMaster("local")
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) }
// After stopping the running context, we should be able to create a new one
resetSparkContext()
sc = new SparkContext(conf)
withSystemProperty("spark.driver.allowMultipleContexts", "false") {
val conf = new SparkConf().setAppName("test").setMaster("local")
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) }
// After stopping the running context, we should be able to create a new one
resetSparkContext()
sc = new SparkContext(conf)
}
}

test("Can still construct a new SparkContext after failing due to missing app name or master") {
val missingMaster = new SparkConf()
val missingAppName = missingMaster.clone.setMaster("local")
val validConf = missingAppName.clone.setAppName("test")
// We shouldn't be able to construct SparkContexts because these are invalid configurations
intercept[SparkException] { new SparkContext(missingMaster) }
intercept[SparkException] { new SparkContext(missingAppName) }
// Even though those earlier calls failed, we should still be able to create a new SparkContext
sc = new SparkContext(validConf)
withSystemProperty("spark.driver.allowMultipleContexts", "false") {
val missingMaster = new SparkConf()
val missingAppName = missingMaster.clone.setMaster("local")
val validConf = missingAppName.clone.setAppName("test")
// We shouldn't be able to construct SparkContexts because these are invalid configurations
intercept[SparkException] {
new SparkContext(missingMaster)
}
intercept[SparkException] {
new SparkContext(missingAppName)
}
// Even though those earlier calls failed, we should still be able to create a new context
sc = new SparkContext(validConf)
}
}

test("Check for multiple SparkContexts can be disabled via undocumented debug option") {
val propertyName = "spark.driver.allowMultipleContexts"
val originalPropertyValue = System.getProperty(propertyName)
var secondSparkContext: SparkContext = null
try {
System.setProperty(propertyName, "true")
val conf = new SparkConf().setAppName("test").setMaster("local")
sc = new SparkContext(conf)
secondSparkContext = new SparkContext(conf)
} finally {
if (secondSparkContext != null) {
secondSparkContext.stop()
}
if (originalPropertyValue != null) {
System.setProperty(propertyName, originalPropertyValue)
} else {
System.clearProperty(propertyName)
withSystemProperty("spark.driver.allowMultipleContexts", "true") {
var secondSparkContext: SparkContext = null
try {
val conf = new SparkConf().setAppName("test").setMaster("local")
sc = new SparkContext(conf)
secondSparkContext = new SparkContext(conf)
} finally {
Option(secondSparkContext).foreach(_.stop())
}
}
}
Expand Down

0 comments on commit 7ba6db8

Please sign in to comment.