aboutsummaryrefslogtreecommitdiff
path: root/streaming
diff options
context:
space:
mode:
Diffstat (limited to 'streaming')
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala6
-rw-r--r--streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala34
2 files changed, 39 insertions, 1 deletions
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
index 97113835f3..aee172a4f5 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
@@ -44,7 +44,7 @@ import org.apache.spark.streaming.dstream._
import org.apache.spark.streaming.receiver.{ActorReceiver, ActorSupervisorStrategy, Receiver}
import org.apache.spark.streaming.scheduler.{JobScheduler, StreamingListener}
import org.apache.spark.streaming.ui.{StreamingJobProgressListener, StreamingTab}
-import org.apache.spark.util.{CallSite, ShutdownHookManager, ThreadUtils, Utils}
+import org.apache.spark.util.{AsynchronousListenerBus, CallSite, ShutdownHookManager, ThreadUtils, Utils}
/**
* Main entry point for Spark Streaming functionality. It provides methods used to create
@@ -693,6 +693,10 @@ class StreamingContext private[streaming] (
*/
def stop(stopSparkContext: Boolean, stopGracefully: Boolean): Unit = {
var shutdownHookRefToRemove: AnyRef = null
+ if (AsynchronousListenerBus.withinListenerThread.value) {
+ throw new SparkException("Cannot stop StreamingContext within listener thread of" +
+ " AsynchronousListenerBus")
+ }
synchronized {
try {
state match {
diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala
index 5dc0472c77..df4575ab25 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala
@@ -21,6 +21,7 @@ import scala.collection.mutable.{ArrayBuffer, HashMap, SynchronizedBuffer, Synch
import scala.concurrent.Future
import scala.concurrent.ExecutionContext.Implicits.global
+import org.apache.spark.SparkException
import org.apache.spark.storage.StorageLevel
import org.apache.spark.streaming.dstream.DStream
import org.apache.spark.streaming.receiver.Receiver
@@ -161,6 +162,14 @@ class StreamingListenerSuite extends TestSuiteBase with Matchers {
}
}
+ test("don't call ssc.stop in listener") {
+ ssc = new StreamingContext("local[2]", "ssc", Milliseconds(1000))
+ val inputStream = ssc.receiverStream(new StreamingListenerSuiteReceiver)
+ inputStream.foreachRDD(_.count)
+
+ startStreamingContextAndCallStop(ssc)
+ }
+
test("onBatchCompleted with successful batch") {
ssc = new StreamingContext("local[2]", "test", Milliseconds(1000))
val inputStream = ssc.receiverStream(new StreamingListenerSuiteReceiver)
@@ -207,6 +216,17 @@ class StreamingListenerSuite extends TestSuiteBase with Matchers {
assert(failureReasons(1).contains("This is another failed job"))
}
+ private def startStreamingContextAndCallStop(_ssc: StreamingContext): Unit = {
+ val contextStoppingCollector = new StreamingContextStoppingCollector(_ssc)
+ _ssc.addStreamingListener(contextStoppingCollector)
+ val batchCounter = new BatchCounter(_ssc)
+ _ssc.start()
+ // Make sure running at least one batch
+ batchCounter.waitUntilBatchesCompleted(expectedNumCompletedBatches = 1, timeout = 10000)
+ _ssc.stop()
+ assert(contextStoppingCollector.sparkExSeen)
+ }
+
private def startStreamingContextAndCollectFailureReasons(
_ssc: StreamingContext, isFailed: Boolean = false): Map[Int, String] = {
val failureReasonsCollector = new FailureReasonsCollector()
@@ -320,3 +340,17 @@ class FailureReasonsCollector extends StreamingListener {
}
}
}
+/**
+ * A StreamingListener that calls StreamingContext.stop().
+ */
+class StreamingContextStoppingCollector(val ssc: StreamingContext) extends StreamingListener {
+ @volatile var sparkExSeen = false
+ override def onBatchCompleted(batchCompleted: StreamingListenerBatchCompleted) {
+ try {
+ ssc.stop()
+ } catch {
+ case se: SparkException =>
+ sparkExSeen = true
+ }
+ }
+}