@@ -67,13 +67,41 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false
6767 )
6868 private val listenerBus = ssc.scheduler.listenerBus
6969
70+ /** Enumeration to identify current state of the ReceiverTracker */
71+ object TrackerState extends Enumeration {
72+ type CheckpointState = Value
73+ val Initialized, Started, Stopping, Stopped = Value
74+ }
75+ import TrackerState ._
76+
77+ /** State of the tracker. Protected by "trackerStateLock" */
78+ private var trackerState = Initialized
79+
80+ /** "trackerStateLock" is used to protect reading/writing "trackerState" */
81+ private val trackerStateLock = new AnyRef
82+
7083 // endpoint is created when generator starts.
7184 // This not being null means the tracker has been started and not stopped
7285 private var endpoint : RpcEndpointRef = null
7386
87+ /** Check if tracker has been marked for starting */
88+ private def isTrackerStarted (): Boolean = trackerStateLock.synchronized {
89+ trackerState == Started
90+ }
91+
92+ /** Check if tracker has been marked for stopping */
93+ private def isTrackerStopping (): Boolean = trackerStateLock.synchronized {
94+ trackerState == Stopping
95+ }
96+
97+ /** Check if tracker has been marked for stopped */
98+ private def isTrackerStopped (): Boolean = trackerStateLock.synchronized {
99+ trackerState == Stopped
100+ }
101+
74102 /** Start the endpoint and receiver execution thread. */
75103 def start (): Unit = synchronized {
76- if (endpoint != null ) {
104+ if (isTrackerStarted ) {
77105 throw new SparkException (" ReceiverTracker already started" )
78106 }
79107
@@ -82,20 +110,29 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false
82110 " ReceiverTracker" , new ReceiverTrackerEndpoint (ssc.env.rpcEnv))
83111 if (! skipReceiverLaunch) receiverExecutor.start()
84112 logInfo(" ReceiverTracker started" )
113+ trackerStateLock.synchronized {
114+ trackerState = Started
115+ }
85116 }
86117 }
87118
88119 /** Stop the receiver execution thread. */
89120 def stop (graceful : Boolean ): Unit = synchronized {
90- if (! receiverInputStreams.isEmpty && endpoint != null ) {
121+ if (isTrackerStarted ) {
91122 // First, stop the receivers
123+ trackerStateLock.synchronized {
124+ trackerState = Stopping
125+ }
92126 if (! skipReceiverLaunch) receiverExecutor.stop(graceful)
93127
94128 // Finally, stop the endpoint
95129 ssc.env.rpcEnv.stop(endpoint)
96130 endpoint = null
97131 receivedBlockTracker.stop()
98132 logInfo(" ReceiverTracker stopped" )
133+ trackerStateLock.synchronized {
134+ trackerState = Stopped
135+ }
99136 }
100137 }
101138
@@ -141,14 +178,24 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false
141178 host : String ,
142179 receiverEndpoint : RpcEndpointRef ,
143180 senderAddress : RpcAddress
144- ) {
181+ ): Boolean = {
145182 if (! receiverInputStreamIds.contains(streamId)) {
146183 throw new SparkException (" Register received for unexpected id " + streamId)
147184 }
148- receiverInfo(streamId) = ReceiverInfo (
149- streamId, s " ${typ}- ${streamId}" , receiverEndpoint, true , host)
150- listenerBus.post(StreamingListenerReceiverStarted (receiverInfo(streamId)))
151- logInfo(" Registered receiver for stream " + streamId + " from " + senderAddress)
185+
186+ trackerStateLock.synchronized {
187+ if (isTrackerStopping || isTrackerStopped) {
188+ false
189+ } else {
190+ // When updating "receiverInfo", we should make sure "trackerState" won't be changed at the
191+ // same time. Therefore the following line should be in "trackerStateLock.synchronized".
192+ receiverInfo(streamId) = ReceiverInfo (
193+ streamId, s " ${typ}- ${streamId}" , receiverEndpoint, true , host)
194+ listenerBus.post(StreamingListenerReceiverStarted (receiverInfo(streamId)))
195+ logInfo(" Registered receiver for stream " + streamId + " from " + senderAddress)
196+ true
197+ }
198+ }
152199 }
153200
154201 /** Deregister a receiver */
@@ -216,8 +263,9 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false
216263
217264 override def receiveAndReply (context : RpcCallContext ): PartialFunction [Any , Unit ] = {
218265 case RegisterReceiver (streamId, typ, host, receiverEndpoint) =>
219- registerReceiver(streamId, typ, host, receiverEndpoint, context.sender.address)
220- context.reply(true )
266+ val successful =
267+ registerReceiver(streamId, typ, host, receiverEndpoint, context.sender.address)
268+ context.reply(successful)
221269 case AddBlock (receivedBlockInfo) =>
222270 context.reply(addBlock(receivedBlockInfo))
223271 case DeregisterReceiver (streamId, message, error) =>
@@ -317,9 +365,12 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false
317365 // Distribute the receivers and start them
318366 logInfo(" Starting " + receivers.length + " receivers" )
319367 running = true
320- ssc.sparkContext.runJob(tempRDD, ssc.sparkContext.clean(startReceiver))
321- running = false
322- logInfo(" All of the receivers have been terminated" )
368+ try {
369+ ssc.sparkContext.runJob(tempRDD, ssc.sparkContext.clean(startReceiver))
370+ logInfo(" All of the receivers have been terminated" )
371+ } finally {
372+ running = false
373+ }
323374 }
324375
325376 /** Stops the receivers. */
0 commit comments