Skip to content

Commit 7162d85

Browse files
JoshRosenpwendell
authored andcommitted
[SPARK-4180] [Core] Prevent creation of multiple active SparkContexts
This patch adds error-detection logic to throw an exception when attempting to create multiple active SparkContexts in the same JVM, since this is currently unsupported and has been known to cause confusing behavior (see SPARK-2243 for more details). **The solution implemented here is only a partial fix.** A complete fix would have the following properties: 1. Only one SparkContext may ever be under construction at any given time. 2. Once a SparkContext has been successfully constructed, any subsequent construction attempts should fail until the active SparkContext is stopped. 3. If the SparkContext constructor throws an exception, then all resources created in the constructor should be cleaned up (SPARK-4194). 4. If a user attempts to create a SparkContext but the creation fails, then the user should be able to create new SparkContexts. This PR only provides 2) and 4); we should be able to provide all of these properties, but the correct fix will involve larger changes to SparkContext's construction / initialization, so we'll target it for a different Spark release. ### The correct solution: I think that the correct way to do this would be to move the construction of SparkContext's dependencies into a static method in the SparkContext companion object. Specifically, we could make the default SparkContext constructor `private` and change it to accept a `SparkContextDependencies` object that contains all of SparkContext's dependencies (e.g. DAGScheduler, ContextCleaner, etc.). Secondary constructors could call a method on the SparkContext companion object to create the `SparkContextDependencies` and pass the result to the primary SparkContext constructor. For example: ```scala class SparkContext private (deps: SparkContextDependencies) { def this(conf: SparkConf) { this(SparkContext.getDeps(conf)) } } object SparkContext( private[spark] def getDeps(conf: SparkConf): SparkContextDependencies = synchronized { if (anotherSparkContextIsActive) { throw Exception(...) } var dagScheduler: DAGScheduler = null try { dagScheduler = new DAGScheduler(...) [...] } catch { case e: Exception => Option(dagScheduler).foreach(_.stop()) [...] } SparkContextDependencies(dagScheduler, ....) } } ``` This gives us mutual exclusion and ensures that any resources created during the failed SparkContext initialization are properly cleaned up. This indirection is necessary to maintain binary compatibility. In retrospect, it would have been nice if SparkContext had no private constructors and could only be created through builder / factory methods on its companion object, since this buys us lots of flexibility and makes dependency injection easier. ### Alternative solutions: As an alternative solution, we could refactor SparkContext's primary constructor to perform all object creation in a giant `try-finally` block. Unfortunately, this will require us to turn a bunch of `vals` into `vars` so that they can be assigned from the `try` block. If we still want `vals`, we could wrap each `val` in its own `try` block (since the try block can return a value), but this will lead to extremely messy code and won't guard against the introduction of future code which doesn't properly handle failures. The more complex approach outlined above gives us some nice dependency injection benefits, so I think that might be preferable to a `var`-ification. ### This PR's solution: - At the start of the constructor, check whether some other SparkContext is active; if so, throw an exception. - If another SparkContext might be under construction (or has thrown an exception during construction), allow the new SparkContext to begin construction but log a warning (since resources might have been leaked from a failed creation attempt). - At the end of the SparkContext constructor, check whether some other SparkContext constructor has raced and successfully created an active context. If so, throw an exception. This guarantees that no two SparkContexts will ever be active and exposed to users (since we check at the very end of the constructor). If two threads race to construct SparkContexts, then one of them will win and another will throw an exception. This exception can be turned into a warning by setting `spark.driver.allowMultipleContexts = true`. The exception is disabled in unit tests, since there are some suites (such as Hive) that may require more significant refactoring to clean up their SparkContexts. I've made a few changes to other suites' test fixtures to properly clean up SparkContexts so that the unit test logs contain fewer warnings. Author: Josh Rosen <[email protected]> Closes #3121 from JoshRosen/SPARK-4180 and squashes the following commits: 23c7123 [Josh Rosen] Merge remote-tracking branch 'origin/master' into SPARK-4180 d38251b [Josh Rosen] Address latest round of feedback. c0987d3 [Josh Rosen] Accept boolean instead of SparkConf in methods. 85a424a [Josh Rosen] Incorporate more review feedback. 372d0d3 [Josh Rosen] Merge remote-tracking branch 'origin/master' into SPARK-4180 f5bb78c [Josh Rosen] Update mvn build, too. d809cb4 [Josh Rosen] Improve handling of failed SparkContext creation attempts. 79a7e6f [Josh Rosen] Fix commented out test a1cba65 [Josh Rosen] Merge remote-tracking branch 'origin/master' into SPARK-4180 7ba6db8 [Josh Rosen] Add utility to set system properties in tests. 4629d5c [Josh Rosen] Set spark.driver.allowMultipleContexts=true in tests. ed17e14 [Josh Rosen] Address review feedback; expose hack workaround for existing unit tests. 1c66070 [Josh Rosen] Merge remote-tracking branch 'origin/master' into SPARK-4180 06c5c54 [Josh Rosen] Add / improve SparkContext cleanup in streaming BasicOperationsSuite d0437eb [Josh Rosen] StreamingContext.stop() should stop SparkContext even if StreamingContext has not been started yet. c4d35a2 [Josh Rosen] Log long form of creation site to aid debugging. 918e878 [Josh Rosen] Document "one SparkContext per JVM" limitation. afaa7e3 [Josh Rosen] [SPARK-4180] Prevent creations of multiple active SparkContexts. (cherry picked from commit 0f3ceb5) Signed-off-by: Patrick Wendell <[email protected]>
1 parent 202627f commit 7162d85

