aboutsummaryrefslogtreecommitdiff
path: root/streaming
diff options
context:
space:
mode:
authorJosh Rosen <joshrosen@databricks.com>2014-11-17 12:48:18 -0800
committerPatrick Wendell <pwendell@gmail.com>2014-11-17 12:48:18 -0800
commit0f3ceb56c78e7260725a09fba0e10aa193cbda4b (patch)
tree0e033ffceb64c5c0f4787837f4cd709750cba9bd /streaming
parentcec1116b4b80c36b36a8a13338b948e4d6ade377 (diff)
downloadspark-0f3ceb56c78e7260725a09fba0e10aa193cbda4b.tar.gz
spark-0f3ceb56c78e7260725a09fba0e10aa193cbda4b.tar.bz2
spark-0f3ceb56c78e7260725a09fba0e10aa193cbda4b.zip
[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 <joshrosen@databricks.com> 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.
Diffstat (limited to 'streaming')
-rw-r--r--streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala186
-rw-r--r--streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala52
2 files changed, 136 insertions, 102 deletions
diff --git a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala
index 30a359677c..86b96785d7 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala
@@ -470,32 +470,31 @@ class BasicOperationsSuite extends TestSuiteBase {
}
test("slice") {
- val ssc = new StreamingContext(conf, Seconds(1))
- val input = Seq(Seq(1), Seq(2), Seq(3), Seq(4))
- val stream = new TestInputStream[Int](ssc, input, 2)
- stream.foreachRDD(_ => {}) // Dummy output stream
- ssc.start()
- Thread.sleep(2000)
- def getInputFromSlice(fromMillis: Long, toMillis: Long) = {
- stream.slice(new Time(fromMillis), new Time(toMillis)).flatMap(_.collect()).toSet
- }
+ withStreamingContext(new StreamingContext(conf, Seconds(1))) { ssc =>
+ val input = Seq(Seq(1), Seq(2), Seq(3), Seq(4))
+ val stream = new TestInputStream[Int](ssc, input, 2)
+ stream.foreachRDD(_ => {}) // Dummy output stream
+ ssc.start()
+ Thread.sleep(2000)
+ def getInputFromSlice(fromMillis: Long, toMillis: Long) = {
+ stream.slice(new Time(fromMillis), new Time(toMillis)).flatMap(_.collect()).toSet
+ }
- assert(getInputFromSlice(0, 1000) == Set(1))
- assert(getInputFromSlice(0, 2000) == Set(1, 2))
- assert(getInputFromSlice(1000, 2000) == Set(1, 2))
- assert(getInputFromSlice(2000, 4000) == Set(2, 3, 4))
- ssc.stop()
- Thread.sleep(1000)
+ assert(getInputFromSlice(0, 1000) == Set(1))
+ assert(getInputFromSlice(0, 2000) == Set(1, 2))
+ assert(getInputFromSlice(1000, 2000) == Set(1, 2))
+ assert(getInputFromSlice(2000, 4000) == Set(2, 3, 4))
+ }
}
-
test("slice - has not been initialized") {
- val ssc = new StreamingContext(conf, Seconds(1))
- val input = Seq(Seq(1), Seq(2), Seq(3), Seq(4))
- val stream = new TestInputStream[Int](ssc, input, 2)
- val thrown = intercept[SparkException] {
- stream.slice(new Time(0), new Time(1000))
+ withStreamingContext(new StreamingContext(conf, Seconds(1))) { ssc =>
+ val input = Seq(Seq(1), Seq(2), Seq(3), Seq(4))
+ val stream = new TestInputStream[Int](ssc, input, 2)
+ val thrown = intercept[SparkException] {
+ stream.slice(new Time(0), new Time(1000))
+ }
+ assert(thrown.getMessage.contains("has not been initialized"))
}
- assert(thrown.getMessage.contains("has not been initialized"))
}
val cleanupTestInput = (0 until 10).map(x => Seq(x, x + 1)).toSeq
@@ -555,73 +554,72 @@ class BasicOperationsSuite extends TestSuiteBase {
test("rdd cleanup - input blocks and persisted RDDs") {
// Actually receive data over through receiver to create BlockRDDs
- // Start the server
- val testServer = new TestServer()
- testServer.start()
-
- // Set up the streaming context and input streams
- val ssc = new StreamingContext(conf, batchDuration)
- val networkStream = ssc.socketTextStream("localhost", testServer.port, StorageLevel.MEMORY_AND_DISK)
- val mappedStream = networkStream.map(_ + ".").persist()
- val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]]
- val outputStream = new TestOutputStream(mappedStream, outputBuffer)
-
- outputStream.register()
- ssc.start()
-
- // Feed data to the server to send to the network receiver
- val clock = ssc.scheduler.clock.asInstanceOf[ManualClock]
- val input = Seq(1, 2, 3, 4, 5, 6)
+ withTestServer(new TestServer()) { testServer =>
+ withStreamingContext(new StreamingContext(conf, batchDuration)) { ssc =>
+ testServer.start()
+ // Set up the streaming context and input streams
+ val networkStream =
+ ssc.socketTextStream("localhost", testServer.port, StorageLevel.MEMORY_AND_DISK)
+ val mappedStream = networkStream.map(_ + ".").persist()
+ val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]]
+ val outputStream = new TestOutputStream(mappedStream, outputBuffer)
+
+ outputStream.register()
+ ssc.start()
+
+ // Feed data to the server to send to the network receiver
+ val clock = ssc.scheduler.clock.asInstanceOf[ManualClock]
+ val input = Seq(1, 2, 3, 4, 5, 6)
+
+ val blockRdds = new mutable.HashMap[Time, BlockRDD[_]]
+ val persistentRddIds = new mutable.HashMap[Time, Int]
+
+ def collectRddInfo() { // get all RDD info required for verification
+ networkStream.generatedRDDs.foreach { case (time, rdd) =>
+ blockRdds(time) = rdd.asInstanceOf[BlockRDD[_]]
+ }
+ mappedStream.generatedRDDs.foreach { case (time, rdd) =>
+ persistentRddIds(time) = rdd.id
+ }
+ }
- val blockRdds = new mutable.HashMap[Time, BlockRDD[_]]
- val persistentRddIds = new mutable.HashMap[Time, Int]
+ Thread.sleep(200)
+ for (i <- 0 until input.size) {
+ testServer.send(input(i).toString + "\n")
+ Thread.sleep(200)
+ clock.addToTime(batchDuration.milliseconds)
+ collectRddInfo()
+ }
- def collectRddInfo() { // get all RDD info required for verification
- networkStream.generatedRDDs.foreach { case (time, rdd) =>
- blockRdds(time) = rdd.asInstanceOf[BlockRDD[_]]
- }
- mappedStream.generatedRDDs.foreach { case (time, rdd) =>
- persistentRddIds(time) = rdd.id
+ Thread.sleep(200)
+ collectRddInfo()
+ logInfo("Stopping server")
+ testServer.stop()
+
+ // verify data has been received
+ assert(outputBuffer.size > 0)
+ assert(blockRdds.size > 0)
+ assert(persistentRddIds.size > 0)
+
+ import Time._
+
+ val latestPersistedRddId = persistentRddIds(persistentRddIds.keySet.max)
+ val earliestPersistedRddId = persistentRddIds(persistentRddIds.keySet.min)
+ val latestBlockRdd = blockRdds(blockRdds.keySet.max)
+ val earliestBlockRdd = blockRdds(blockRdds.keySet.min)
+ // verify that the latest mapped RDD is persisted but the earliest one has been unpersisted
+ assert(ssc.sparkContext.persistentRdds.contains(latestPersistedRddId))
+ assert(!ssc.sparkContext.persistentRdds.contains(earliestPersistedRddId))
+
+ // verify that the latest input blocks are present but the earliest blocks have been removed
+ assert(latestBlockRdd.isValid)
+ assert(latestBlockRdd.collect != null)
+ assert(!earliestBlockRdd.isValid)
+ earliestBlockRdd.blockIds.foreach { blockId =>
+ assert(!ssc.sparkContext.env.blockManager.master.contains(blockId))
+ }
}
}
-
- Thread.sleep(200)
- for (i <- 0 until input.size) {
- testServer.send(input(i).toString + "\n")
- Thread.sleep(200)
- clock.addToTime(batchDuration.milliseconds)
- collectRddInfo()
- }
-
- Thread.sleep(200)
- collectRddInfo()
- logInfo("Stopping server")
- testServer.stop()
- logInfo("Stopping context")
-
- // verify data has been received
- assert(outputBuffer.size > 0)
- assert(blockRdds.size > 0)
- assert(persistentRddIds.size > 0)
-
- import Time._
-
- val latestPersistedRddId = persistentRddIds(persistentRddIds.keySet.max)
- val earliestPersistedRddId = persistentRddIds(persistentRddIds.keySet.min)
- val latestBlockRdd = blockRdds(blockRdds.keySet.max)
- val earliestBlockRdd = blockRdds(blockRdds.keySet.min)
- // verify that the latest mapped RDD is persisted but the earliest one has been unpersisted
- assert(ssc.sparkContext.persistentRdds.contains(latestPersistedRddId))
- assert(!ssc.sparkContext.persistentRdds.contains(earliestPersistedRddId))
-
- // verify that the latest input blocks are present but the earliest blocks have been removed
- assert(latestBlockRdd.isValid)
- assert(latestBlockRdd.collect != null)
- assert(!earliestBlockRdd.isValid)
- earliestBlockRdd.blockIds.foreach { blockId =>
- assert(!ssc.sparkContext.env.blockManager.master.contains(blockId))
- }
- ssc.stop()
}
/** Test cleanup of RDDs in DStream metadata */
@@ -635,13 +633,15 @@ class BasicOperationsSuite extends TestSuiteBase {
// Setup the stream computation
assert(batchDuration === Seconds(1),
"Batch duration has changed from 1 second, check cleanup tests")
- val ssc = setupStreams(cleanupTestInput, operation)
- val operatedStream = ssc.graph.getOutputStreams().head.dependencies.head.asInstanceOf[DStream[T]]
- if (rememberDuration != null) ssc.remember(rememberDuration)
- val output = runStreams[(Int, Int)](ssc, cleanupTestInput.size, numExpectedOutput)
- val clock = ssc.scheduler.clock.asInstanceOf[ManualClock]
- assert(clock.time === Seconds(10).milliseconds)
- assert(output.size === numExpectedOutput)
- operatedStream
+ withStreamingContext(setupStreams(cleanupTestInput, operation)) { ssc =>
+ val operatedStream =
+ ssc.graph.getOutputStreams().head.dependencies.head.asInstanceOf[DStream[T]]
+ if (rememberDuration != null) ssc.remember(rememberDuration)
+ val output = runStreams[(Int, Int)](ssc, cleanupTestInput.size, numExpectedOutput)
+ val clock = ssc.scheduler.clock.asInstanceOf[ManualClock]
+ assert(clock.time === Seconds(10).milliseconds)
+ assert(output.size === numExpectedOutput)
+ operatedStream
+ }
}
}
diff --git a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala
index 2154c24abd..52972f63c6 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala
@@ -164,6 +164,40 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging {
after(afterFunction)
/**
+ * Run a block of code with the given StreamingContext and automatically
+ * stop the context when the block completes or when an exception is thrown.
+ */
+ def withStreamingContext[R](ssc: StreamingContext)(block: StreamingContext => R): R = {
+ try {
+ block(ssc)
+ } finally {
+ try {
+ ssc.stop(stopSparkContext = true)
+ } catch {
+ case e: Exception =>
+ logError("Error stopping StreamingContext", e)
+ }
+ }
+ }
+
+ /**
+ * Run a block of code with the given TestServer and automatically
+ * stop the server when the block completes or when an exception is thrown.
+ */
+ def withTestServer[R](testServer: TestServer)(block: TestServer => R): R = {
+ try {
+ block(testServer)
+ } finally {
+ try {
+ testServer.stop()
+ } catch {
+ case e: Exception =>
+ logError("Error stopping TestServer", e)
+ }
+ }
+ }
+
+ /**
* Set up required DStreams to test the DStream operation using the two sequences
* of input collections.
*/
@@ -282,10 +316,8 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging {
assert(output.size === numExpectedOutput, "Unexpected number of outputs generated")
Thread.sleep(100) // Give some time for the forgetting old RDDs to complete
- } catch {
- case e: Exception => {e.printStackTrace(); throw e}
} finally {
- ssc.stop()
+ ssc.stop(stopSparkContext = true)
}
output
}
@@ -351,9 +383,10 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging {
useSet: Boolean
) {
val numBatches_ = if (numBatches > 0) numBatches else expectedOutput.size
- val ssc = setupStreams[U, V](input, operation)
- val output = runStreams[V](ssc, numBatches_, expectedOutput.size)
- verifyOutput[V](output, expectedOutput, useSet)
+ withStreamingContext(setupStreams[U, V](input, operation)) { ssc =>
+ val output = runStreams[V](ssc, numBatches_, expectedOutput.size)
+ verifyOutput[V](output, expectedOutput, useSet)
+ }
}
/**
@@ -389,8 +422,9 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging {
useSet: Boolean
) {
val numBatches_ = if (numBatches > 0) numBatches else expectedOutput.size
- val ssc = setupStreams[U, V, W](input1, input2, operation)
- val output = runStreams[W](ssc, numBatches_, expectedOutput.size)
- verifyOutput[W](output, expectedOutput, useSet)
+ withStreamingContext(setupStreams[U, V, W](input1, input2, operation)) { ssc =>
+ val output = runStreams[W](ssc, numBatches_, expectedOutput.size)
+ verifyOutput[W](output, expectedOutput, useSet)
+ }
}
}