-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-13669][SPARK-20898][Core] Improve the blacklist mechanism to handle external shuffle service unavailable situation #17113
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
50ac2e4
759ebc9
533ee17
f633a3f
44c7108
524fbfc
9a14105
3cf9cfd
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 |
|---|---|---|
|
|
@@ -61,6 +61,7 @@ private[scheduler] class BlacklistTracker ( | |
| private val MAX_FAILURES_PER_EXEC = conf.get(config.MAX_FAILURES_PER_EXEC) | ||
| private val MAX_FAILED_EXEC_PER_NODE = conf.get(config.MAX_FAILED_EXEC_PER_NODE) | ||
| val BLACKLIST_TIMEOUT_MILLIS = BlacklistTracker.getBlacklistTimeout(conf) | ||
| private val BLACKLIST_FETCH_FAILURE_ENABLED = conf.get(config.BLACKLIST_FETCH_FAILURE_ENABLED) | ||
|
|
||
| /** | ||
| * A map from executorId to information on task failures. Tracks the time of each task failure, | ||
|
|
@@ -145,6 +146,74 @@ private[scheduler] class BlacklistTracker ( | |
| nextExpiryTime = math.min(execMinExpiry, nodeMinExpiry) | ||
| } | ||
|
|
||
| private def killBlacklistedExecutor(exec: String): Unit = { | ||
| if (conf.get(config.BLACKLIST_KILL_ENABLED)) { | ||
| allocationClient match { | ||
| case Some(a) => | ||
| logInfo(s"Killing blacklisted executor id $exec " + | ||
| s"since ${config.BLACKLIST_KILL_ENABLED.key} is set.") | ||
| a.killExecutors(Seq(exec), true, true) | ||
| case None => | ||
| logWarning(s"Not attempting to kill blacklisted executor id $exec " + | ||
| s"since allocation client is not defined.") | ||
| } | ||
| } | ||
| } | ||
|
|
||
| private def killExecutorsOnBlacklistedNode(node: String): Unit = { | ||
| if (conf.get(config.BLACKLIST_KILL_ENABLED)) { | ||
| allocationClient match { | ||
| case Some(a) => | ||
| logInfo(s"Killing all executors on blacklisted host $node " + | ||
| s"since ${config.BLACKLIST_KILL_ENABLED.key} is set.") | ||
| if (a.killExecutorsOnHost(node) == false) { | ||
| logError(s"Killing executors on node $node failed.") | ||
| } | ||
| case None => | ||
| logWarning(s"Not attempting to kill executors on blacklisted host $node " + | ||
| s"since allocation client is not defined.") | ||
| } | ||
| } | ||
| } | ||
|
|
||
| def updateBlacklistForFetchFailure(host: String, exec: String): Unit = { | ||
| if (BLACKLIST_FETCH_FAILURE_ENABLED) { | ||
| // If we blacklist on fetch failures, we are implicitly saying that we believe the failure is | ||
| // non-transient, and can't be recovered from (even if this is the first fetch failure, | ||
| // stage is retried after just one failure, so we don't always get a chance to collect | ||
| // multiple fetch failures). | ||
| // If the external shuffle-service is on, then every other executor on this node would | ||
| // be suffering from the same issue, so we should blacklist (and potentially kill) all | ||
| // of them immediately. | ||
|
|
||
| val now = clock.getTimeMillis() | ||
| val expiryTimeForNewBlacklists = now + BLACKLIST_TIMEOUT_MILLIS | ||
|
|
||
| if (conf.get(config.SHUFFLE_SERVICE_ENABLED)) { | ||
| if (!nodeIdToBlacklistExpiryTime.contains(host)) { | ||
| logInfo(s"blacklisting node $host due to fetch failure of external shuffle service") | ||
|
|
||
| nodeIdToBlacklistExpiryTime.put(host, expiryTimeForNewBlacklists) | ||
| listenerBus.post(SparkListenerNodeBlacklisted(now, host, 1)) | ||
| _nodeBlacklist.set(nodeIdToBlacklistExpiryTime.keySet.toSet) | ||
| killExecutorsOnBlacklistedNode(host) | ||
|
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. I think we need to call updateNextExpiryTime() here now since there isn't an explicit executorid being blacklisted which would have normally called it.
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. it would also be nice to make sure we have a test for the expiry. |
||
| updateNextExpiryTime() | ||
| } | ||
| } else if (!executorIdToBlacklistStatus.contains(exec)) { | ||
| logInfo(s"Blacklisting executor $exec due to fetch failure") | ||
|
|
||
| executorIdToBlacklistStatus.put(exec, BlacklistedExecutor(host, expiryTimeForNewBlacklists)) | ||
| // We hardcoded number of failure tasks to 1 for fetch failure, because there's no | ||
| // reattempt for such failure. | ||
| listenerBus.post(SparkListenerExecutorBlacklisted(now, exec, 1)) | ||
| updateNextExpiryTime() | ||
| killBlacklistedExecutor(exec) | ||
|
|
||
| val blacklistedExecsOnNode = nodeToBlacklistedExecs.getOrElseUpdate(exec, HashSet[String]()) | ||
| blacklistedExecsOnNode += exec | ||
| } | ||
| } | ||
| } | ||
|
|
||
| def updateBlacklistForSuccessfulTaskSet( | ||
| stageId: Int, | ||
|
|
@@ -174,17 +243,7 @@ private[scheduler] class BlacklistTracker ( | |
| listenerBus.post(SparkListenerExecutorBlacklisted(now, exec, newTotal)) | ||
| executorIdToFailureList.remove(exec) | ||
| updateNextExpiryTime() | ||
| if (conf.get(config.BLACKLIST_KILL_ENABLED)) { | ||
| allocationClient match { | ||
| case Some(allocationClient) => | ||
| logInfo(s"Killing blacklisted executor id $exec " + | ||
| s"since spark.blacklist.killBlacklistedExecutors is set.") | ||
| allocationClient.killExecutors(Seq(exec), true, true) | ||
| case None => | ||
| logWarning(s"Not attempting to kill blacklisted executor id $exec " + | ||
| s"since allocation client is not defined.") | ||
| } | ||
| } | ||
| killBlacklistedExecutor(exec) | ||
|
|
||
| // In addition to blacklisting the executor, we also update the data for failures on the | ||
| // node, and potentially put the entire node into a blacklist as well. | ||
|
|
@@ -199,19 +258,7 @@ private[scheduler] class BlacklistTracker ( | |
| nodeIdToBlacklistExpiryTime.put(node, expiryTimeForNewBlacklists) | ||
| listenerBus.post(SparkListenerNodeBlacklisted(now, node, blacklistedExecsOnNode.size)) | ||
| _nodeBlacklist.set(nodeIdToBlacklistExpiryTime.keySet.toSet) | ||
| if (conf.get(config.BLACKLIST_KILL_ENABLED)) { | ||
| allocationClient match { | ||
| case Some(allocationClient) => | ||
| logInfo(s"Killing all executors on blacklisted host $node " + | ||
| s"since spark.blacklist.killBlacklistedExecutors is set.") | ||
| if (allocationClient.killExecutorsOnHost(node) == false) { | ||
| logError(s"Killing executors on node $node failed.") | ||
| } | ||
| case None => | ||
| logWarning(s"Not attempting to kill executors on blacklisted host $node " + | ||
| s"since allocation client is not defined.") | ||
| } | ||
| } | ||
| killExecutorsOnBlacklistedNode(node) | ||
| } | ||
| } | ||
| } | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -529,4 +529,59 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M | |
| verify(allocationClientMock).killExecutors(Seq("2"), true, true) | ||
| verify(allocationClientMock).killExecutorsOnHost("hostA") | ||
| } | ||
|
|
||
| test("fetch failure blacklisting kills executors, configured by BLACKLIST_KILL_ENABLED") { | ||
| val allocationClientMock = mock[ExecutorAllocationClient] | ||
| when(allocationClientMock.killExecutors(any(), any(), any())).thenReturn(Seq("called")) | ||
| when(allocationClientMock.killExecutorsOnHost("hostA")).thenAnswer(new Answer[Boolean] { | ||
| // To avoid a race between blacklisting and killing, it is important that the nodeBlacklist | ||
| // is updated before we ask the executor allocation client to kill all the executors | ||
| // on a particular host. | ||
| override def answer(invocation: InvocationOnMock): Boolean = { | ||
| if (blacklist.nodeBlacklist.contains("hostA") == false) { | ||
| throw new IllegalStateException("hostA should be on the blacklist") | ||
| } | ||
| true | ||
| } | ||
| }) | ||
|
|
||
| conf.set(config.BLACKLIST_FETCH_FAILURE_ENABLED, true) | ||
| blacklist = new BlacklistTracker(listenerBusMock, conf, Some(allocationClientMock), clock) | ||
|
|
||
| // Disable auto-kill. Blacklist an executor and make sure killExecutors is not called. | ||
| conf.set(config.BLACKLIST_KILL_ENABLED, false) | ||
| blacklist.updateBlacklistForFetchFailure("hostA", exec = "1") | ||
|
|
||
| verify(allocationClientMock, never).killExecutors(any(), any(), any()) | ||
| verify(allocationClientMock, never).killExecutorsOnHost(any()) | ||
|
|
||
| // Enable auto-kill. Blacklist an executor and make sure killExecutors is called. | ||
| conf.set(config.BLACKLIST_KILL_ENABLED, true) | ||
| blacklist = new BlacklistTracker(listenerBusMock, conf, Some(allocationClientMock), clock) | ||
| clock.advance(1000) | ||
| blacklist.updateBlacklistForFetchFailure("hostA", exec = "1") | ||
|
|
||
| verify(allocationClientMock).killExecutors(Seq("1"), true, true) | ||
| verify(allocationClientMock, never).killExecutorsOnHost(any()) | ||
|
|
||
| assert(blacklist.executorIdToBlacklistStatus.contains("1")) | ||
| assert(blacklist.executorIdToBlacklistStatus("1").node === "hostA") | ||
| assert(blacklist.executorIdToBlacklistStatus("1").expiryTime === | ||
| 1000 + blacklist.BLACKLIST_TIMEOUT_MILLIS) | ||
| assert(blacklist.nextExpiryTime === 1000 + blacklist.BLACKLIST_TIMEOUT_MILLIS) | ||
| assert(blacklist.nodeIdToBlacklistExpiryTime.isEmpty) | ||
|
|
||
| // Enable external shuffle service to see if all the executors on this node will be killed. | ||
| conf.set(config.SHUFFLE_SERVICE_ENABLED, true) | ||
| clock.advance(1000) | ||
| blacklist.updateBlacklistForFetchFailure("hostA", exec = "2") | ||
|
|
||
| verify(allocationClientMock, never).killExecutors(Seq("2"), true, true) | ||
| verify(allocationClientMock).killExecutorsOnHost("hostA") | ||
|
|
||
| assert(blacklist.nodeIdToBlacklistExpiryTime.contains("hostA")) | ||
| assert(blacklist.nodeIdToBlacklistExpiryTime("hostA") === | ||
| 2000 + blacklist.BLACKLIST_TIMEOUT_MILLIS) | ||
| assert(blacklist.nextExpiryTime === 1000 + blacklist.BLACKLIST_TIMEOUT_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. Should we also test with SHUFFLE_SERVICE_ENABLED=true and BLACKLIST_KILL_ENABLED=false ?
Contributor
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. If BLACKLIST_KILL_ENABLED=false, the scenario should be the same as here. It looks like a duplication not so necessary?
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. make sense. |
||
| } | ||
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.
Why not make this a private val?
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.
It might be a just self-preferred choice, this code was not written by me, I just made some refactoring.