-
Notifications
You must be signed in to change notification settings - Fork 28.1k
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
[SPARK-10137][Streaming]Avoid to restart receivers if scheduleReceivers returns balanced results #8340
[SPARK-10137][Streaming]Avoid to restart receivers if scheduleReceivers returns balanced results #8340
Changes from all commits
5fba8d4
7283bf7
f0d1f6e
55d9957
c70690f
a1a1085
f2954eb
b8faa3a
55fe99e
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 |
---|---|---|
|
@@ -244,8 +244,21 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false | |
} | ||
|
||
if (isTrackerStopping || isTrackerStopped) { | ||
false | ||
} else if (!scheduleReceiver(streamId).contains(hostPort)) { | ||
return false | ||
} | ||
|
||
val scheduledExecutors = receiverTrackingInfos(streamId).scheduledExecutors | ||
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. It might be more intuitive rewrite this nested if to the following:
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. Updated. |
||
val accetableExecutors = if (scheduledExecutors.nonEmpty) { | ||
// This receiver is registering and it's scheduled by | ||
// ReceiverSchedulingPolicy.scheduleReceivers. So use "scheduledExecutors" to check it. | ||
scheduledExecutors.get | ||
} else { | ||
// This receiver is scheduled by "ReceiverSchedulingPolicy.rescheduleReceiver", so calling | ||
// "ReceiverSchedulingPolicy.rescheduleReceiver" again to check it. | ||
scheduleReceiver(streamId) | ||
} | ||
|
||
if (!accetableExecutors.contains(hostPort)) { | ||
// Refuse it since it's scheduled to a wrong executor | ||
false | ||
} else { | ||
|
@@ -426,12 +439,25 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false | |
startReceiver(receiver, executors) | ||
} | ||
case RestartReceiver(receiver) => | ||
val scheduledExecutors = schedulingPolicy.rescheduleReceiver( | ||
receiver.streamId, | ||
receiver.preferredLocation, | ||
receiverTrackingInfos, | ||
getExecutors) | ||
updateReceiverScheduledExecutors(receiver.streamId, scheduledExecutors) | ||
// Old scheduled executors minus the ones that are not active any more | ||
val oldScheduledExecutors = getStoredScheduledExecutors(receiver.streamId) | ||
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. Add a comment say this is old scheduled executors minus the one that are not active any more. 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. Added. |
||
val scheduledExecutors = if (oldScheduledExecutors.nonEmpty) { | ||
// Try global scheduling again | ||
oldScheduledExecutors | ||
} else { | ||
val oldReceiverInfo = receiverTrackingInfos(receiver.streamId) | ||
// Clear "scheduledExecutors" to indicate we are going to do local scheduling | ||
val newReceiverInfo = oldReceiverInfo.copy( | ||
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. Isnt it already marked INACTIVE when the previous jobs failed? 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. No. If |
||
state = ReceiverState.INACTIVE, scheduledExecutors = None) | ||
receiverTrackingInfos(receiver.streamId) = newReceiverInfo | ||
schedulingPolicy.rescheduleReceiver( | ||
receiver.streamId, | ||
receiver.preferredLocation, | ||
receiverTrackingInfos, | ||
getExecutors) | ||
} | ||
// Assume there is one receiver restarting at one time, so we don't need to update | ||
// receiverTrackingInfos | ||
startReceiver(receiver, scheduledExecutors) | ||
case c: CleanupOldBlocks => | ||
receiverTrackingInfos.values.flatMap(_.endpoint).foreach(_.send(c)) | ||
|
@@ -464,6 +490,24 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false | |
context.reply(true) | ||
} | ||
|
||
/** | ||
* Return the stored scheduled executors that are still alive. | ||
*/ | ||
private def getStoredScheduledExecutors(receiverId: Int): Seq[String] = { | ||
if (receiverTrackingInfos.contains(receiverId)) { | ||
val scheduledExecutors = receiverTrackingInfos(receiverId).scheduledExecutors | ||
if (scheduledExecutors.nonEmpty) { | ||
val executors = getExecutors.toSet | ||
// Only return the alive executors | ||
scheduledExecutors.get.filter(executors) | ||
} else { | ||
Nil | ||
} | ||
} else { | ||
Nil | ||
} | ||
} | ||
|
||
/** | ||
* Start a receiver along with its scheduled executors | ||
*/ | ||
|
@@ -484,7 +528,23 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false | |
new SerializableConfiguration(ssc.sparkContext.hadoopConfiguration) | ||
|
||
// Function to start the receiver on the worker node | ||
val startReceiverFunc = new StartReceiverFunc(checkpointDirOption, serializableHadoopConf) | ||
val startReceiverFunc: Iterator[Receiver[_]] => Unit = | ||
(iterator: Iterator[Receiver[_]]) => { | ||
if (!iterator.hasNext) { | ||
throw new SparkException( | ||
"Could not start receiver as object not found.") | ||
} | ||
if (TaskContext.get().attemptNumber() == 0) { | ||
val receiver = iterator.next() | ||
assert(iterator.hasNext == false) | ||
val supervisor = new ReceiverSupervisorImpl( | ||
receiver, SparkEnv.get, serializableHadoopConf.value, checkpointDirOption) | ||
supervisor.start() | ||
supervisor.awaitTermination() | ||
} else { | ||
// It's restarted by TaskScheduler, but we want to reschedule it again. So exit it. | ||
} | ||
} | ||
|
||
// Create the RDD using the scheduledExecutors to run the receiver in a Spark job | ||
val receiverRDD: RDD[Receiver[_]] = | ||
|
@@ -541,31 +601,3 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false | |
} | ||
|
||
} | ||
|
||
/** | ||
* Function to start the receiver on the worker node. Use a class instead of closure to avoid | ||
* the serialization issue. | ||
*/ | ||
private[streaming] class StartReceiverFunc( | ||
checkpointDirOption: Option[String], | ||
serializableHadoopConf: SerializableConfiguration) | ||
extends (Iterator[Receiver[_]] => Unit) with Serializable { | ||
|
||
override def apply(iterator: Iterator[Receiver[_]]): Unit = { | ||
if (!iterator.hasNext) { | ||
throw new SparkException( | ||
"Could not start receiver as object not found.") | ||
} | ||
if (TaskContext.get().attemptNumber() == 0) { | ||
val receiver = iterator.next() | ||
assert(iterator.hasNext == false) | ||
val supervisor = new ReceiverSupervisorImpl( | ||
receiver, SparkEnv.get, serializableHadoopConf.value, checkpointDirOption) | ||
supervisor.start() | ||
supervisor.awaitTermination() | ||
} else { | ||
// It's restarted by TaskScheduler, but we want to reschedule it again. So exit it. | ||
} | ||
} | ||
|
||
} |
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.
Are you talking about
executorWeights no being empty? And why it cannot be empty? Is it because that by the time
rescheduleReceiveris called, all the receivers have already been scheduling by
scheduleReceivers` which means that there are some weights for at some executors?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.
Here I mean, because idleExecutors is empty and executors is not empty, since
sortedExecutors.keys == executors - idleExecutors
, sortedExecutors must be not empty.