Skip to content

Commit

Permalink
[SPARK-4180] [Core] Prevent creation of multiple active SparkContexts
Browse files Browse the repository at this point in the history
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 <joshrosen@databricks.com>

Closes apache#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.
  • Loading branch information
JoshRosen authored and pwendell committed Nov 17, 2014
1 parent cec1116 commit 0f3ceb5
Show file tree
Hide file tree
Showing 9 changed files with 347 additions and 126 deletions.
167 changes: 145 additions & 22 deletions core/src/main/scala/org/apache/spark/SparkContext.scala
Original file line number Diff line number Diff line change
Expand Up @@ -58,12 +58,26 @@ import org.apache.spark.util._
* Main entry point for Spark functionality. A SparkContext represents the connection to a Spark
* cluster, and can be used to create RDDs, accumulators and broadcast variables on that cluster.
*
* Only one SparkContext may be active per JVM. You must `stop()` the active SparkContext before
* creating a new one. This limitation may eventually be removed; see SPARK-2243 for more details.
*
* @param config a Spark Config object describing the application configuration. Any settings in
* this config overrides the default configs as well as system properties.
*/

class SparkContext(config: SparkConf) extends Logging {

// The call site where this SparkContext was constructed.
private val creationSite: CallSite = Utils.getCallSite()

// If true, log warnings instead of throwing exceptions when multiple SparkContexts are active
private val allowMultipleContexts: Boolean =
config.getBoolean("spark.driver.allowMultipleContexts", false)

// In order to prevent multiple SparkContexts from being active at the same time, mark this
// context as having started construction.
// NOTE: this must be placed at the beginning of the SparkContext constructor.
SparkContext.markPartiallyConstructed(this, allowMultipleContexts)

// This is used only by YARN for now, but should be relevant to other cluster types (Mesos,
// etc) too. This is typically generated from InputFormatInfo.computePreferredLocations. It
// contains a map from hostname to a list of input format splits on the host.
Expand Down Expand Up @@ -1166,27 +1180,30 @@ class SparkContext(config: SparkConf) extends Logging {

/** Shut down the SparkContext. */
def stop() {
postApplicationEnd()
ui.foreach(_.stop())
// Do this only if not stopped already - best case effort.
// prevent NPE if stopped more than once.
val dagSchedulerCopy = dagScheduler
dagScheduler = null
if (dagSchedulerCopy != null) {
env.metricsSystem.report()
metadataCleaner.cancel()
env.actorSystem.stop(heartbeatReceiver)
cleaner.foreach(_.stop())
dagSchedulerCopy.stop()
taskScheduler = null
// TODO: Cache.stop()?
env.stop()
SparkEnv.set(null)
listenerBus.stop()
eventLogger.foreach(_.stop())
logInfo("Successfully stopped SparkContext")
} else {
logInfo("SparkContext already stopped")
SparkContext.SPARK_CONTEXT_CONSTRUCTOR_LOCK.synchronized {
postApplicationEnd()
ui.foreach(_.stop())
// Do this only if not stopped already - best case effort.
// prevent NPE if stopped more than once.
val dagSchedulerCopy = dagScheduler
dagScheduler = null
if (dagSchedulerCopy != null) {
env.metricsSystem.report()
metadataCleaner.cancel()
env.actorSystem.stop(heartbeatReceiver)
cleaner.foreach(_.stop())
dagSchedulerCopy.stop()
taskScheduler = null
// TODO: Cache.stop()?
env.stop()
SparkEnv.set(null)
listenerBus.stop()
eventLogger.foreach(_.stop())
logInfo("Successfully stopped SparkContext")
SparkContext.clearActiveContext()
} else {
logInfo("SparkContext already stopped")
}
}
}

Expand Down Expand Up @@ -1475,6 +1492,11 @@ class SparkContext(config: SparkConf) extends Logging {
private[spark] def cleanup(cleanupTime: Long) {
persistentRdds.clearOldValues(cleanupTime)
}

// In order to prevent multiple SparkContexts from being active at the same time, mark this
// context as having finished construction.
// NOTE: this must be placed at the end of the SparkContext constructor.
SparkContext.setActiveContext(this, allowMultipleContexts)
}

/**
Expand All @@ -1483,6 +1505,107 @@ class SparkContext(config: SparkConf) extends Logging {
*/
object SparkContext extends Logging {

/**
* Lock that guards access to global variables that track SparkContext construction.
*/
private val SPARK_CONTEXT_CONSTRUCTOR_LOCK = new Object()

/**
* The active, fully-constructed SparkContext. If no SparkContext is active, then this is `None`.
*
* Access to this field is guarded by SPARK_CONTEXT_CONSTRUCTOR_LOCK
*/
private var activeContext: Option[SparkContext] = None

/**
* Points to a partially-constructed SparkContext if some thread is in the SparkContext
* constructor, or `None` if no SparkContext is being constructed.
*
* Access to this field is guarded by SPARK_CONTEXT_CONSTRUCTOR_LOCK
*/
private var contextBeingConstructed: Option[SparkContext] = None

/**
* Called to ensure that no other SparkContext is running in this JVM.
*
* Throws an exception if a running context is detected and logs a warning if another thread is
* constructing a SparkContext. This warning is necessary because the current locking scheme
* prevents us from reliably distinguishing between cases where another context is being
* constructed and cases where another constructor threw an exception.
*/
private def assertNoOtherContextIsRunning(
sc: SparkContext,
allowMultipleContexts: Boolean): Unit = {
SPARK_CONTEXT_CONSTRUCTOR_LOCK.synchronized {
contextBeingConstructed.foreach { otherContext =>
if (otherContext ne sc) { // checks for reference equality
// Since otherContext might point to a partially-constructed context, guard against
// its creationSite field being null:
val otherContextCreationSite =
Option(otherContext.creationSite).map(_.longForm).getOrElse("unknown location")
val warnMsg = "Another SparkContext is being constructed (or threw an exception in its" +
" constructor). This may indicate an error, since only one SparkContext may be" +
" running in this JVM (see SPARK-2243)." +
s" The other SparkContext was created at:\n$otherContextCreationSite"
logWarning(warnMsg)
}

activeContext.foreach { ctx =>
val errMsg = "Only one SparkContext may be running in this JVM (see SPARK-2243)." +
" To ignore this error, set spark.driver.allowMultipleContexts = true. " +
s"The currently running SparkContext was created at:\n${ctx.creationSite.longForm}"
val exception = new SparkException(errMsg)
if (allowMultipleContexts) {
logWarning("Multiple running SparkContexts detected in the same JVM!", exception)
} else {
throw exception
}
}
}
}
}

/**
* Called at the beginning of the SparkContext constructor to ensure that no SparkContext is
* running. Throws an exception if a running context is detected and logs a warning if another
* thread is constructing a SparkContext. This warning is necessary because the current locking
* scheme prevents us from reliably distinguishing between cases where another context is being
* constructed and cases where another constructor threw an exception.
*/
private[spark] def markPartiallyConstructed(
sc: SparkContext,
allowMultipleContexts: Boolean): Unit = {
SPARK_CONTEXT_CONSTRUCTOR_LOCK.synchronized {
assertNoOtherContextIsRunning(sc, allowMultipleContexts)
contextBeingConstructed = Some(sc)
}
}

/**
* Called at the end of the SparkContext constructor to ensure that no other SparkContext has
* raced with this constructor and started.
*/
private[spark] def setActiveContext(
sc: SparkContext,
allowMultipleContexts: Boolean): Unit = {
SPARK_CONTEXT_CONSTRUCTOR_LOCK.synchronized {
assertNoOtherContextIsRunning(sc, allowMultipleContexts)
contextBeingConstructed = None
activeContext = Some(sc)
}
}

/**
* Clears the active SparkContext metadata. This is called by `SparkContext#stop()`. It's
* also called in unit tests to prevent a flood of warnings from test suites that don't / can't
* properly clean up their SparkContexts.
*/
private[spark] def clearActiveContext(): Unit = {
SPARK_CONTEXT_CONSTRUCTOR_LOCK.synchronized {
activeContext = None
}
}

private[spark] val SPARK_JOB_DESCRIPTION = "spark.job.description"

private[spark] val SPARK_JOB_GROUP_ID = "spark.jobGroup.id"
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,9 @@ import org.apache.spark.rdd.{EmptyRDD, HadoopRDD, NewHadoopRDD, RDD}
/**
* A Java-friendly version of [[org.apache.spark.SparkContext]] that returns
* [[org.apache.spark.api.java.JavaRDD]]s and works with Java collections instead of Scala ones.
*
* Only one SparkContext may be active per JVM. You must `stop()` the active SparkContext before
* creating a new one. This limitation may eventually be removed; see SPARK-2243 for more details.
*/
class JavaSparkContext(val sc: SparkContext)
extends JavaSparkContextVarargsWorkaround with Closeable {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,20 +37,24 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext {
.set("spark.dynamicAllocation.enabled", "true")
intercept[SparkException] { new SparkContext(conf) }
SparkEnv.get.stop() // cleanup the created environment
SparkContext.clearActiveContext()

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

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

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

// Both min and max, and min == max
val sc1 = createSparkContext(1, 1)
Expand Down
57 changes: 55 additions & 2 deletions core/src/test/scala/org/apache/spark/SparkContextSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -21,9 +21,62 @@ import org.scalatest.FunSuite

import org.apache.hadoop.io.BytesWritable

class SparkContextSuite extends FunSuite {
//Regression test for SPARK-3121
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
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 to construct a previous one") {
withSystemProperty("spark.driver.allowMultipleContexts", "false") {
// This is an invalid configuration (no app name or master URL)
intercept[SparkException] {
new SparkContext(new SparkConf())
}
// 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"))
}
}

test("Check for multiple SparkContexts can be disabled via undocumented debug option") {
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())
}
}
}

test("BytesWritable implicit conversion is correct") {
// Regression test for SPARK-3121
val bytesWritable = new BytesWritable()
val inputArray = (1 to 10).map(_.toByte).toArray
bytesWritable.set(inputArray, 0, 10)
Expand Down
2 changes: 2 additions & 0 deletions docs/programming-guide.md
Original file line number Diff line number Diff line change
Expand Up @@ -117,6 +117,8 @@ The first thing a Spark program must do is to create a [SparkContext](api/scala/
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
that contains information about your application.

Only one SparkContext may be active per JVM. You must `stop()` the active SparkContext before creating a new one.

{% highlight scala %}
val conf = new SparkConf().setAppName(appName).setMaster(master)
new SparkContext(conf)
Expand Down
1 change: 1 addition & 0 deletions pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -978,6 +978,7 @@
<spark.testing>1</spark.testing>
<spark.ui.enabled>false</spark.ui.enabled>
<spark.executor.extraClassPath>${test_classpath}</spark.executor.extraClassPath>
<spark.driver.allowMultipleContexts>true</spark.driver.allowMultipleContexts>
</systemProperties>
</configuration>
<executions>
Expand Down
1 change: 1 addition & 0 deletions project/SparkBuild.scala
Original file line number Diff line number Diff line change
Expand Up @@ -377,6 +377,7 @@ object TestSettings {
javaOptions in Test += "-Dspark.testing=1",
javaOptions in Test += "-Dspark.port.maxRetries=100",
javaOptions in Test += "-Dspark.ui.enabled=false",
javaOptions in Test += "-Dspark.driver.allowMultipleContexts=true",
javaOptions in Test += "-Dsun.io.serialization.extendedDebugInfo=true",
javaOptions in Test ++= System.getProperties.filter(_._1 startsWith "spark")
.map { case (k,v) => s"-D$k=$v" }.toSeq,
Expand Down
Loading

0 comments on commit 0f3ceb5

Please sign in to comment.