File tree

9 files changed

+347
-126
lines changed

9 files changed

+347
-126
lines changed

core/src/main/scala/org/apache/spark/SparkContext.scala

Lines changed: 145 additions & 22 deletions
Original file line numberDiff line numberDiff line change
@@ -58,12 +58,26 @@ import org.apache.spark.util._
5858
* Main entry point for Spark functionality. A SparkContext represents the connection to a Spark
5959
* cluster, and can be used to create RDDs, accumulators and broadcast variables on that cluster.
6060
*
61+
* Only one SparkContext may be active per JVM. You must `stop()` the active SparkContext before
62+
* creating a new one. This limitation may eventually be removed; see SPARK-2243 for more details.
63+
*
6164
* @param config a Spark Config object describing the application configuration. Any settings in
6265
* this config overrides the default configs as well as system properties.
6366
*/
64-
6567
class SparkContext(config: SparkConf) extends Logging {
6668

69+
// The call site where this SparkContext was constructed.
70+
private val creationSite: CallSite = Utils.getCallSite()
71+
72+
// If true, log warnings instead of throwing exceptions when multiple SparkContexts are active
73+
private val allowMultipleContexts: Boolean =
74+
config.getBoolean("spark.driver.allowMultipleContexts", false)
75+
76+
// In order to prevent multiple SparkContexts from being active at the same time, mark this
77+
// context as having started construction.
78+
// NOTE: this must be placed at the beginning of the SparkContext constructor.
79+
SparkContext.markPartiallyConstructed(this, allowMultipleContexts)
80+
6781
// This is used only by YARN for now, but should be relevant to other cluster types (Mesos,
6882
// etc) too. This is typically generated from InputFormatInfo.computePreferredLocations. It
6983
// contains a map from hostname to a list of input format splits on the host.
@@ -1166,27 +1180,30 @@ class SparkContext(config: SparkConf) extends Logging {
11661180

11671181
/** Shut down the SparkContext. */
11681182
def stop() {
1169-
postApplicationEnd()
1170-
ui.foreach(_.stop())
1171-
// Do this only if not stopped already - best case effort.
1172-
// prevent NPE if stopped more than once.
1173-
val dagSchedulerCopy = dagScheduler
1174-
dagScheduler = null
1175-
if (dagSchedulerCopy != null) {
1176-
env.metricsSystem.report()
1177-
metadataCleaner.cancel()
1178-
env.actorSystem.stop(heartbeatReceiver)
1179-
cleaner.foreach(_.stop())
1180-
dagSchedulerCopy.stop()
1181-
taskScheduler = null
1182-
// TODO: Cache.stop()?
1183-
env.stop()
1184-
SparkEnv.set(null)
1185-
listenerBus.stop()
1186-
eventLogger.foreach(_.stop())
1187-
logInfo("Successfully stopped SparkContext")
1188-
} else {
1189-
logInfo("SparkContext already stopped")
1183+
SparkContext.SPARK_CONTEXT_CONSTRUCTOR_LOCK.synchronized {
1184+
postApplicationEnd()
1185+
ui.foreach(_.stop())
1186+
// Do this only if not stopped already - best case effort.
1187+
// prevent NPE if stopped more than once.
1188+
val dagSchedulerCopy = dagScheduler
1189+
dagScheduler = null
1190+
if (dagSchedulerCopy != null) {
1191+
env.metricsSystem.report()
1192+
metadataCleaner.cancel()
1193+
env.actorSystem.stop(heartbeatReceiver)
1194+
cleaner.foreach(_.stop())
1195+
dagSchedulerCopy.stop()
1196+
taskScheduler = null
1197+
// TODO: Cache.stop()?
1198+
env.stop()
1199+
SparkEnv.set(null)
1200+
listenerBus.stop()
1201+
eventLogger.foreach(_.stop())
1202+
logInfo("Successfully stopped SparkContext")
1203+
SparkContext.clearActiveContext()
1204+
} else {
1205+
logInfo("SparkContext already stopped")
1206+
}
11901207
}
11911208
}
11921209

@@ -1475,6 +1492,11 @@ class SparkContext(config: SparkConf) extends Logging {
14751492
private[spark] def cleanup(cleanupTime: Long) {
14761493
persistentRdds.clearOldValues(cleanupTime)
14771494
}
1495+
1496+
// In order to prevent multiple SparkContexts from being active at the same time, mark this
1497+
// context as having finished construction.
1498+
// NOTE: this must be placed at the end of the SparkContext constructor.
1499+
SparkContext.setActiveContext(this, allowMultipleContexts)
14781500
}
14791501

14801502
/**
@@ -1483,6 +1505,107 @@ class SparkContext(config: SparkConf) extends Logging {
14831505
*/
14841506
object SparkContext extends Logging {
14851507

1508+
/**
1509+
* Lock that guards access to global variables that track SparkContext construction.
1510+
*/
1511+
private val SPARK_CONTEXT_CONSTRUCTOR_LOCK = new Object()
1512+
1513+
/**
1514+
* The active, fully-constructed SparkContext. If no SparkContext is active, then this is `None`.
1515+
*
1516+
* Access to this field is guarded by SPARK_CONTEXT_CONSTRUCTOR_LOCK
1517+
*/
1518+
private var activeContext: Option[SparkContext] = None
1519+
1520+
/**
1521+
* Points to a partially-constructed SparkContext if some thread is in the SparkContext
1522+
* constructor, or `None` if no SparkContext is being constructed.
1523+
*
1524+
* Access to this field is guarded by SPARK_CONTEXT_CONSTRUCTOR_LOCK
1525+
*/
1526+
private var contextBeingConstructed: Option[SparkContext] = None
1527+
1528+
/**
1529+
* Called to ensure that no other SparkContext is running in this JVM.
1530+
*
1531+
* Throws an exception if a running context is detected and logs a warning if another thread is
1532+
* constructing a SparkContext. This warning is necessary because the current locking scheme
1533+
* prevents us from reliably distinguishing between cases where another context is being
1534+
* constructed and cases where another constructor threw an exception.
1535+
*/
1536+
private def assertNoOtherContextIsRunning(
1537+
sc: SparkContext,
1538+
allowMultipleContexts: Boolean): Unit = {
1539+
SPARK_CONTEXT_CONSTRUCTOR_LOCK.synchronized {
1540+
contextBeingConstructed.foreach { otherContext =>
1541+
if (otherContext ne sc) { // checks for reference equality
1542+
// Since otherContext might point to a partially-constructed context, guard against
1543+
// its creationSite field being null:
1544+
val otherContextCreationSite =
1545+
Option(otherContext.creationSite).map(_.longForm).getOrElse("unknown location")
1546+
val warnMsg = "Another SparkContext is being constructed (or threw an exception in its" +
1547+
" constructor). This may indicate an error, since only one SparkContext may be" +
1548+
" running in this JVM (see SPARK-2243)." +
1549+
s" The other SparkContext was created at:\n$otherContextCreationSite"
1550+
logWarning(warnMsg)
1551+
}
1552+
1553+
activeContext.foreach { ctx =>
1554+
val errMsg = "Only one SparkContext may be running in this JVM (see SPARK-2243)." +
1555+
" To ignore this error, set spark.driver.allowMultipleContexts = true. " +
1556+
s"The currently running SparkContext was created at:\n${ctx.creationSite.longForm}"
1557+
val exception = new SparkException(errMsg)
1558+
if (allowMultipleContexts) {
1559+
logWarning("Multiple running SparkContexts detected in the same JVM!", exception)
1560+
} else {
1561+
throw exception
1562+
}
1563+
}
1564+
}
1565+
}
1566+
}
1567+
1568+
/**
1569+
* Called at the beginning of the SparkContext constructor to ensure that no SparkContext is
1570+
* running. Throws an exception if a running context is detected and logs a warning if another
1571+
* thread is constructing a SparkContext. This warning is necessary because the current locking
1572+
* scheme prevents us from reliably distinguishing between cases where another context is being
1573+
* constructed and cases where another constructor threw an exception.
1574+
*/
1575+
private[spark] def markPartiallyConstructed(
1576+
sc: SparkContext,
1577+
allowMultipleContexts: Boolean): Unit = {
1578+
SPARK_CONTEXT_CONSTRUCTOR_LOCK.synchronized {
1579+
assertNoOtherContextIsRunning(sc, allowMultipleContexts)
1580+
contextBeingConstructed = Some(sc)
1581+
}
1582+
}
1583+
1584+
/**
1585+
* Called at the end of the SparkContext constructor to ensure that no other SparkContext has
1586+
* raced with this constructor and started.
1587+
*/
1588+
private[spark] def setActiveContext(
1589+
sc: SparkContext,
1590+
allowMultipleContexts: Boolean): Unit = {
1591+
SPARK_CONTEXT_CONSTRUCTOR_LOCK.synchronized {
1592+
assertNoOtherContextIsRunning(sc, allowMultipleContexts)
1593+
contextBeingConstructed = None
1594+
activeContext = Some(sc)
1595+
}
1596+
}
1597+
1598+
/**
1599+
* Clears the active SparkContext metadata. This is called by `SparkContext#stop()`. It's
1600+
* also called in unit tests to prevent a flood of warnings from test suites that don't / can't
1601+
* properly clean up their SparkContexts.
1602+
*/
1603+
private[spark] def clearActiveContext(): Unit = {
1604+
SPARK_CONTEXT_CONSTRUCTOR_LOCK.synchronized {
1605+
activeContext = None
1606+
}
1607+
}
1608+
14861609
private[spark] val SPARK_JOB_DESCRIPTION = "spark.job.description"
14871610

14881611
private[spark] val SPARK_JOB_GROUP_ID = "spark.jobGroup.id"

core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -42,6 +42,9 @@ import org.apache.spark.rdd.{EmptyRDD, HadoopRDD, NewHadoopRDD, RDD}
4242
/**
4343
* A Java-friendly version of [[org.apache.spark.SparkContext]] that returns
4444
* [[org.apache.spark.api.java.JavaRDD]]s and works with Java collections instead of Scala ones.
45+
*
46+
* Only one SparkContext may be active per JVM. You must `stop()` the active SparkContext before
47+
* creating a new one. This limitation may eventually be removed; see SPARK-2243 for more details.
4548
*/
4649
class JavaSparkContext(val sc: SparkContext)
4750
extends JavaSparkContextVarargsWorkaround with Closeable {

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

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -37,20 +37,24 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext {
3737
.set("spark.dynamicAllocation.enabled", "true")
3838
intercept[SparkException] { new SparkContext(conf) }
3939
SparkEnv.get.stop() // cleanup the created environment
40+
SparkContext.clearActiveContext()
4041

4142
// Only min
4243
val conf1 = conf.clone().set("spark.dynamicAllocation.minExecutors", "1")
4344
intercept[SparkException] { new SparkContext(conf1) }
4445
SparkEnv.get.stop()
46+
SparkContext.clearActiveContext()
4547

4648
// Only max
4749
val conf2 = conf.clone().set("spark.dynamicAllocation.maxExecutors", "2")
4850
intercept[SparkException] { new SparkContext(conf2) }
4951
SparkEnv.get.stop()
52+
SparkContext.clearActiveContext()
5053

5154
// Both min and max, but min > max
5255
intercept[SparkException] { createSparkContext(2, 1) }
5356
SparkEnv.get.stop()
57+
SparkContext.clearActiveContext()
5458

5559
// Both min and max, and min == max
5660
val sc1 = createSparkContext(1, 1)

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

Lines changed: 55 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -21,9 +21,62 @@ import org.scalatest.FunSuite
2121

2222
import org.apache.hadoop.io.BytesWritable
2323

24-
class SparkContextSuite extends FunSuite {
25-
//Regression test for SPARK-3121
24+
class SparkContextSuite extends FunSuite with LocalSparkContext {
25+
26+
/** Allows system properties to be changed in tests */
27+
private def withSystemProperty[T](property: String, value: String)(block: => T): T = {
28+
val originalValue = System.getProperty(property)
29+
try {
30+
System.setProperty(property, value)
31+
block
32+
} finally {
33+
if (originalValue == null) {
34+
System.clearProperty(property)
35+
} else {
36+
System.setProperty(property, originalValue)
37+
}
38+
}
39+
}
40+
41+
test("Only one SparkContext may be active at a time") {
42+
// Regression test for SPARK-4180
43+
withSystemProperty("spark.driver.allowMultipleContexts", "false") {
44+
val conf = new SparkConf().setAppName("test").setMaster("local")
45+
sc = new SparkContext(conf)
46+
// A SparkContext is already running, so we shouldn't be able to create a second one
47+
intercept[SparkException] { new SparkContext(conf) }
48+
// After stopping the running context, we should be able to create a new one
49+
resetSparkContext()
50+
sc = new SparkContext(conf)
51+
}
52+
}
53+
54+
test("Can still construct a new SparkContext after failing to construct a previous one") {
55+
withSystemProperty("spark.driver.allowMultipleContexts", "false") {
56+
// This is an invalid configuration (no app name or master URL)
57+
intercept[SparkException] {
58+
new SparkContext(new SparkConf())
59+
}
60+
// Even though those earlier calls failed, we should still be able to create a new context
61+
sc = new SparkContext(new SparkConf().setMaster("local").setAppName("test"))
62+
}
63+
}
64+
65+
test("Check for multiple SparkContexts can be disabled via undocumented debug option") {
66+
withSystemProperty("spark.driver.allowMultipleContexts", "true") {
67+
var secondSparkContext: SparkContext = null
68+
try {
69+
val conf = new SparkConf().setAppName("test").setMaster("local")
70+
sc = new SparkContext(conf)
71+
secondSparkContext = new SparkContext(conf)
72+
} finally {
73+
Option(secondSparkContext).foreach(_.stop())
74+
}
75+
}
76+
}
77+
2678
test("BytesWritable implicit conversion is correct") {
79+
// Regression test for SPARK-3121
2780
val bytesWritable = new BytesWritable()
2881
val inputArray = (1 to 10).map(_.toByte).toArray
2982
bytesWritable.set(inputArray, 0, 10)

docs/programming-guide.md

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -117,6 +117,8 @@ The first thing a Spark program must do is to create a [SparkContext](api/scala/
117117
how to access a cluster. To create a `SparkContext` you first need to build a [SparkConf](api/scala/index.html#org.apache.spark.SparkConf) object
118118
that contains information about your application.
119119

120+
Only one SparkContext may be active per JVM. You must `stop()` the active SparkContext before creating a new one.
121+
120122
{% highlight scala %}
121123
val conf = new SparkConf().setAppName(appName).setMaster(master)
122124
new SparkContext(conf)

pom.xml

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -978,6 +978,7 @@
978978
<spark.testing>1</spark.testing>
979979
<spark.ui.enabled>false</spark.ui.enabled>
980980
<spark.executor.extraClassPath>${test_classpath}</spark.executor.extraClassPath>
981+
<spark.driver.allowMultipleContexts>true</spark.driver.allowMultipleContexts>
981982
</systemProperties>
982983
</configuration>
983984
<executions>

project/SparkBuild.scala

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -377,6 +377,7 @@ object TestSettings {
377377
javaOptions in Test += "-Dspark.testing=1",
378378
javaOptions in Test += "-Dspark.port.maxRetries=100",
379379
javaOptions in Test += "-Dspark.ui.enabled=false",
380+
javaOptions in Test += "-Dspark.driver.allowMultipleContexts=true",
380381
javaOptions in Test += "-Dsun.io.serialization.extendedDebugInfo=true",
381382
javaOptions in Test ++= System.getProperties.filter(_._1 startsWith "spark")
382383
.map { case (k,v) => s"-D$k=$v" }.toSeq,

0 commit comments

Comments
 (0)