-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-5681][Streaming] Move 'stopReceivers' to the event loop to resolve the race condition #6294
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
77983f3
ae0d9fd
3d568e8
355f9ce
9e1a760
c419677
34c18dc
19b76d9
e0ef72a
fff63f9
15ed4a1
7b1d9af
a8120c0
7c73c1f
c41ee94
b4c29e7
3cb19a3
51fb07e
a178d37
f637142
e103e8a
709ac1f
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -20,7 +20,6 @@ package org.apache.spark.streaming.scheduler | |
| import scala.collection.mutable.{ArrayBuffer, HashMap, SynchronizedMap} | ||
| import scala.language.existentials | ||
| import scala.math.max | ||
| import org.apache.spark.rdd._ | ||
|
|
||
| import org.apache.spark.streaming.util.WriteAheadLogUtils | ||
| import org.apache.spark.{Logging, SparkEnv, SparkException} | ||
|
|
@@ -47,6 +46,8 @@ private[streaming] case class ReportError(streamId: Int, message: String, error: | |
| private[streaming] case class DeregisterReceiver(streamId: Int, msg: String, error: String) | ||
| extends ReceiverTrackerMessage | ||
|
|
||
| private[streaming] case object StopAllReceivers extends ReceiverTrackerMessage | ||
|
|
||
| /** | ||
| * This class manages the execution of the receivers of ReceiverInputDStreams. Instance of | ||
| * this class must be created after all input streams have been added and StreamingContext.start() | ||
|
|
@@ -71,13 +72,23 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false | |
| ) | ||
| private val listenerBus = ssc.scheduler.listenerBus | ||
|
|
||
| /** Enumeration to identify current state of the ReceiverTracker */ | ||
| object TrackerState extends Enumeration { | ||
| type TrackerState = Value | ||
| val Initialized, Started, Stopping, Stopped = Value | ||
| } | ||
| import TrackerState._ | ||
|
|
||
| /** State of the tracker. Protected by "trackerStateLock" */ | ||
| @volatile private var trackerState = Initialized | ||
|
|
||
| // endpoint is created when generator starts. | ||
| // This not being null means the tracker has been started and not stopped | ||
| private var endpoint: RpcEndpointRef = null | ||
|
|
||
| /** Start the endpoint and receiver execution thread. */ | ||
| def start(): Unit = synchronized { | ||
| if (endpoint != null) { | ||
| if (isTrackerStarted) { | ||
| throw new SparkException("ReceiverTracker already started") | ||
| } | ||
|
|
||
|
|
@@ -86,20 +97,46 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false | |
| "ReceiverTracker", new ReceiverTrackerEndpoint(ssc.env.rpcEnv)) | ||
| if (!skipReceiverLaunch) receiverExecutor.start() | ||
| logInfo("ReceiverTracker started") | ||
| trackerState = Started | ||
| } | ||
| } | ||
|
|
||
| /** Stop the receiver execution thread. */ | ||
| def stop(graceful: Boolean): Unit = synchronized { | ||
| if (!receiverInputStreams.isEmpty && endpoint != null) { | ||
| if (isTrackerStarted) { | ||
| // First, stop the receivers | ||
| if (!skipReceiverLaunch) receiverExecutor.stop(graceful) | ||
| trackerState = Stopping | ||
| if (!skipReceiverLaunch) { | ||
| // Send the stop signal to all the receivers | ||
| endpoint.askWithRetry[Boolean](StopAllReceivers) | ||
|
|
||
| // Wait for the Spark job that runs the receivers to be over | ||
| // That is, for the receivers to quit gracefully. | ||
| receiverExecutor.awaitTermination(10000) | ||
|
|
||
| if (graceful) { | ||
| val pollTime = 100 | ||
| logInfo("Waiting for receiver job to terminate gracefully") | ||
| while (receiverInfo.nonEmpty || receiverExecutor.running) { | ||
| Thread.sleep(pollTime) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This needs to fixed, this could get stuck indefinitely. But different PR. |
||
| } | ||
| logInfo("Waited for receiver job to terminate gracefully") | ||
| } | ||
|
|
||
| // Check if all the receivers have been deregistered or not | ||
| if (receiverInfo.nonEmpty) { | ||
| logWarning("Not all of the receivers have deregistered, " + receiverInfo) | ||
| } else { | ||
| logInfo("All of the receivers have deregistered successfully") | ||
| } | ||
| } | ||
|
|
||
| // Finally, stop the endpoint | ||
| ssc.env.rpcEnv.stop(endpoint) | ||
| endpoint = null | ||
| receivedBlockTracker.stop() | ||
| logInfo("ReceiverTracker stopped") | ||
| trackerState = Stopped | ||
| } | ||
| } | ||
|
|
||
|
|
@@ -145,14 +182,23 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false | |
| host: String, | ||
| receiverEndpoint: RpcEndpointRef, | ||
| senderAddress: RpcAddress | ||
| ) { | ||
| ): Boolean = { | ||
| if (!receiverInputStreamIds.contains(streamId)) { | ||
| throw new SparkException("Register received for unexpected id " + streamId) | ||
| } | ||
| receiverInfo(streamId) = ReceiverInfo( | ||
| streamId, s"${typ}-${streamId}", receiverEndpoint, true, host) | ||
| listenerBus.post(StreamingListenerReceiverStarted(receiverInfo(streamId))) | ||
| logInfo("Registered receiver for stream " + streamId + " from " + senderAddress) | ||
|
|
||
| if (isTrackerStopping || isTrackerStopped) { | ||
| false | ||
| } else { | ||
| // "stopReceivers" won't happen at the same time because both "registerReceiver" and are | ||
| // called in the event loop. So here we can assume "stopReceivers" has not yet been called. If | ||
| // "stopReceivers" is called later, it should be able to see this receiver. | ||
| receiverInfo(streamId) = ReceiverInfo( | ||
| streamId, s"${typ}-${streamId}", receiverEndpoint, true, host) | ||
| listenerBus.post(StreamingListenerReceiverStarted(receiverInfo(streamId))) | ||
| logInfo("Registered receiver for stream " + streamId + " from " + senderAddress) | ||
| true | ||
| } | ||
| } | ||
|
|
||
| /** Deregister a receiver */ | ||
|
|
@@ -220,20 +266,33 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false | |
|
|
||
| override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { | ||
| case RegisterReceiver(streamId, typ, host, receiverEndpoint) => | ||
| registerReceiver(streamId, typ, host, receiverEndpoint, context.sender.address) | ||
| context.reply(true) | ||
| val successful = | ||
| registerReceiver(streamId, typ, host, receiverEndpoint, context.sender.address) | ||
| context.reply(successful) | ||
| case AddBlock(receivedBlockInfo) => | ||
| context.reply(addBlock(receivedBlockInfo)) | ||
| case DeregisterReceiver(streamId, message, error) => | ||
| deregisterReceiver(streamId, message, error) | ||
| context.reply(true) | ||
| case StopAllReceivers => | ||
| assert(isTrackerStopping || isTrackerStopped) | ||
| stopReceivers() | ||
| context.reply(true) | ||
| } | ||
|
|
||
| /** Send stop signal to the receivers. */ | ||
| private def stopReceivers() { | ||
| // Signal the receivers to stop | ||
| receiverInfo.values.flatMap { info => Option(info.endpoint)} | ||
| .foreach { _.send(StopReceiver) } | ||
| logInfo("Sent stop signal to all " + receiverInfo.size + " receivers") | ||
| } | ||
| } | ||
|
|
||
| /** This thread class runs all the receivers on the cluster. */ | ||
| class ReceiverLauncher { | ||
| @transient val env = ssc.env | ||
| @volatile @transient private var running = false | ||
| @volatile @transient var running = false | ||
| @transient val thread = new Thread() { | ||
| override def run() { | ||
| try { | ||
|
|
@@ -249,31 +308,6 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false | |
| thread.start() | ||
| } | ||
|
|
||
| def stop(graceful: Boolean) { | ||
| // Send the stop signal to all the receivers | ||
| stopReceivers() | ||
|
|
||
| // Wait for the Spark job that runs the receivers to be over | ||
| // That is, for the receivers to quit gracefully. | ||
| thread.join(10000) | ||
|
|
||
| if (graceful) { | ||
| val pollTime = 100 | ||
| logInfo("Waiting for receiver job to terminate gracefully") | ||
| while (receiverInfo.nonEmpty || running) { | ||
| Thread.sleep(pollTime) | ||
| } | ||
| logInfo("Waited for receiver job to terminate gracefully") | ||
| } | ||
|
|
||
| // Check if all the receivers have been deregistered or not | ||
| if (receiverInfo.nonEmpty) { | ||
| logWarning("Not all of the receivers have deregistered, " + receiverInfo) | ||
| } else { | ||
| logInfo("All of the receivers have deregistered successfully") | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * Get the list of executors excluding driver | ||
| */ | ||
|
|
@@ -358,17 +392,30 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false | |
| // Distribute the receivers and start them | ||
| logInfo("Starting " + receivers.length + " receivers") | ||
| running = true | ||
| ssc.sparkContext.runJob(tempRDD, ssc.sparkContext.clean(startReceiver)) | ||
| running = false | ||
| logInfo("All of the receivers have been terminated") | ||
| try { | ||
| ssc.sparkContext.runJob(tempRDD, ssc.sparkContext.clean(startReceiver)) | ||
| logInfo("All of the receivers have been terminated") | ||
| } finally { | ||
| running = false | ||
| } | ||
| } | ||
|
|
||
| /** Stops the receivers. */ | ||
| private def stopReceivers() { | ||
| // Signal the receivers to stop | ||
| receiverInfo.values.flatMap { info => Option(info.endpoint)} | ||
| .foreach { _.send(StopReceiver) } | ||
| logInfo("Sent stop signal to all " + receiverInfo.size + " receivers") | ||
| /** | ||
| * Wait until the Spark job that runs the receivers is terminated, or return when | ||
| * `milliseconds` elapses | ||
| */ | ||
| def awaitTermination(milliseconds: Long): Unit = { | ||
| thread.join(milliseconds) | ||
| } | ||
| } | ||
|
|
||
| /** Check if tracker has been marked for starting */ | ||
| private def isTrackerStarted(): Boolean = trackerState == Started | ||
|
|
||
| /** Check if tracker has been marked for stopping */ | ||
| private def isTrackerStopping(): Boolean = trackerState == Stopping | ||
|
|
||
| /** Check if tracker has been marked for stopped */ | ||
| private def isTrackerStopped(): Boolean = trackerState == Stopped | ||
|
|
||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -285,6 +285,21 @@ class StreamingContextSuite extends SparkFunSuite with BeforeAndAfter with Timeo | |
| } | ||
| } | ||
|
|
||
| test("stop gracefully even if a receiver misses StopReceiver") { | ||
| // This is not a deterministic unit. But if this unit test is flaky, then there is definitely | ||
| // something wrong. See SPARK-5681 | ||
| val conf = new SparkConf().setMaster(master).setAppName(appName) | ||
| sc = new SparkContext(conf) | ||
| ssc = new StreamingContext(sc, Milliseconds(100)) | ||
| val input = ssc.receiverStream(new TestReceiver) | ||
| input.foreachRDD(_ => {}) | ||
| ssc.start() | ||
| // Call `ssc.stop` at once so that it's possible that the receiver will miss "StopReceiver" | ||
| failAfter(30000 millis) { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. How is this test deterministic? I dont see how it is guaranteed to test the situation where the context is stopped before the receiver has started?
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This test cannot always reproduce the issue. But the probability is high.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. please comment this at the top of this unit test that this is not a deterministic unit. If this unit test is flaky, then there is definitely something wrong. Point to the JIRa.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Also, could you run this unit test locally repeatedly ... like 1000 times .. to see if it fail any time?
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I ran |
||
| ssc.stop(stopSparkContext = true, stopGracefully = true) | ||
| } | ||
| } | ||
|
|
||
| test("stop slow receiver gracefully") { | ||
| val conf = new SparkConf().setMaster(master).setAppName(appName) | ||
| conf.set("spark.streaming.gracefulStopTimeout", "20000s") | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@tdas now we stop the receiver only if
receiverStateisStarted. What do you think?There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thats the right way to do this. But then by that logic we should start only when the state is Init or Stopped. We are not doing that. In fact, that leads to the question of locking the state correctly and all. I dont want to get into fixing those in this PR.
I think its better not change stopReceiver() and rather change stop() to call stopReceiver() only when the state is started
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I changed it here because
stopReceiveris called by bothstopandrestartReceiver.