-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-5681][Streaming] Add tracker status and stop to receive messages when stopping tracker #4467
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
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 |
|---|---|---|
|
|
@@ -37,6 +37,12 @@ private[streaming] case class RegisterReceiver( | |
| host: String, | ||
| receiverEndpoint: RpcEndpointRef | ||
| ) extends ReceiverTrackerMessage | ||
| private[streaming] case class ReceiverStarted( | ||
| streamId: Int, | ||
| typ: String, | ||
| host: String, | ||
| receiverEndpoint: RpcEndpointRef | ||
| ) extends ReceiverTrackerMessage | ||
| private[streaming] case class AddBlock(receivedBlockInfo: ReceivedBlockInfo) | ||
| extends ReceiverTrackerMessage | ||
| private[streaming] case class ReportError(streamId: Int, message: String, error: String) | ||
|
|
@@ -67,13 +73,38 @@ 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 CheckpointState = Value | ||
| val Initialized, Started, Stopping, Stopped = Value | ||
| } | ||
| import TrackerState._ | ||
|
|
||
| /** State of the tracker */ | ||
| @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 | ||
|
|
||
| /** 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 | ||
| } | ||
|
|
||
| /** Start the endpoint and receiver execution thread. */ | ||
| def start(): Unit = synchronized { | ||
| if (endpoint != null) { | ||
| if (isTrackerStarted) { | ||
| throw new SparkException("ReceiverTracker already started") | ||
| } | ||
|
|
||
|
|
@@ -82,12 +113,14 @@ 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) { | ||
| trackerState = Stopping | ||
| // First, stop the receivers | ||
| if (!skipReceiverLaunch) receiverExecutor.stop(graceful) | ||
|
|
||
|
|
@@ -96,6 +129,7 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false | |
| endpoint = null | ||
| receivedBlockTracker.stop() | ||
| logInfo("ReceiverTracker stopped") | ||
| trackerState = Stopped | ||
| } | ||
| } | ||
|
|
||
|
|
@@ -147,10 +181,27 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false | |
| } | ||
| receiverInfo(streamId) = ReceiverInfo( | ||
| streamId, s"${typ}-${streamId}", receiverEndpoint, true, host) | ||
| listenerBus.post(StreamingListenerReceiverStarted(receiverInfo(streamId))) | ||
| listenerBus.post(StreamingListenerReceiverRegistered(receiverInfo(streamId))) | ||
| logInfo("Registered receiver for stream " + streamId + " from " + senderAddress) | ||
| } | ||
|
|
||
|
|
||
| /** Receiver started */ | ||
| private def receiverStarted( | ||
| streamId: Int, | ||
| typ: String, | ||
| host: String, | ||
| receiverEndpoint: RpcEndpointRef, | ||
| senderAddress: RpcAddress | ||
| ) { | ||
| if (!receiverInputStreamIds.contains(streamId)) { | ||
| throw new SparkException("Start received for unexpected id " + streamId) | ||
| } | ||
| receiverInfo(streamId) = ReceiverInfo( | ||
| streamId, s"${typ}-${streamId}", receiverEndpoint, true, host) | ||
| listenerBus.post(StreamingListenerReceiverStarted(receiverInfo(streamId))) | ||
| logInfo("Receiver started for stream " + streamId + " from " + senderAddress) | ||
| } | ||
|
|
||
| /** Deregister a receiver */ | ||
| private def deregisterReceiver(streamId: Int, message: String, error: String) { | ||
| val newReceiverInfo = receiverInfo.get(streamId) match { | ||
|
|
@@ -216,8 +267,19 @@ 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) | ||
| if (!isTrackerStopping) { | ||
| registerReceiver(streamId, typ, host, receiverEndpoint, context.sender.address) | ||
| context.reply(true) | ||
| } else { | ||
| context.reply(false) | ||
| } | ||
|
Member
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. You need to send something back, or
Member
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 propose that returning false if
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 was originally intended to let it timeout and throw exception. Returning false and throw exception is good too. I will update it. |
||
| case ReceiverStarted(streamId, typ, host, receiverEndpoint) => | ||
| if (!isTrackerStopping) { | ||
| receiverStarted(streamId, typ, host, receiverEndpoint, context.sender.address) | ||
| context.reply(true) | ||
| } else { | ||
| context.reply(false) | ||
| } | ||
| case AddBlock(receivedBlockInfo) => | ||
| context.reply(addBlock(receivedBlockInfo)) | ||
| case DeregisterReceiver(streamId, message, error) => | ||
|
|
||
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.
changing trackerState to Stoppingandcalling stopReceivers()should be atomic, or it's still possible thatStopReceiverwon't be sent because theregisterReceivermethod runs in another thread.And you need to use another lock rather than adding
synchronizedtoregisterReceiverbecause it may blockReceiverTrackerEndpoint, which is actually a Akka thread. If it blocks the Akka thread for a long time, some dead-lock may happen.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.
Oh, I see.
ReceiverStartedwill be rejected and force the receiver stop. It's really tricky. Do you think we can solve this issue withoutReceiverStarted?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.
We need to prevent the receiver to step forward if the tracker is entering stopping status. So we don't have the risk to lose data. Without
ReceiverStarted, we may have to modify the current receiver initialization.