diff --git a/core/src/main/java/org/apache/spark/SparkFirehoseListener.java b/core/src/main/java/org/apache/spark/SparkFirehoseListener.java index c0e72b57d48bd..7cb2455affe48 100644 --- a/core/src/main/java/org/apache/spark/SparkFirehoseListener.java +++ b/core/src/main/java/org/apache/spark/SparkFirehoseListener.java @@ -17,6 +17,7 @@ package org.apache.spark; +import org.apache.spark.annotation.DeveloperApi; import org.apache.spark.scheduler.*; /** @@ -27,7 +28,11 @@ * new methods to SparkListener: forgetting to add a method will result in a compilation error (if * this was a concrete Scala class, default implementations of new event handlers would be inherited * from the SparkListener trait). + * + * Please note until Spark 3.1.0 this was missing the DevelopApi annotation, this needs to be + * taken into account if changing this API before a major release. */ +@DeveloperApi public class SparkFirehoseListener implements SparkListenerInterface { public void onEvent(SparkListenerEvent event) { } @@ -124,34 +129,67 @@ public final void onExecutorBlacklisted(SparkListenerExecutorBlacklisted executo onEvent(executorBlacklisted); } + @Override + public final void onExecutorExcluded(SparkListenerExecutorExcluded executorExcluded) { + onEvent(executorExcluded); + } + @Override public void onExecutorBlacklistedForStage( SparkListenerExecutorBlacklistedForStage executorBlacklistedForStage) { onEvent(executorBlacklistedForStage); } + @Override + public void onExecutorExcludedForStage( + SparkListenerExecutorExcludedForStage executorExcludedForStage) { + onEvent(executorExcludedForStage); + } + @Override public void onNodeBlacklistedForStage( SparkListenerNodeBlacklistedForStage nodeBlacklistedForStage) { onEvent(nodeBlacklistedForStage); } + @Override + public void onNodeExcludedForStage( + SparkListenerNodeExcludedForStage nodeExcludedForStage) { + onEvent(nodeExcludedForStage); + } + @Override public final void onExecutorUnblacklisted( SparkListenerExecutorUnblacklisted executorUnblacklisted) { onEvent(executorUnblacklisted); } + @Override + public final void onExecutorUnexcluded( + SparkListenerExecutorUnexcluded executorUnexcluded) { + onEvent(executorUnexcluded); + } + @Override public final void onNodeBlacklisted(SparkListenerNodeBlacklisted nodeBlacklisted) { onEvent(nodeBlacklisted); } + @Override + public final void onNodeExcluded(SparkListenerNodeExcluded nodeExcluded) { + onEvent(nodeExcluded); + } + @Override public final void onNodeUnblacklisted(SparkListenerNodeUnblacklisted nodeUnblacklisted) { onEvent(nodeUnblacklisted); } + @Override + public final void onNodeUnexcluded(SparkListenerNodeUnexcluded nodeUnexcluded) { + onEvent(nodeUnexcluded); + } + @Override public void onBlockUpdated(SparkListenerBlockUpdated blockUpdated) { onEvent(blockUpdated); diff --git a/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html b/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html index 0729dfe1cef72..5e835c053eb6c 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html +++ b/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html @@ -56,8 +56,8 @@

Summary

- Blacklisted + title="Number of executors excluded by the scheduler due to task failures."> + Excluded diff --git a/core/src/main/resources/org/apache/spark/ui/static/executorspage.js b/core/src/main/resources/org/apache/spark/ui/static/executorspage.js index 520edb9cc3e34..d4eaea9103771 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/executorspage.js +++ b/core/src/main/resources/org/apache/spark/ui/static/executorspage.js @@ -26,15 +26,15 @@ function getThreadDumpEnabled() { } function formatStatus(status, type, row) { - if (row.isBlacklisted) { - return "Blacklisted"; + if (row.isExcluded) { + return "Excluded"; } if (status) { - if (row.blacklistedInStages.length == 0) { + if (row.excludedInStages.length == 0) { return "Active" } - return "Active (Blacklisted in Stages: [" + row.blacklistedInStages.join(", ") + "])"; + return "Active (Excluded in Stages: [" + row.excludedInStages.join(", ") + "])"; } return "Dead" } @@ -168,7 +168,7 @@ $(document).ready(function () { var allTotalInputBytes = 0; var allTotalShuffleRead = 0; var allTotalShuffleWrite = 0; - var allTotalBlacklisted = 0; + var allTotalExcluded = 0; var activeExecCnt = 0; var activeRDDBlocks = 0; @@ -190,7 +190,7 @@ $(document).ready(function () { var activeTotalInputBytes = 0; var activeTotalShuffleRead = 0; var activeTotalShuffleWrite = 0; - var activeTotalBlacklisted = 0; + var activeTotalExcluded = 0; var deadExecCnt = 0; var deadRDDBlocks = 0; @@ -212,7 +212,7 @@ $(document).ready(function () { var deadTotalInputBytes = 0; var deadTotalShuffleRead = 0; var deadTotalShuffleWrite = 0; - var deadTotalBlacklisted = 0; + var deadTotalExcluded = 0; response.forEach(function (exec) { var memoryMetrics = { @@ -246,7 +246,7 @@ $(document).ready(function () { allTotalInputBytes += exec.totalInputBytes; allTotalShuffleRead += exec.totalShuffleRead; allTotalShuffleWrite += exec.totalShuffleWrite; - allTotalBlacklisted += exec.isBlacklisted ? 1 : 0; + allTotalExcluded += exec.isExcluded ? 1 : 0; if (exec.isActive) { activeExecCnt += 1; activeRDDBlocks += exec.rddBlocks; @@ -268,7 +268,7 @@ $(document).ready(function () { activeTotalInputBytes += exec.totalInputBytes; activeTotalShuffleRead += exec.totalShuffleRead; activeTotalShuffleWrite += exec.totalShuffleWrite; - activeTotalBlacklisted += exec.isBlacklisted ? 1 : 0; + activeTotalExcluded += exec.isExcluded ? 1 : 0; } else { deadExecCnt += 1; deadRDDBlocks += exec.rddBlocks; @@ -290,7 +290,7 @@ $(document).ready(function () { deadTotalInputBytes += exec.totalInputBytes; deadTotalShuffleRead += exec.totalShuffleRead; deadTotalShuffleWrite += exec.totalShuffleWrite; - deadTotalBlacklisted += exec.isBlacklisted ? 1 : 0; + deadTotalExcluded += exec.isExcluded ? 1 : 0; // todo - TEST BACKWARDS compatibility history? } }); @@ -315,7 +315,7 @@ $(document).ready(function () { "allTotalInputBytes": allTotalInputBytes, "allTotalShuffleRead": allTotalShuffleRead, "allTotalShuffleWrite": allTotalShuffleWrite, - "allTotalBlacklisted": allTotalBlacklisted + "allTotalExcluded": allTotalExcluded }; var activeSummary = { "execCnt": ( "Active(" + activeExecCnt + ")"), @@ -338,7 +338,7 @@ $(document).ready(function () { "allTotalInputBytes": activeTotalInputBytes, "allTotalShuffleRead": activeTotalShuffleRead, "allTotalShuffleWrite": activeTotalShuffleWrite, - "allTotalBlacklisted": activeTotalBlacklisted + "allTotalExcluded": activeTotalExcluded }; var deadSummary = { "execCnt": ( "Dead(" + deadExecCnt + ")" ), @@ -361,7 +361,7 @@ $(document).ready(function () { "allTotalInputBytes": deadTotalInputBytes, "allTotalShuffleRead": deadTotalShuffleRead, "allTotalShuffleWrite": deadTotalShuffleWrite, - "allTotalBlacklisted": deadTotalBlacklisted + "allTotalExcluded": deadTotalExcluded }; var data = {executors: response, "execSummary": [activeSummary, deadSummary, totalSummary]}; @@ -547,7 +547,7 @@ $(document).ready(function () { {data: 'allTotalInputBytes', render: formatBytes}, {data: 'allTotalShuffleRead', render: formatBytes}, {data: 'allTotalShuffleWrite', render: formatBytes}, - {data: 'allTotalBlacklisted'} + {data: 'allTotalExcluded'} ], "paging": false, "searching": false, diff --git a/core/src/main/resources/org/apache/spark/ui/static/stagepage.js b/core/src/main/resources/org/apache/spark/ui/static/stagepage.js index 93b37c296271b..ee1115868f69b 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/stagepage.js +++ b/core/src/main/resources/org/apache/spark/ui/static/stagepage.js @@ -433,7 +433,7 @@ $(document).ready(function () { {data : "failedTasks"}, {data : "killedTasks"}, {data : "succeededTasks"}, - {data : "isBlacklistedForStage"}, + {data : "isExcludedForStage"}, { data : function (row, type) { return row.inputRecords != 0 ? formatBytes(row.inputBytes, type) + " / " + row.inputRecords : ""; diff --git a/core/src/main/resources/org/apache/spark/ui/static/stagespage-template.html b/core/src/main/resources/org/apache/spark/ui/static/stagespage-template.html index 77ea70e4ad966..9b40d0dc4a230 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/stagespage-template.html +++ b/core/src/main/resources/org/apache/spark/ui/static/stagespage-template.html @@ -50,8 +50,8 @@

Aggregated Metrics by Executor

Succeeded Tasks - Blacklisted + title="Shows if this executor has been excluded by the scheduler due to task failures."> + Excluded Input Size / Records Output Size / Records diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala index 1dd64df106bc2..e445f188e1eed 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -312,8 +312,8 @@ private[spark] class ExecutorAllocationManager( if (unschedulableTaskSets > 0) { // Request additional executors to account for task sets having tasks that are unschedulable - // due to blacklisting when the active executor count has already reached the max needed - // which we would normally get. + // due to executors excluded for failures when the active executor count has already reached + // the max needed which we would normally get. val maxNeededForUnschedulables = math.ceil(unschedulableTaskSets * executorAllocationRatio / tasksPerExecutor).toInt math.max(maxNeededWithSpeculationLocalityOffset, @@ -662,10 +662,10 @@ private[spark] class ExecutorAllocationManager( private val resourceProfileIdToStageAttempt = new mutable.HashMap[Int, mutable.Set[StageAttempt]] - // Keep track of unschedulable task sets due to blacklisting. This is a Set of StageAttempt's - // because we'll only take the last unschedulable task in a taskset although there can be more. - // This is done in order to avoid costly loops in the scheduling. - // Check TaskSetManager#getCompletelyBlacklistedTaskIfAny for more details. + // Keep track of unschedulable task sets because of executor/node exclusions from too many task + // failures. This is a Set of StageAttempt's because we'll only take the last unschedulable task + // in a taskset although there can be more. This is done in order to avoid costly loops in the + // scheduling. Check TaskSetManager#getCompletelyExcludedTaskIfAny for more details. private val unschedulableTaskSets = new mutable.HashSet[StageAttempt] // stageAttempt to tuple (the number of task with locality preferences, a map where each pair diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 427e98e616515..5f37a1abb1909 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -603,7 +603,7 @@ private[spark] object SparkConf extends Logging { "are no longer accepted. To specify the equivalent now, one may use '64k'."), DeprecatedConfig("spark.rpc", "2.0", "Not used anymore."), DeprecatedConfig("spark.scheduler.executorTaskBlacklistTime", "2.1.0", - "Please use the new blacklisting options, spark.blacklist.*"), + "Please use the new excludedOnFailure options, spark.excludeOnFailure.*"), DeprecatedConfig("spark.yarn.am.port", "2.0.0", "Not used anymore"), DeprecatedConfig("spark.executor.port", "2.0.0", "Not used anymore"), DeprecatedConfig("spark.shuffle.service.index.cache.entries", "2.3.0", @@ -612,7 +612,31 @@ private[spark] object SparkConf extends Logging { DeprecatedConfig("spark.yarn.credentials.file.retention.days", "2.4.0", "Not used anymore."), DeprecatedConfig("spark.yarn.services", "3.0.0", "Feature no longer available."), DeprecatedConfig("spark.executor.plugins", "3.0.0", - "Feature replaced with new plugin API. See Monitoring documentation.") + "Feature replaced with new plugin API. See Monitoring documentation."), + DeprecatedConfig("spark.blacklist.enabled", "3.1.0", + "Please use spark.excludeOnFailure.enabled"), + DeprecatedConfig("spark.blacklist.task.maxTaskAttemptsPerExecutor", "3.1.0", + "Please use spark.excludeOnFailure.task.maxTaskAttemptsPerExecutor"), + DeprecatedConfig("spark.blacklist.task.maxTaskAttemptsPerNode", "3.1.0", + "Please use spark.excludeOnFailure.task.maxTaskAttemptsPerNode"), + DeprecatedConfig("spark.blacklist.application.maxFailedTasksPerExecutor", "3.1.0", + "Please use spark.excludeOnFailure.application.maxFailedTasksPerExecutor"), + DeprecatedConfig("spark.blacklist.stage.maxFailedTasksPerExecutor", "3.1.0", + "Please use spark.excludeOnFailure.stage.maxFailedTasksPerExecutor"), + DeprecatedConfig("spark.blacklist.application.maxFailedExecutorsPerNode", "3.1.0", + "Please use spark.excludeOnFailure.application.maxFailedExecutorsPerNode"), + DeprecatedConfig("spark.blacklist.stage.maxFailedExecutorsPerNode", "3.1.0", + "Please use spark.excludeOnFailure.stage.maxFailedExecutorsPerNode"), + DeprecatedConfig("spark.blacklist.timeout", "3.1.0", + "Please use spark.excludeOnFailure.timeout"), + DeprecatedConfig("spark.blacklist.application.fetchFailure.enabled", "3.1.0", + "Please use spark.excludeOnFailure.application.fetchFailure.enabled"), + DeprecatedConfig("spark.scheduler.blacklist.unschedulableTaskSetTimeout", "3.1.0", + "Please use spark.scheduler.excludeOnFailure.unschedulableTaskSetTimeout"), + DeprecatedConfig("spark.blacklist.killBlacklistedExecutors", "3.1.0", + "Please use spark.excludeOnFailure.killExcludedExecutors"), + DeprecatedConfig("spark.yarn.blacklist.executor.launch.blacklisting.enabled", "3.1.0", + "Please use spark.yarn.executor.launch.excludeOnFailure.enabled") ) Map(configs.map { cfg => (cfg.key -> cfg) } : _*) diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala index 6606d317e7b86..b304eb97fbdf6 100644 --- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala +++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala @@ -98,10 +98,11 @@ case class FetchFailed( /** * Fetch failures lead to a different failure handling path: (1) we don't abort the stage after * 4 task failures, instead we immediately go back to the stage which generated the map output, - * and regenerate the missing data. (2) we don't count fetch failures for blacklisting, since - * presumably its not the fault of the executor where the task ran, but the executor which - * stored the data. This is especially important because we might rack up a bunch of - * fetch-failures in rapid succession, on all nodes of the cluster, due to one bad node. + * and regenerate the missing data. (2) we don't count fetch failures from executors excluded + * due to too many task failures, since presumably its not the fault of the executor where + * the task ran, but the executor which stored the data. This is especially important because + * we might rack up a bunch of fetch-failures in rapid succession, on all nodes of the cluster, + * due to one bad node. */ override def countTowardsTaskFailures: Boolean = false } diff --git a/core/src/main/scala/org/apache/spark/deploy/history/BasicEventFilterBuilder.scala b/core/src/main/scala/org/apache/spark/deploy/history/BasicEventFilterBuilder.scala index b18bf2665d6ce..c659d32d16314 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/BasicEventFilterBuilder.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/BasicEventFilterBuilder.scala @@ -160,6 +160,8 @@ private[spark] class BasicEventFilter( case e: SparkListenerExecutorRemoved => liveExecutors.contains(e.executorId) case e: SparkListenerExecutorBlacklisted => liveExecutors.contains(e.executorId) case e: SparkListenerExecutorUnblacklisted => liveExecutors.contains(e.executorId) + case e: SparkListenerExecutorExcluded => liveExecutors.contains(e.executorId) + case e: SparkListenerExecutorUnexcluded => liveExecutors.contains(e.executorId) case e: SparkListenerStageExecutorMetrics => liveExecutors.contains(e.execId) case e: SparkListenerBlockManagerAdded => acceptBlockManagerEvent(e.blockManagerId) case e: SparkListenerBlockManagerRemoved => acceptBlockManagerEvent(e.blockManagerId) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryAppStatusStore.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryAppStatusStore.scala index 7973652b3e254..ac0f102d81a6a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryAppStatusStore.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryAppStatusStore.scala @@ -73,7 +73,7 @@ private[spark] class HistoryAppStatusStore( source.totalShuffleWrite, source.isBlacklisted, source.maxMemory, source.addTime, source.removeTime, source.removeReason, newExecutorLogs, source.memoryMetrics, source.blacklistedInStages, source.peakMemoryMetrics, source.attributes, source.resources, - source.resourceProfileId) + source.resourceProfileId, source.isExcluded, source.excludedInStages) } } diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 491395c3cbcde..6239ef0491a6f 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -722,74 +722,83 @@ package object config { .booleanConf .createWithDefault(true) - // Blacklist confs - private[spark] val BLACKLIST_ENABLED = - ConfigBuilder("spark.blacklist.enabled") - .version("2.1.0") + private[spark] val EXCLUDE_ON_FAILURE_ENABLED = + ConfigBuilder("spark.excludeOnFailure.enabled") + .version("3.1.0") + .withAlternative("spark.blacklist.enabled") .booleanConf .createOptional private[spark] val MAX_TASK_ATTEMPTS_PER_EXECUTOR = - ConfigBuilder("spark.blacklist.task.maxTaskAttemptsPerExecutor") - .version("2.1.0") + ConfigBuilder("spark.excludeOnFailure.task.maxTaskAttemptsPerExecutor") + .version("3.1.0") + .withAlternative("spark.blacklist.task.maxTaskAttemptsPerExecutor") .intConf .createWithDefault(1) private[spark] val MAX_TASK_ATTEMPTS_PER_NODE = - ConfigBuilder("spark.blacklist.task.maxTaskAttemptsPerNode") - .version("2.1.0") + ConfigBuilder("spark.excludeOnFailure.task.maxTaskAttemptsPerNode") + .version("3.1.0") + .withAlternative("spark.blacklist.task.maxTaskAttemptsPerNode") .intConf .createWithDefault(2) private[spark] val MAX_FAILURES_PER_EXEC = - ConfigBuilder("spark.blacklist.application.maxFailedTasksPerExecutor") - .version("2.2.0") + ConfigBuilder("spark.excludeOnFailure.application.maxFailedTasksPerExecutor") + .version("3.1.0") + .withAlternative("spark.blacklist.application.maxFailedTasksPerExecutor") .intConf .createWithDefault(2) private[spark] val MAX_FAILURES_PER_EXEC_STAGE = - ConfigBuilder("spark.blacklist.stage.maxFailedTasksPerExecutor") - .version("2.1.0") + ConfigBuilder("spark.excludeOnFailure.stage.maxFailedTasksPerExecutor") + .version("3.1.0") + .withAlternative("spark.blacklist.stage.maxFailedTasksPerExecutor") .intConf .createWithDefault(2) private[spark] val MAX_FAILED_EXEC_PER_NODE = - ConfigBuilder("spark.blacklist.application.maxFailedExecutorsPerNode") - .version("2.2.0") + ConfigBuilder("spark.excludeOnFailure.application.maxFailedExecutorsPerNode") + .version("3.1.0") + .withAlternative("spark.blacklist.application.maxFailedExecutorsPerNode") .intConf .createWithDefault(2) private[spark] val MAX_FAILED_EXEC_PER_NODE_STAGE = - ConfigBuilder("spark.blacklist.stage.maxFailedExecutorsPerNode") - .version("2.1.0") + ConfigBuilder("spark.excludeOnFailure.stage.maxFailedExecutorsPerNode") + .version("3.1.0") + .withAlternative("spark.blacklist.stage.maxFailedExecutorsPerNode") .intConf .createWithDefault(2) - private[spark] val BLACKLIST_TIMEOUT_CONF = - ConfigBuilder("spark.blacklist.timeout") - .version("2.1.0") + private[spark] val EXCLUDE_ON_FAILURE_TIMEOUT_CONF = + ConfigBuilder("spark.excludeOnFailure.timeout") + .version("3.1.0") + .withAlternative("spark.blacklist.timeout") .timeConf(TimeUnit.MILLISECONDS) .createOptional - private[spark] val BLACKLIST_KILL_ENABLED = - ConfigBuilder("spark.blacklist.killBlacklistedExecutors") - .version("2.2.0") + private[spark] val EXCLUDE_ON_FAILURE_KILL_ENABLED = + ConfigBuilder("spark.excludeOnFailure.killExcludedExecutors") + .version("3.1.0") + .withAlternative("spark.blacklist.killBlacklistedExecutors") .booleanConf .createWithDefault(false) - private[spark] val BLACKLIST_LEGACY_TIMEOUT_CONF = - ConfigBuilder("spark.scheduler.executorTaskBlacklistTime") + private[spark] val EXCLUDE_ON_FAILURE_LEGACY_TIMEOUT_CONF = + ConfigBuilder("spark.scheduler.executorTaskExcludeOnFailureTime") .internal() - .version("1.0.0") + .version("3.1.0") + .withAlternative("spark.scheduler.executorTaskBlacklistTime") .timeConf(TimeUnit.MILLISECONDS) .createOptional - private[spark] val BLACKLIST_FETCH_FAILURE_ENABLED = - ConfigBuilder("spark.blacklist.application.fetchFailure.enabled") - .version("2.3.0") + private[spark] val EXCLUDE_ON_FAILURE_FETCH_FAILURE_ENABLED = + ConfigBuilder("spark.excludeOnFailure.application.fetchFailure.enabled") + .version("3.1.0") + .withAlternative("spark.blacklist.application.fetchFailure.enabled") .booleanConf .createWithDefault(false) - // End blacklist confs private[spark] val UNREGISTER_OUTPUT_ON_HOST_ON_FETCH_FAILURE = ConfigBuilder("spark.files.fetchFailure.unRegisterOutputOnHost") @@ -1453,10 +1462,12 @@ package object config { .createWithDefaultString("365d") private[spark] val UNSCHEDULABLE_TASKSET_TIMEOUT = - ConfigBuilder("spark.scheduler.blacklist.unschedulableTaskSetTimeout") + ConfigBuilder("spark.scheduler.excludeOnFailure.unschedulableTaskSetTimeout") .doc("The timeout in seconds to wait to acquire a new executor and schedule a task " + - "before aborting a TaskSet which is unschedulable because of being completely blacklisted.") - .version("2.4.1") + "before aborting a TaskSet which is unschedulable because all executors are " + + "excluded due to failures.") + .version("3.1.0") + .withAlternative("spark.scheduler.blacklist.unschedulableTaskSetTimeout") .timeConf(TimeUnit.SECONDS) .checkValue(v => v >= 0, "The value should be a non negative time value.") .createWithDefault(120) diff --git a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala deleted file mode 100644 index 9e524c52267be..0000000000000 --- a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala +++ /dev/null @@ -1,477 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.scheduler - -import java.util.concurrent.atomic.AtomicReference - -import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} - -import org.apache.spark.{ExecutorAllocationClient, SparkConf, SparkContext} -import org.apache.spark.internal.Logging -import org.apache.spark.internal.config -import org.apache.spark.util.{Clock, SystemClock, Utils} - -/** - * BlacklistTracker is designed to track problematic executors and nodes. It supports blacklisting - * executors and nodes across an entire application (with a periodic expiry). TaskSetManagers add - * additional blacklisting of executors and nodes for individual tasks and stages which works in - * concert with the blacklisting here. - * - * The tracker needs to deal with a variety of workloads, eg.: - * - * * bad user code -- this may lead to many task failures, but that should not count against - * individual executors - * * many small stages -- this may prevent a bad executor for having many failures within one - * stage, but still many failures over the entire application - * * "flaky" executors -- they don't fail every task, but are still faulty enough to merit - * blacklisting - * - * See the design doc on SPARK-8425 for a more in-depth discussion. - * - * THREADING: As with most helpers of TaskSchedulerImpl, this is not thread-safe. Though it is - * called by multiple threads, callers must already have a lock on the TaskSchedulerImpl. The - * one exception is [[nodeBlacklist()]], which can be called without holding a lock. - */ -private[scheduler] class BlacklistTracker ( - private val listenerBus: LiveListenerBus, - conf: SparkConf, - allocationClient: Option[ExecutorAllocationClient], - clock: Clock = new SystemClock()) extends Logging { - - def this(sc: SparkContext, allocationClient: Option[ExecutorAllocationClient]) = { - this(sc.listenerBus, sc.conf, allocationClient) - } - - BlacklistTracker.validateBlacklistConfs(conf) - 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, - * so that we can avoid blacklisting executors due to failures that are very far apart. We do not - * actively remove from this as soon as tasks hit their timeouts, to avoid the time it would take - * to do so. But it will not grow too large, because as soon as an executor gets too many - * failures, we blacklist the executor and remove its entry here. - */ - private val executorIdToFailureList = new HashMap[String, ExecutorFailureList]() - val executorIdToBlacklistStatus = new HashMap[String, BlacklistedExecutor]() - val nodeIdToBlacklistExpiryTime = new HashMap[String, Long]() - /** - * An immutable copy of the set of nodes that are currently blacklisted. Kept in an - * AtomicReference to make [[nodeBlacklist()]] thread-safe. - */ - private val _nodeBlacklist = new AtomicReference[Set[String]](Set()) - /** - * Time when the next blacklist will expire. Used as a - * shortcut to avoid iterating over all entries in the blacklist when none will have expired. - */ - var nextExpiryTime: Long = Long.MaxValue - /** - * Mapping from nodes to all of the executors that have been blacklisted on that node. We do *not* - * remove from this when executors are removed from spark, so we can track when we get multiple - * successive blacklisted executors on one node. Nonetheless, it will not grow too large because - * there cannot be many blacklisted executors on one node, before we stop requesting more - * executors on that node, and we clean up the list of blacklisted executors once an executor has - * been blacklisted for BLACKLIST_TIMEOUT_MILLIS. - */ - val nodeToBlacklistedExecs = new HashMap[String, HashSet[String]]() - - /** - * Un-blacklists executors and nodes that have been blacklisted for at least - * BLACKLIST_TIMEOUT_MILLIS - */ - def applyBlacklistTimeout(): Unit = { - val now = clock.getTimeMillis() - // quickly check if we've got anything to expire from blacklist -- if not, avoid doing any work - if (now > nextExpiryTime) { - // Apply the timeout to blacklisted nodes and executors - val execsToUnblacklist = executorIdToBlacklistStatus.filter(_._2.expiryTime < now).keys - if (execsToUnblacklist.nonEmpty) { - // Un-blacklist any executors that have been blacklisted longer than the blacklist timeout. - logInfo(s"Removing executors $execsToUnblacklist from blacklist because the blacklist " + - s"for those executors has timed out") - execsToUnblacklist.foreach { exec => - val status = executorIdToBlacklistStatus.remove(exec).get - val failedExecsOnNode = nodeToBlacklistedExecs(status.node) - listenerBus.post(SparkListenerExecutorUnblacklisted(now, exec)) - failedExecsOnNode.remove(exec) - if (failedExecsOnNode.isEmpty) { - nodeToBlacklistedExecs.remove(status.node) - } - } - } - val nodesToUnblacklist = nodeIdToBlacklistExpiryTime.filter(_._2 < now).keys - if (nodesToUnblacklist.nonEmpty) { - // Un-blacklist any nodes that have been blacklisted longer than the blacklist timeout. - logInfo(s"Removing nodes $nodesToUnblacklist from blacklist because the blacklist " + - s"has timed out") - nodesToUnblacklist.foreach { node => - nodeIdToBlacklistExpiryTime.remove(node) - listenerBus.post(SparkListenerNodeUnblacklisted(now, node)) - } - _nodeBlacklist.set(nodeIdToBlacklistExpiryTime.keySet.toSet) - } - updateNextExpiryTime() - } - } - - private def updateNextExpiryTime(): Unit = { - val execMinExpiry = if (executorIdToBlacklistStatus.nonEmpty) { - executorIdToBlacklistStatus.map{_._2.expiryTime}.min - } else { - Long.MaxValue - } - val nodeMinExpiry = if (nodeIdToBlacklistExpiryTime.nonEmpty) { - nodeIdToBlacklistExpiryTime.values.min - } else { - Long.MaxValue - } - nextExpiryTime = math.min(execMinExpiry, nodeMinExpiry) - } - - private def killExecutor(exec: String, msg: String): Unit = { - allocationClient match { - case Some(a) => - logInfo(msg) - a.killExecutors(Seq(exec), adjustTargetNumExecutors = false, countFailures = false, - force = true) - case None => - logInfo(s"Not attempting to kill blacklisted executor id $exec " + - s"since allocation client is not defined.") - } - } - - private def killBlacklistedExecutor(exec: String): Unit = { - if (conf.get(config.BLACKLIST_KILL_ENABLED)) { - killExecutor(exec, - s"Killing blacklisted executor id $exec since ${config.BLACKLIST_KILL_ENABLED.key} is set.") - } - } - - private[scheduler] def killBlacklistedIdleExecutor(exec: String): Unit = { - killExecutor(exec, - s"Killing blacklisted idle executor id $exec because of task unschedulability and trying " + - "to acquire a new executor.") - } - - 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) - 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(host, HashSet[String]()) - blacklistedExecsOnNode += exec - } - } - } - - def updateBlacklistForSuccessfulTaskSet( - stageId: Int, - stageAttemptId: Int, - failuresByExec: HashMap[String, ExecutorFailuresInTaskSet]): Unit = { - // if any tasks failed, we count them towards the overall failure count for the executor at - // this point. - val now = clock.getTimeMillis() - failuresByExec.foreach { case (exec, failuresInTaskSet) => - val appFailuresOnExecutor = - executorIdToFailureList.getOrElseUpdate(exec, new ExecutorFailureList) - appFailuresOnExecutor.addFailures(stageId, stageAttemptId, failuresInTaskSet) - appFailuresOnExecutor.dropFailuresWithTimeoutBefore(now) - val newTotal = appFailuresOnExecutor.numUniqueTaskFailures - - val expiryTimeForNewBlacklists = now + BLACKLIST_TIMEOUT_MILLIS - // If this pushes the total number of failures over the threshold, blacklist the executor. - // If its already blacklisted, we avoid "re-blacklisting" (which can happen if there were - // other tasks already running in another taskset when it got blacklisted), because it makes - // some of the logic around expiry times a little more confusing. But it also wouldn't be a - // problem to re-blacklist, with a later expiry time. - if (newTotal >= MAX_FAILURES_PER_EXEC && !executorIdToBlacklistStatus.contains(exec)) { - logInfo(s"Blacklisting executor id: $exec because it has $newTotal" + - s" task failures in successful task sets") - val node = failuresInTaskSet.node - executorIdToBlacklistStatus.put(exec, BlacklistedExecutor(node, expiryTimeForNewBlacklists)) - listenerBus.post(SparkListenerExecutorBlacklisted(now, exec, newTotal)) - executorIdToFailureList.remove(exec) - updateNextExpiryTime() - 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. - val blacklistedExecsOnNode = nodeToBlacklistedExecs.getOrElseUpdate(node, HashSet[String]()) - blacklistedExecsOnNode += exec - // If the node is already in the blacklist, we avoid adding it again with a later expiry - // time. - if (blacklistedExecsOnNode.size >= MAX_FAILED_EXEC_PER_NODE && - !nodeIdToBlacklistExpiryTime.contains(node)) { - logInfo(s"Blacklisting node $node because it has ${blacklistedExecsOnNode.size} " + - s"executors blacklisted: ${blacklistedExecsOnNode}") - nodeIdToBlacklistExpiryTime.put(node, expiryTimeForNewBlacklists) - listenerBus.post(SparkListenerNodeBlacklisted(now, node, blacklistedExecsOnNode.size)) - _nodeBlacklist.set(nodeIdToBlacklistExpiryTime.keySet.toSet) - killExecutorsOnBlacklistedNode(node) - } - } - } - } - - def isExecutorBlacklisted(executorId: String): Boolean = { - executorIdToBlacklistStatus.contains(executorId) - } - - /** - * Get the full set of nodes that are blacklisted. Unlike other methods in this class, this *IS* - * thread-safe -- no lock required on a taskScheduler. - */ - def nodeBlacklist(): Set[String] = { - _nodeBlacklist.get() - } - - def isNodeBlacklisted(node: String): Boolean = { - nodeIdToBlacklistExpiryTime.contains(node) - } - - def handleRemovedExecutor(executorId: String): Unit = { - // We intentionally do not clean up executors that are already blacklisted in - // nodeToBlacklistedExecs, so that if another executor on the same node gets blacklisted, we can - // blacklist the entire node. We also can't clean up executorIdToBlacklistStatus, so we can - // eventually remove the executor after the timeout. Despite not clearing those structures - // here, we don't expect they will grow too big since you won't get too many executors on one - // node, and the timeout will clear it up periodically in any case. - executorIdToFailureList -= executorId - } - - - /** - * Tracks all failures for one executor (that have not passed the timeout). - * - * In general we actually expect this to be extremely small, since it won't contain more than the - * maximum number of task failures before an executor is failed (default 2). - */ - private[scheduler] final class ExecutorFailureList extends Logging { - - private case class TaskId(stage: Int, stageAttempt: Int, taskIndex: Int) - - /** - * All failures on this executor in successful task sets. - */ - private var failuresAndExpiryTimes = ArrayBuffer[(TaskId, Long)]() - /** - * As an optimization, we track the min expiry time over all entries in failuresAndExpiryTimes - * so its quick to tell if there are any failures with expiry before the current time. - */ - private var minExpiryTime = Long.MaxValue - - def addFailures( - stage: Int, - stageAttempt: Int, - failuresInTaskSet: ExecutorFailuresInTaskSet): Unit = { - failuresInTaskSet.taskToFailureCountAndFailureTime.foreach { - case (taskIdx, (_, failureTime)) => - val expiryTime = failureTime + BLACKLIST_TIMEOUT_MILLIS - failuresAndExpiryTimes += ((TaskId(stage, stageAttempt, taskIdx), expiryTime)) - if (expiryTime < minExpiryTime) { - minExpiryTime = expiryTime - } - } - } - - /** - * The number of unique tasks that failed on this executor. Only counts failures within the - * timeout, and in successful tasksets. - */ - def numUniqueTaskFailures: Int = failuresAndExpiryTimes.size - - def isEmpty: Boolean = failuresAndExpiryTimes.isEmpty - - /** - * Apply the timeout to individual tasks. This is to prevent one-off failures that are very - * spread out in time (and likely have nothing to do with problems on the executor) from - * triggering blacklisting. However, note that we do *not* remove executors and nodes from - * the blacklist as we expire individual task failures -- each have their own timeout. Eg., - * suppose: - * * timeout = 10, maxFailuresPerExec = 2 - * * Task 1 fails on exec 1 at time 0 - * * Task 2 fails on exec 1 at time 5 - * --> exec 1 is blacklisted from time 5 - 15. - * This is to simplify the implementation, as well as keep the behavior easier to understand - * for the end user. - */ - def dropFailuresWithTimeoutBefore(dropBefore: Long): Unit = { - if (minExpiryTime < dropBefore) { - var newMinExpiry = Long.MaxValue - val newFailures = new ArrayBuffer[(TaskId, Long)] - failuresAndExpiryTimes.foreach { case (task, expiryTime) => - if (expiryTime >= dropBefore) { - newFailures += ((task, expiryTime)) - if (expiryTime < newMinExpiry) { - newMinExpiry = expiryTime - } - } - } - failuresAndExpiryTimes = newFailures - minExpiryTime = newMinExpiry - } - } - - override def toString(): String = { - s"failures = $failuresAndExpiryTimes" - } - } - -} - -private[spark] object BlacklistTracker extends Logging { - - private val DEFAULT_TIMEOUT = "1h" - - /** - * Returns true if the blacklist is enabled, based on checking the configuration in the following - * order: - * 1. Is it specifically enabled or disabled? - * 2. Is it enabled via the legacy timeout conf? - * 3. Default is off - */ - def isBlacklistEnabled(conf: SparkConf): Boolean = { - conf.get(config.BLACKLIST_ENABLED) match { - case Some(enabled) => - enabled - case None => - // if they've got a non-zero setting for the legacy conf, always enable the blacklist, - // otherwise, use the default. - val legacyKey = config.BLACKLIST_LEGACY_TIMEOUT_CONF.key - conf.get(config.BLACKLIST_LEGACY_TIMEOUT_CONF).exists { legacyTimeout => - if (legacyTimeout == 0) { - logWarning(s"Turning off blacklisting due to legacy configuration: $legacyKey == 0") - false - } else { - logWarning(s"Turning on blacklisting due to legacy configuration: $legacyKey > 0") - true - } - } - } - } - - def getBlacklistTimeout(conf: SparkConf): Long = { - conf.get(config.BLACKLIST_TIMEOUT_CONF).getOrElse { - conf.get(config.BLACKLIST_LEGACY_TIMEOUT_CONF).getOrElse { - Utils.timeStringAsMs(DEFAULT_TIMEOUT) - } - } - } - - /** - * Verify that blacklist configurations are consistent; if not, throw an exception. Should only - * be called if blacklisting is enabled. - * - * The configuration for the blacklist is expected to adhere to a few invariants. Default - * values follow these rules of course, but users may unwittingly change one configuration - * without making the corresponding adjustment elsewhere. This ensures we fail-fast when - * there are such misconfigurations. - */ - def validateBlacklistConfs(conf: SparkConf): Unit = { - - def mustBePos(k: String, v: String): Unit = { - throw new IllegalArgumentException(s"$k was $v, but must be > 0.") - } - - Seq( - config.MAX_TASK_ATTEMPTS_PER_EXECUTOR, - config.MAX_TASK_ATTEMPTS_PER_NODE, - config.MAX_FAILURES_PER_EXEC_STAGE, - config.MAX_FAILED_EXEC_PER_NODE_STAGE, - config.MAX_FAILURES_PER_EXEC, - config.MAX_FAILED_EXEC_PER_NODE - ).foreach { config => - val v = conf.get(config) - if (v <= 0) { - mustBePos(config.key, v.toString) - } - } - - val timeout = getBlacklistTimeout(conf) - if (timeout <= 0) { - // first, figure out where the timeout came from, to include the right conf in the message. - conf.get(config.BLACKLIST_TIMEOUT_CONF) match { - case Some(t) => - mustBePos(config.BLACKLIST_TIMEOUT_CONF.key, timeout.toString) - case None => - mustBePos(config.BLACKLIST_LEGACY_TIMEOUT_CONF.key, timeout.toString) - } - } - - val maxTaskFailures = conf.get(config.TASK_MAX_FAILURES) - val maxNodeAttempts = conf.get(config.MAX_TASK_ATTEMPTS_PER_NODE) - - if (maxNodeAttempts >= maxTaskFailures) { - throw new IllegalArgumentException(s"${config.MAX_TASK_ATTEMPTS_PER_NODE.key} " + - s"( = ${maxNodeAttempts}) was >= ${config.TASK_MAX_FAILURES.key} " + - s"( = ${maxTaskFailures} ). Though blacklisting is enabled, with this configuration, " + - s"Spark will not be robust to one bad node. Decrease " + - s"${config.MAX_TASK_ATTEMPTS_PER_NODE.key}, increase ${config.TASK_MAX_FAILURES.key}, " + - s"or disable blacklisting with ${config.BLACKLIST_ENABLED.key}") - } - } -} - -private final case class BlacklistedExecutor(node: String, expiryTime: Long) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 080e0e7f1552f..13b766e654832 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -333,8 +333,8 @@ private[spark] class DAGScheduler( } /** - * Called by the TaskSetManager when a taskset becomes unschedulable due to blacklisting and - * dynamic allocation is enabled. + * Called by the TaskSetManager when a taskset becomes unschedulable due to executors being + * excluded because of too many task failures and dynamic allocation is enabled. */ def unschedulableTaskSetAdded( stageId: Int, diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index b2e9a0b2a04e8..1fda03f732636 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -191,27 +191,53 @@ private[spark] class EventLoggingListener( logEvent(event, flushLogger = true) } + override def onExecutorExcluded(event: SparkListenerExecutorExcluded): Unit = { + logEvent(event, flushLogger = true) + } + override def onExecutorBlacklistedForStage( event: SparkListenerExecutorBlacklistedForStage): Unit = { logEvent(event, flushLogger = true) } + override def onExecutorExcludedForStage( + event: SparkListenerExecutorExcludedForStage): Unit = { + logEvent(event, flushLogger = true) + } + override def onNodeBlacklistedForStage(event: SparkListenerNodeBlacklistedForStage): Unit = { logEvent(event, flushLogger = true) } + override def onNodeExcludedForStage(event: SparkListenerNodeExcludedForStage): Unit = { + logEvent(event, flushLogger = true) + } + override def onExecutorUnblacklisted(event: SparkListenerExecutorUnblacklisted): Unit = { logEvent(event, flushLogger = true) } + override def onExecutorUnexcluded(event: SparkListenerExecutorUnexcluded): Unit = { + logEvent(event, flushLogger = true) + } + + override def onNodeBlacklisted(event: SparkListenerNodeBlacklisted): Unit = { logEvent(event, flushLogger = true) } + override def onNodeExcluded(event: SparkListenerNodeExcluded): Unit = { + logEvent(event, flushLogger = true) + } + override def onNodeUnblacklisted(event: SparkListenerNodeUnblacklisted): Unit = { logEvent(event, flushLogger = true) } + override def onNodeUnexcluded(event: SparkListenerNodeUnexcluded): Unit = { + logEvent(event, flushLogger = true) + } + override def onBlockUpdated(event: SparkListenerBlockUpdated): Unit = { if (shouldLogBlockUpdates) { logEvent(event, flushLogger = true) diff --git a/core/src/main/scala/org/apache/spark/scheduler/ExecutorFailuresInTaskSet.scala b/core/src/main/scala/org/apache/spark/scheduler/ExecutorFailuresInTaskSet.scala index 70553d8be28b5..f27c1560f8272 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ExecutorFailuresInTaskSet.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ExecutorFailuresInTaskSet.scala @@ -19,7 +19,7 @@ package org.apache.spark.scheduler import scala.collection.mutable.HashMap /** - * Small helper for tracking failed tasks for blacklisting purposes. Info on all failures on one + * Small helper for tracking failed tasks for exclusion purposes. Info on all failures on one * executor, within one task set. */ private[scheduler] class ExecutorFailuresInTaskSet(val node: String) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/HealthTracker.scala b/core/src/main/scala/org/apache/spark/scheduler/HealthTracker.scala new file mode 100644 index 0000000000000..9bbacea94bf68 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/HealthTracker.scala @@ -0,0 +1,491 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.scheduler + +import java.util.concurrent.atomic.AtomicReference + +import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} + +import org.apache.spark.{ExecutorAllocationClient, SparkConf, SparkContext} +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config +import org.apache.spark.util.{Clock, SystemClock, Utils} + +/** + * HealthTracker is designed to track problematic executors and nodes. It supports excluding + * executors and nodes across an entire application (with a periodic expiry). TaskSetManagers add + * additional logic for exclusion of executors and nodes for individual tasks and stages which + * works in concert with the logic here. + * + * The tracker needs to deal with a variety of workloads, eg.: + * + * * bad user code -- this may lead to many task failures, but that should not count against + * individual executors + * * many small stages -- this may prevent a bad executor for having many failures within one + * stage, but still many failures over the entire application + * * "flaky" executors -- they don't fail every task, but are still faulty enough to merit + * excluding + * + * See the design doc on SPARK-8425 for a more in-depth discussion. Note SPARK-32037 renamed + * the feature. + * + * THREADING: As with most helpers of TaskSchedulerImpl, this is not thread-safe. Though it is + * called by multiple threads, callers must already have a lock on the TaskSchedulerImpl. The + * one exception is [[excludedNodeList()]], which can be called without holding a lock. + */ +private[scheduler] class HealthTracker ( + private val listenerBus: LiveListenerBus, + conf: SparkConf, + allocationClient: Option[ExecutorAllocationClient], + clock: Clock = new SystemClock()) extends Logging { + + def this(sc: SparkContext, allocationClient: Option[ExecutorAllocationClient]) = { + this(sc.listenerBus, sc.conf, allocationClient) + } + + HealthTracker.validateExcludeOnFailureConfs(conf) + 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 EXCLUDE_ON_FAILURE_TIMEOUT_MILLIS = HealthTracker.getExludeOnFailureTimeout(conf) + private val EXCLUDE_FETCH_FAILURE_ENABLED = + conf.get(config.EXCLUDE_ON_FAILURE_FETCH_FAILURE_ENABLED) + + /** + * A map from executorId to information on task failures. Tracks the time of each task failure, + * so that we can avoid excluding executors due to failures that are very far apart. We do not + * actively remove from this as soon as tasks hit their timeouts, to avoid the time it would take + * to do so. But it will not grow too large, because as soon as an executor gets too many + * failures, we exclude the executor and remove its entry here. + */ + private val executorIdToFailureList = new HashMap[String, ExecutorFailureList]() + val executorIdToExcludedStatus = new HashMap[String, ExcludedExecutor]() + val nodeIdToExcludedExpiryTime = new HashMap[String, Long]() + /** + * An immutable copy of the set of nodes that are currently excluded. Kept in an + * AtomicReference to make [[excludedNodeList()]] thread-safe. + */ + private val _excludedNodeList = new AtomicReference[Set[String]](Set()) + /** + * Time when the next excluded node will expire. Used as a shortcut to + * avoid iterating over all entries in the excludedNodeList when none will have expired. + */ + var nextExpiryTime: Long = Long.MaxValue + /** + * Mapping from nodes to all of the executors that have been excluded on that node. We do *not* + * remove from this when executors are removed from spark, so we can track when we get multiple + * successive excluded executors on one node. Nonetheless, it will not grow too large because + * there cannot be many excluded executors on one node, before we stop requesting more + * executors on that node, and we clean up the list of exluded executors once an executor has + * been excluded for EXCLUDE_ON_FAILURE_TIMEOUT_MILLIS. + */ + val nodeToExcludedExecs = new HashMap[String, HashSet[String]]() + + /** + * Include executors and nodes that have been excluded for at least + * EXCLUDE_ON_FAILURE_TIMEOUT_MILLIS + */ + def applyExcludeOnFailureTimeout(): Unit = { + val now = clock.getTimeMillis() + // quickly check if we've got anything to expire that is excluded -- if not, + // avoid doing any work + if (now > nextExpiryTime) { + // Apply the timeout to excluded nodes and executors + val execsToInclude = executorIdToExcludedStatus.filter(_._2.expiryTime < now).keys + if (execsToInclude.nonEmpty) { + // Include any executors that have been exluded longer than the excludeOnFailure timeout. + logInfo(s"Removing executors $execsToInclude from exclude list because the " + + s"the executors have reached the timed out") + execsToInclude.foreach { exec => + val status = executorIdToExcludedStatus.remove(exec).get + val failedExecsOnNode = nodeToExcludedExecs(status.node) + // post both to keep backwards compatibility + listenerBus.post(SparkListenerExecutorUnblacklisted(now, exec)) + listenerBus.post(SparkListenerExecutorUnexcluded(now, exec)) + failedExecsOnNode.remove(exec) + if (failedExecsOnNode.isEmpty) { + nodeToExcludedExecs.remove(status.node) + } + } + } + val nodesToInclude = nodeIdToExcludedExpiryTime.filter(_._2 < now).keys + if (nodesToInclude.nonEmpty) { + // Include any nodes that have been excluded longer than the excludeOnFailure timeout. + logInfo(s"Removing nodes $nodesToInclude from exclude list because the " + + s"nodes have reached has timed out") + nodesToInclude.foreach { node => + nodeIdToExcludedExpiryTime.remove(node) + // post both to keep backwards compatibility + listenerBus.post(SparkListenerNodeUnblacklisted(now, node)) + listenerBus.post(SparkListenerNodeUnexcluded(now, node)) + } + _excludedNodeList.set(nodeIdToExcludedExpiryTime.keySet.toSet) + } + updateNextExpiryTime() + } + } + + private def updateNextExpiryTime(): Unit = { + val execMinExpiry = if (executorIdToExcludedStatus.nonEmpty) { + executorIdToExcludedStatus.map{_._2.expiryTime}.min + } else { + Long.MaxValue + } + val nodeMinExpiry = if (nodeIdToExcludedExpiryTime.nonEmpty) { + nodeIdToExcludedExpiryTime.values.min + } else { + Long.MaxValue + } + nextExpiryTime = math.min(execMinExpiry, nodeMinExpiry) + } + + private def killExecutor(exec: String, msg: String): Unit = { + allocationClient match { + case Some(a) => + logInfo(msg) + a.killExecutors(Seq(exec), adjustTargetNumExecutors = false, countFailures = false, + force = true) + case None => + logInfo(s"Not attempting to kill excluded executor id $exec " + + s"since allocation client is not defined.") + } + } + + private def killExcludedExecutor(exec: String): Unit = { + if (conf.get(config.EXCLUDE_ON_FAILURE_KILL_ENABLED)) { + killExecutor(exec, s"Killing excluded executor id $exec since " + + s"${config.EXCLUDE_ON_FAILURE_KILL_ENABLED.key} is set.") + } + } + + private[scheduler] def killExcludedIdleExecutor(exec: String): Unit = { + killExecutor(exec, + s"Killing excluded idle executor id $exec because of task unschedulability and trying " + + "to acquire a new executor.") + } + + private def killExecutorsOnExcludedNode(node: String): Unit = { + if (conf.get(config.EXCLUDE_ON_FAILURE_KILL_ENABLED)) { + allocationClient match { + case Some(a) => + logInfo(s"Killing all executors on excluded host $node " + + s"since ${config.EXCLUDE_ON_FAILURE_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 excluded host $node " + + s"since allocation client is not defined.") + } + } + } + + def updateExcludedForFetchFailure(host: String, exec: String): Unit = { + if (EXCLUDE_FETCH_FAILURE_ENABLED) { + // If we exclude 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 exclude (and potentially kill) all + // of them immediately. + + val now = clock.getTimeMillis() + val expiryTimeForNewExcludes = now + EXCLUDE_ON_FAILURE_TIMEOUT_MILLIS + + if (conf.get(config.SHUFFLE_SERVICE_ENABLED)) { + if (!nodeIdToExcludedExpiryTime.contains(host)) { + logInfo(s"excluding node $host due to fetch failure of external shuffle service") + + nodeIdToExcludedExpiryTime.put(host, expiryTimeForNewExcludes) + // post both to keep backwards compatibility + listenerBus.post(SparkListenerNodeBlacklisted(now, host, 1)) + listenerBus.post(SparkListenerNodeExcluded(now, host, 1)) + _excludedNodeList.set(nodeIdToExcludedExpiryTime.keySet.toSet) + killExecutorsOnExcludedNode(host) + updateNextExpiryTime() + } + } else if (!executorIdToExcludedStatus.contains(exec)) { + logInfo(s"Excluding executor $exec due to fetch failure") + + executorIdToExcludedStatus.put(exec, ExcludedExecutor(host, expiryTimeForNewExcludes)) + // We hardcoded number of failure tasks to 1 for fetch failure, because there's no + // reattempt for such failure. + // post both to keep backwards compatibility + listenerBus.post(SparkListenerExecutorBlacklisted(now, exec, 1)) + listenerBus.post(SparkListenerExecutorExcluded(now, exec, 1)) + updateNextExpiryTime() + killExcludedExecutor(exec) + + val excludedExecsOnNode = nodeToExcludedExecs.getOrElseUpdate(host, HashSet[String]()) + excludedExecsOnNode += exec + } + } + } + + def updateExcludedForSuccessfulTaskSet( + stageId: Int, + stageAttemptId: Int, + failuresByExec: HashMap[String, ExecutorFailuresInTaskSet]): Unit = { + // if any tasks failed, we count them towards the overall failure count for the executor at + // this point. + val now = clock.getTimeMillis() + failuresByExec.foreach { case (exec, failuresInTaskSet) => + val appFailuresOnExecutor = + executorIdToFailureList.getOrElseUpdate(exec, new ExecutorFailureList) + appFailuresOnExecutor.addFailures(stageId, stageAttemptId, failuresInTaskSet) + appFailuresOnExecutor.dropFailuresWithTimeoutBefore(now) + val newTotal = appFailuresOnExecutor.numUniqueTaskFailures + + val expiryTimeForNewExcludes = now + EXCLUDE_ON_FAILURE_TIMEOUT_MILLIS + // If this pushes the total number of failures over the threshold, exclude the executor. + // If its already excluded, we avoid "re-excluding" (which can happen if there were + // other tasks already running in another taskset when it got excluded), because it makes + // some of the logic around expiry times a little more confusing. But it also wouldn't be a + // problem to re-exclude, with a later expiry time. + if (newTotal >= MAX_FAILURES_PER_EXEC && !executorIdToExcludedStatus.contains(exec)) { + logInfo(s"Excluding executor id: $exec because it has $newTotal" + + s" task failures in successful task sets") + val node = failuresInTaskSet.node + executorIdToExcludedStatus.put(exec, ExcludedExecutor(node, expiryTimeForNewExcludes)) + // post both to keep backwards compatibility + listenerBus.post(SparkListenerExecutorBlacklisted(now, exec, newTotal)) + listenerBus.post(SparkListenerExecutorExcluded(now, exec, newTotal)) + executorIdToFailureList.remove(exec) + updateNextExpiryTime() + killExcludedExecutor(exec) + + // In addition to excluding the executor, we also update the data for failures on the + // node, and potentially exclude the entire node as well. + val excludedExecsOnNode = nodeToExcludedExecs.getOrElseUpdate(node, HashSet[String]()) + excludedExecsOnNode += exec + // If the node is already excluded, we avoid adding it again with a later expiry + // time. + if (excludedExecsOnNode.size >= MAX_FAILED_EXEC_PER_NODE && + !nodeIdToExcludedExpiryTime.contains(node)) { + logInfo(s"Excluding node $node because it has ${excludedExecsOnNode.size} " + + s"executors excluded: ${excludedExecsOnNode}") + nodeIdToExcludedExpiryTime.put(node, expiryTimeForNewExcludes) + // post both to keep backwards compatibility + listenerBus.post(SparkListenerNodeBlacklisted(now, node, excludedExecsOnNode.size)) + listenerBus.post(SparkListenerNodeExcluded(now, node, excludedExecsOnNode.size)) + _excludedNodeList.set(nodeIdToExcludedExpiryTime.keySet.toSet) + killExecutorsOnExcludedNode(node) + } + } + } + } + + def isExecutorExcluded(executorId: String): Boolean = { + executorIdToExcludedStatus.contains(executorId) + } + + /** + * Get the full set of nodes that are excluded. Unlike other methods in this class, this *IS* + * thread-safe -- no lock required on a taskScheduler. + */ + def excludedNodeList(): Set[String] = { + _excludedNodeList.get() + } + + def isNodeExcluded(node: String): Boolean = { + nodeIdToExcludedExpiryTime.contains(node) + } + + def handleRemovedExecutor(executorId: String): Unit = { + // We intentionally do not clean up executors that are already excluded in + // nodeToExcludedExecs, so that if another executor on the same node gets excluded, we can + // exclude the entire node. We also can't clean up executorIdToExcludedStatus, so we can + // eventually remove the executor after the timeout. Despite not clearing those structures + // here, we don't expect they will grow too big since you won't get too many executors on one + // node, and the timeout will clear it up periodically in any case. + executorIdToFailureList -= executorId + } + + /** + * Tracks all failures for one executor (that have not passed the timeout). + * + * In general we actually expect this to be extremely small, since it won't contain more than the + * maximum number of task failures before an executor is failed (default 2). + */ + private[scheduler] final class ExecutorFailureList extends Logging { + + private case class TaskId(stage: Int, stageAttempt: Int, taskIndex: Int) + + /** + * All failures on this executor in successful task sets. + */ + private var failuresAndExpiryTimes = ArrayBuffer[(TaskId, Long)]() + /** + * As an optimization, we track the min expiry time over all entries in failuresAndExpiryTimes + * so its quick to tell if there are any failures with expiry before the current time. + */ + private var minExpiryTime = Long.MaxValue + + def addFailures( + stage: Int, + stageAttempt: Int, + failuresInTaskSet: ExecutorFailuresInTaskSet): Unit = { + failuresInTaskSet.taskToFailureCountAndFailureTime.foreach { + case (taskIdx, (_, failureTime)) => + val expiryTime = failureTime + EXCLUDE_ON_FAILURE_TIMEOUT_MILLIS + failuresAndExpiryTimes += ((TaskId(stage, stageAttempt, taskIdx), expiryTime)) + if (expiryTime < minExpiryTime) { + minExpiryTime = expiryTime + } + } + } + + /** + * The number of unique tasks that failed on this executor. Only counts failures within the + * timeout, and in successful tasksets. + */ + def numUniqueTaskFailures: Int = failuresAndExpiryTimes.size + + def isEmpty: Boolean = failuresAndExpiryTimes.isEmpty + + /** + * Apply the timeout to individual tasks. This is to prevent one-off failures that are very + * spread out in time (and likely have nothing to do with problems on the executor) from + * triggering exlusion. However, note that we do *not* remove executors and nodes from + * being excluded as we expire individual task failures -- each have their own timeout. Eg., + * suppose: + * * timeout = 10, maxFailuresPerExec = 2 + * * Task 1 fails on exec 1 at time 0 + * * Task 2 fails on exec 1 at time 5 + * --> exec 1 is excluded from time 5 - 15. + * This is to simplify the implementation, as well as keep the behavior easier to understand + * for the end user. + */ + def dropFailuresWithTimeoutBefore(dropBefore: Long): Unit = { + if (minExpiryTime < dropBefore) { + var newMinExpiry = Long.MaxValue + val newFailures = new ArrayBuffer[(TaskId, Long)] + failuresAndExpiryTimes.foreach { case (task, expiryTime) => + if (expiryTime >= dropBefore) { + newFailures += ((task, expiryTime)) + if (expiryTime < newMinExpiry) { + newMinExpiry = expiryTime + } + } + } + failuresAndExpiryTimes = newFailures + minExpiryTime = newMinExpiry + } + } + + override def toString(): String = { + s"failures = $failuresAndExpiryTimes" + } + } + +} + +private[spark] object HealthTracker extends Logging { + + private val DEFAULT_TIMEOUT = "1h" + + /** + * Returns true if the excludeOnFailure is enabled, based on checking the configuration + * in the following order: + * 1. Is it specifically enabled or disabled? + * 2. Is it enabled via the legacy timeout conf? + * 3. Default is off + */ + def isExcludeOnFailureEnabled(conf: SparkConf): Boolean = { + conf.get(config.EXCLUDE_ON_FAILURE_ENABLED) match { + case Some(enabled) => + enabled + case None => + // if they've got a non-zero setting for the legacy conf, always enable it, + // otherwise, use the default. + val legacyKey = config.EXCLUDE_ON_FAILURE_LEGACY_TIMEOUT_CONF.key + conf.get(config.EXCLUDE_ON_FAILURE_LEGACY_TIMEOUT_CONF).exists { legacyTimeout => + if (legacyTimeout == 0) { + logWarning(s"Turning off excludeOnFailure due to legacy configuration: $legacyKey == 0") + false + } else { + logWarning(s"Turning on excludeOnFailure due to legacy configuration: $legacyKey > 0") + true + } + } + } + } + + def getExludeOnFailureTimeout(conf: SparkConf): Long = { + conf.get(config.EXCLUDE_ON_FAILURE_TIMEOUT_CONF).getOrElse { + conf.get(config.EXCLUDE_ON_FAILURE_LEGACY_TIMEOUT_CONF).getOrElse { + Utils.timeStringAsMs(DEFAULT_TIMEOUT) + } + } + } + + /** + * Verify that exclude on failure configurations are consistent; if not, throw an exception. + * Should only be called if excludeOnFailure is enabled. + * + * The configuration is expected to adhere to a few invariants. Default values + * follow these rules of course, but users may unwittingly change one configuration + * without making the corresponding adjustment elsewhere. This ensures we fail-fast when + * there are such misconfigurations. + */ + def validateExcludeOnFailureConfs(conf: SparkConf): Unit = { + + def mustBePos(k: String, v: String): Unit = { + throw new IllegalArgumentException(s"$k was $v, but must be > 0.") + } + + Seq( + config.MAX_TASK_ATTEMPTS_PER_EXECUTOR, + config.MAX_TASK_ATTEMPTS_PER_NODE, + config.MAX_FAILURES_PER_EXEC_STAGE, + config.MAX_FAILED_EXEC_PER_NODE_STAGE, + config.MAX_FAILURES_PER_EXEC, + config.MAX_FAILED_EXEC_PER_NODE + ).foreach { config => + val v = conf.get(config) + if (v <= 0) { + mustBePos(config.key, v.toString) + } + } + + val timeout = getExludeOnFailureTimeout(conf) + if (timeout <= 0) { + // first, figure out where the timeout came from, to include the right conf in the message. + conf.get(config.EXCLUDE_ON_FAILURE_TIMEOUT_CONF) match { + case Some(t) => + mustBePos(config.EXCLUDE_ON_FAILURE_TIMEOUT_CONF.key, timeout.toString) + case None => + mustBePos(config.EXCLUDE_ON_FAILURE_LEGACY_TIMEOUT_CONF.key, timeout.toString) + } + } + + val maxTaskFailures = conf.get(config.TASK_MAX_FAILURES) + val maxNodeAttempts = conf.get(config.MAX_TASK_ATTEMPTS_PER_NODE) + + if (maxNodeAttempts >= maxTaskFailures) { + throw new IllegalArgumentException(s"${config.MAX_TASK_ATTEMPTS_PER_NODE.key} " + + s"( = ${maxNodeAttempts}) was >= ${config.TASK_MAX_FAILURES.key} " + + s"( = ${maxTaskFailures} ). Though excludeOnFailure is enabled, with this configuration, " + + s"Spark will not be robust to one bad node. Decrease " + + s"${config.MAX_TASK_ATTEMPTS_PER_NODE.key}, increase ${config.TASK_MAX_FAILURES.key}, " + + s"or disable excludeOnFailure with ${config.EXCLUDE_ON_FAILURE_ENABLED.key}") + } + } +} + +private final case class ExcludedExecutor(node: String, expiryTime: Long) diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala index 8119215b8b74f..3fcb35b604ef6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -118,12 +118,21 @@ case class SparkListenerExecutorRemoved(time: Long, executorId: String, reason: extends SparkListenerEvent @DeveloperApi +@deprecated("use SparkListenerExecutorExcluded instead", "3.1.0") case class SparkListenerExecutorBlacklisted( time: Long, executorId: String, taskFailures: Int) extends SparkListenerEvent +@DeveloperApi +case class SparkListenerExecutorExcluded( + time: Long, + executorId: String, + taskFailures: Int) + extends SparkListenerEvent + +@deprecated("use SparkListenerExecutorExcludedForStage instead", "3.1.0") @DeveloperApi case class SparkListenerExecutorBlacklistedForStage( time: Long, @@ -133,6 +142,17 @@ case class SparkListenerExecutorBlacklistedForStage( stageAttemptId: Int) extends SparkListenerEvent + +@DeveloperApi +case class SparkListenerExecutorExcludedForStage( + time: Long, + executorId: String, + taskFailures: Int, + stageId: Int, + stageAttemptId: Int) + extends SparkListenerEvent + +@deprecated("use SparkListenerNodeExcludedForStage instead", "3.1.0") @DeveloperApi case class SparkListenerNodeBlacklistedForStage( time: Long, @@ -142,10 +162,27 @@ case class SparkListenerNodeBlacklistedForStage( stageAttemptId: Int) extends SparkListenerEvent + +@DeveloperApi +case class SparkListenerNodeExcludedForStage( + time: Long, + hostId: String, + executorFailures: Int, + stageId: Int, + stageAttemptId: Int) + extends SparkListenerEvent + +@deprecated("use SparkListenerExecutorUnexcluded instead", "3.1.0") @DeveloperApi case class SparkListenerExecutorUnblacklisted(time: Long, executorId: String) extends SparkListenerEvent + +@DeveloperApi +case class SparkListenerExecutorUnexcluded(time: Long, executorId: String) + extends SparkListenerEvent + +@deprecated("use SparkListenerNodeExcluded instead", "3.1.0") @DeveloperApi case class SparkListenerNodeBlacklisted( time: Long, @@ -153,10 +190,23 @@ case class SparkListenerNodeBlacklisted( executorFailures: Int) extends SparkListenerEvent + +@DeveloperApi +case class SparkListenerNodeExcluded( + time: Long, + hostId: String, + executorFailures: Int) + extends SparkListenerEvent + +@deprecated("use SparkListenerNodeUnexcluded instead", "3.1.0") @DeveloperApi case class SparkListenerNodeUnblacklisted(time: Long, hostId: String) extends SparkListenerEvent +@DeveloperApi +case class SparkListenerNodeUnexcluded(time: Long, hostId: String) + extends SparkListenerEvent + @DeveloperApi case class SparkListenerUnschedulableTaskSetAdded( stageId: Int, @@ -319,38 +369,75 @@ private[spark] trait SparkListenerInterface { def onExecutorRemoved(executorRemoved: SparkListenerExecutorRemoved): Unit /** - * Called when the driver blacklists an executor for a Spark application. + * Called when the driver excludes an executor for a Spark application. */ + @deprecated("use onExecutorExcluded instead", "3.1.0") def onExecutorBlacklisted(executorBlacklisted: SparkListenerExecutorBlacklisted): Unit /** - * Called when the driver blacklists an executor for a stage. + * Called when the driver excludes an executor for a Spark application. */ + def onExecutorExcluded(executorExcluded: SparkListenerExecutorExcluded): Unit + + /** + * Called when the driver excludes an executor for a stage. + */ + @deprecated("use onExecutorExcludedForStage instead", "3.1.0") def onExecutorBlacklistedForStage( executorBlacklistedForStage: SparkListenerExecutorBlacklistedForStage): Unit /** - * Called when the driver blacklists a node for a stage. + * Called when the driver excludes an executor for a stage. + */ + def onExecutorExcludedForStage( + executorExcludedForStage: SparkListenerExecutorExcludedForStage): Unit + + /** + * Called when the driver excludes a node for a stage. */ + @deprecated("use onNodeExcludedForStage instead", "3.1.0") def onNodeBlacklistedForStage(nodeBlacklistedForStage: SparkListenerNodeBlacklistedForStage): Unit /** - * Called when the driver re-enables a previously blacklisted executor. + * Called when the driver excludes a node for a stage. + */ + def onNodeExcludedForStage(nodeExcludedForStage: SparkListenerNodeExcludedForStage): Unit + + /** + * Called when the driver re-enables a previously excluded executor. */ + @deprecated("use onExecutorUnexcluded instead", "3.1.0") def onExecutorUnblacklisted(executorUnblacklisted: SparkListenerExecutorUnblacklisted): Unit /** - * Called when the driver blacklists a node for a Spark application. + * Called when the driver re-enables a previously excluded executor. + */ + def onExecutorUnexcluded(executorUnexcluded: SparkListenerExecutorUnexcluded): Unit + + /** + * Called when the driver excludes a node for a Spark application. */ + @deprecated("use onNodeExcluded instead", "3.1.0") def onNodeBlacklisted(nodeBlacklisted: SparkListenerNodeBlacklisted): Unit /** - * Called when the driver re-enables a previously blacklisted node. + * Called when the driver excludes a node for a Spark application. */ + def onNodeExcluded(nodeExcluded: SparkListenerNodeExcluded): Unit + + /** + * Called when the driver re-enables a previously excluded node. + */ + @deprecated("use onNodeUnexcluded instead", "3.1.0") def onNodeUnblacklisted(nodeUnblacklisted: SparkListenerNodeUnblacklisted): Unit /** - * Called when a taskset becomes unschedulable due to blacklisting and dynamic allocation + * Called when the driver re-enables a previously excluded node. + */ + def onNodeUnexcluded(nodeUnexcluded: SparkListenerNodeUnexcluded): Unit + + /** + * Called when a taskset becomes unschedulable due to exludeOnFailure and dynamic allocation * is enabled. */ def onUnschedulableTaskSetAdded( @@ -433,21 +520,33 @@ abstract class SparkListener extends SparkListenerInterface { override def onExecutorBlacklisted( executorBlacklisted: SparkListenerExecutorBlacklisted): Unit = { } + override def onExecutorExcluded( + executorExcluded: SparkListenerExecutorExcluded): Unit = { } - def onExecutorBlacklistedForStage( + override def onExecutorBlacklistedForStage( executorBlacklistedForStage: SparkListenerExecutorBlacklistedForStage): Unit = { } + override def onExecutorExcludedForStage( + executorExcludedForStage: SparkListenerExecutorExcludedForStage): Unit = { } - def onNodeBlacklistedForStage( + override def onNodeBlacklistedForStage( nodeBlacklistedForStage: SparkListenerNodeBlacklistedForStage): Unit = { } + override def onNodeExcludedForStage( + nodeExcludedForStage: SparkListenerNodeExcludedForStage): Unit = { } override def onExecutorUnblacklisted( executorUnblacklisted: SparkListenerExecutorUnblacklisted): Unit = { } + override def onExecutorUnexcluded( + executorUnexcluded: SparkListenerExecutorUnexcluded): Unit = { } override def onNodeBlacklisted( nodeBlacklisted: SparkListenerNodeBlacklisted): Unit = { } + override def onNodeExcluded( + nodeExcluded: SparkListenerNodeExcluded): Unit = { } override def onNodeUnblacklisted( nodeUnblacklisted: SparkListenerNodeUnblacklisted): Unit = { } + override def onNodeUnexcluded( + nodeUnexcluded: SparkListenerNodeUnexcluded): Unit = { } override def onUnschedulableTaskSetAdded( unschedulableTaskSetAdded: SparkListenerUnschedulableTaskSetAdded): Unit = { } diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala index 13e65f4291fd0..ec0c0cf3cf82b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala @@ -75,6 +75,18 @@ private[spark] trait SparkListenerBus listener.onNodeBlacklisted(nodeBlacklisted) case nodeUnblacklisted: SparkListenerNodeUnblacklisted => listener.onNodeUnblacklisted(nodeUnblacklisted) + case executorExcludedForStage: SparkListenerExecutorExcludedForStage => + listener.onExecutorExcludedForStage(executorExcludedForStage) + case nodeExcludedForStage: SparkListenerNodeExcludedForStage => + listener.onNodeExcludedForStage(nodeExcludedForStage) + case executorExcluded: SparkListenerExecutorExcluded => + listener.onExecutorExcluded(executorExcluded) + case executorUnexcluded: SparkListenerExecutorUnexcluded => + listener.onExecutorUnexcluded(executorUnexcluded) + case nodeExcluded: SparkListenerNodeExcluded => + listener.onNodeExcluded(nodeExcluded) + case nodeUnexcluded: SparkListenerNodeUnexcluded => + listener.onNodeUnexcluded(nodeUnexcluded) case blockUpdated: SparkListenerBlockUpdated => listener.onBlockUpdated(blockUpdated) case speculativeTaskSubmitted: SparkListenerSpeculativeTaskSubmitted => diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 2fcf13d5268f8..57e219999b0d0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -91,9 +91,9 @@ private[spark] class TaskSchedulerImpl( this(sc, sc.conf.get(config.TASK_MAX_FAILURES)) } - // Lazily initializing blacklistTrackerOpt to avoid getting empty ExecutorAllocationClient, + // Lazily initializing healthTrackerOpt to avoid getting empty ExecutorAllocationClient, // because ExecutorAllocationClient is created after this TaskSchedulerImpl. - private[scheduler] lazy val blacklistTrackerOpt = maybeCreateBlacklistTracker(sc) + private[scheduler] lazy val healthTrackerOpt = maybeCreateHealthTracker(sc) val conf = sc.conf @@ -281,7 +281,7 @@ private[spark] class TaskSchedulerImpl( private[scheduler] def createTaskSetManager( taskSet: TaskSet, maxTaskFailures: Int): TaskSetManager = { - new TaskSetManager(this, taskSet, maxTaskFailures, blacklistTrackerOpt, clock) + new TaskSetManager(this, taskSet, maxTaskFailures, healthTrackerOpt, clock) } override def cancelTasks(stageId: Int, interruptThread: Boolean): Unit = synchronized { @@ -381,7 +381,7 @@ private[spark] class TaskSchedulerImpl( : (Boolean, Option[TaskLocality]) = { var noDelayScheduleRejects = true var minLaunchedLocality: Option[TaskLocality] = None - // nodes and executors that are blacklisted for the entire application have already been + // nodes and executors that are excluded for the entire application have already been // filtered out by this point for (i <- 0 until shuffledOffers.size) { val execId = shuffledOffers(i).executorId @@ -515,15 +515,15 @@ private[spark] class TaskSchedulerImpl( hostsByRack.getOrElseUpdate(rack, new HashSet[String]()) += host } - // Before making any offers, remove any nodes from the blacklist whose blacklist has expired. Do + // Before making any offers, include any nodes whose expireOnFailure timeout has expired. Do // this here to avoid a separate thread and added synchronization overhead, and also because - // updating the blacklist is only relevant when task offers are being made. - blacklistTrackerOpt.foreach(_.applyBlacklistTimeout()) + // updating the excluded executors and nodes is only relevant when task offers are being made. + healthTrackerOpt.foreach(_.applyExcludeOnFailureTimeout()) - val filteredOffers = blacklistTrackerOpt.map { blacklistTracker => + val filteredOffers = healthTrackerOpt.map { healthTracker => offers.filter { offer => - !blacklistTracker.isNodeBlacklisted(offer.host) && - !blacklistTracker.isExecutorBlacklisted(offer.executorId) + !healthTracker.isNodeExcluded(offer.host) && + !healthTracker.isExecutorExcluded(offer.executorId) } }.getOrElse(offers) @@ -602,15 +602,15 @@ private[spark] class TaskSchedulerImpl( } if (!launchedAnyTask) { - taskSet.getCompletelyBlacklistedTaskIfAny(hostToExecutors).foreach { taskIndex => - // If the taskSet is unschedulable we try to find an existing idle blacklisted + taskSet.getCompletelyExcludedTaskIfAny(hostToExecutors).foreach { taskIndex => + // If the taskSet is unschedulable we try to find an existing idle excluded // executor and kill the idle executor and kick off an abortTimer which if it doesn't // schedule a task within the the timeout will abort the taskSet if we were unable to // schedule any task from the taskSet. // Note 1: We keep track of schedulability on a per taskSet basis rather than on a per // task basis. // Note 2: The taskSet can still be aborted when there are more than one idle - // blacklisted executors and dynamic allocation is on. This can happen when a killed + // excluded executors and dynamic allocation is on. This can happen when a killed // idle executor isn't replaced in time by ExecutorAllocationManager as it relies on // pending tasks and doesn't kill executors on idle timeouts, resulting in the abort // timer to expire and abort the taskSet. @@ -621,7 +621,7 @@ private[spark] class TaskSchedulerImpl( executorIdToRunningTaskIds.find(x => !isExecutorBusy(x._1)) match { case Some ((executorId, _)) => if (!unschedulableTaskSetToExpiryTime.contains(taskSet)) { - blacklistTrackerOpt.foreach(blt => blt.killBlacklistedIdleExecutor(executorId)) + healthTrackerOpt.foreach(blt => blt.killExcludedIdleExecutor(executorId)) updateUnschedulableTaskSetTimeoutAndStartAbortTimer(taskSet, taskIndex) } case None => @@ -638,18 +638,19 @@ private[spark] class TaskSchedulerImpl( } } else { // Abort Immediately - logInfo("Cannot schedule any task because of complete blacklisting. No idle" + - s" executors can be found to kill. Aborting stage ${taskSet.stageId}.") - taskSet.abortSinceCompletelyBlacklisted(taskIndex) + logInfo("Cannot schedule any task because all executors excluded from " + + "failures. No idle executors can be found to kill. Aborting stage " + + s"${taskSet.stageId}.") + taskSet.abortSinceCompletelyExcludedOnFailure(taskIndex) } } } } else { - // We want to defer killing any taskSets as long as we have a non blacklisted executor + // We want to defer killing any taskSets as long as we have a non excluded executor // which can be used to schedule a task from any active taskSets. This ensures that the // job can make progress. // Note: It is theoretically possible that a taskSet never gets scheduled on a - // non-blacklisted executor and the abort timer doesn't kick in because of a constant + // non-excluded executor and the abort timer doesn't kick in because of a constant // submission of new TaskSets. See the PR for more details. if (unschedulableTaskSetToExpiryTime.nonEmpty) { logInfo("Clearing the expiry times for all unschedulable taskSets as a task was " + @@ -710,7 +711,7 @@ private[spark] class TaskSchedulerImpl( val timeout = conf.get(config.UNSCHEDULABLE_TASKSET_TIMEOUT) * 1000 unschedulableTaskSetToExpiryTime(taskSet) = clock.getTimeMillis() + timeout logInfo(s"Waiting for $timeout ms for completely " + - s"blacklisted task to be schedulable again before aborting stage ${taskSet.stageId}.") + s"excluded task to be schedulable again before aborting stage ${taskSet.stageId}.") abortTimer.schedule( createUnschedulableTaskSetAbortTimer(taskSet, taskIndex), timeout) } @@ -722,9 +723,9 @@ private[spark] class TaskSchedulerImpl( override def run(): Unit = TaskSchedulerImpl.this.synchronized { if (unschedulableTaskSetToExpiryTime.contains(taskSet) && unschedulableTaskSetToExpiryTime(taskSet) <= clock.getTimeMillis()) { - logInfo("Cannot schedule any task because of complete blacklisting. " + + logInfo("Cannot schedule any task because all executors excluded due to failures. " + s"Wait time for scheduling expired. Aborting stage ${taskSet.stageId}.") - taskSet.abortSinceCompletelyBlacklisted(taskIndex) + taskSet.abortSinceCompletelyExcludedOnFailure(taskIndex) } else { this.cancel() } @@ -1019,7 +1020,7 @@ private[spark] class TaskSchedulerImpl( executorIdToHost -= executorId rootPool.executorLost(executorId, host, reason) } - blacklistTrackerOpt.foreach(_.handleRemovedExecutor(executorId)) + healthTrackerOpt.foreach(_.handleRemovedExecutor(executorId)) } def executorAdded(execId: String, host: String): Unit = { @@ -1060,11 +1061,11 @@ private[spark] class TaskSchedulerImpl( } /** - * Get a snapshot of the currently blacklisted nodes for the entire application. This is + * Get a snapshot of the currently excluded nodes for the entire application. This is * thread-safe -- it can be called without a lock on the TaskScheduler. */ - def nodeBlacklist(): Set[String] = { - blacklistTrackerOpt.map(_.nodeBlacklist()).getOrElse(Set.empty) + def excludedNodes(): Set[String] = { + healthTrackerOpt.map(_.excludedNodeList()).getOrElse(Set.empty) } /** @@ -1223,13 +1224,13 @@ private[spark] object TaskSchedulerImpl { retval.toList } - private def maybeCreateBlacklistTracker(sc: SparkContext): Option[BlacklistTracker] = { - if (BlacklistTracker.isBlacklistEnabled(sc.conf)) { + private def maybeCreateHealthTracker(sc: SparkContext): Option[HealthTracker] = { + if (HealthTracker.isExcludeOnFailureEnabled(sc.conf)) { val executorAllocClient: Option[ExecutorAllocationClient] = sc.schedulerBackend match { case b: ExecutorAllocationClient => Some(b) case _ => None } - Some(new BlacklistTracker(sc, executorAllocClient)) + Some(new HealthTracker(sc, executorAllocClient)) } else { None } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetBlacklist.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetExcludeList.scala similarity index 63% rename from core/src/main/scala/org/apache/spark/scheduler/TaskSetBlacklist.scala rename to core/src/main/scala/org/apache/spark/scheduler/TaskSetExcludeList.scala index 4df2889089ee9..d8c46db166fc5 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetBlacklist.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetExcludeList.scala @@ -24,19 +24,19 @@ import org.apache.spark.internal.config import org.apache.spark.util.Clock /** - * Handles blacklisting executors and nodes within a taskset. This includes blacklisting specific - * (task, executor) / (task, nodes) pairs, and also completely blacklisting executors and nodes + * Handles excluding executors and nodes within a taskset. This includes excluding specific + * (task, executor) / (task, nodes) pairs, and also completely excluding executors and nodes * for the entire taskset. * - * It also must store sufficient information in task failures for application level blacklisting, - * which is handled by [[BlacklistTracker]]. Note that BlacklistTracker does not know anything + * It also must store sufficient information in task failures for application level exclusion, + * which is handled by [[HealthTracker]]. Note that HealthTracker does not know anything * about task failures until a taskset completes successfully. * * THREADING: This class is a helper to [[TaskSetManager]]; as with the methods in * [[TaskSetManager]] this class is designed only to be called from code with a lock on the * TaskScheduler (e.g. its event handlers). It should not be called from other threads. */ -private[scheduler] class TaskSetBlacklist( +private[scheduler] class TaskSetExcludelist( private val listenerBus: LiveListenerBus, val conf: SparkConf, val stageId: Int, @@ -49,9 +49,9 @@ private[scheduler] class TaskSetBlacklist( private val MAX_FAILED_EXEC_PER_NODE_STAGE = conf.get(config.MAX_FAILED_EXEC_PER_NODE_STAGE) /** - * A map from each executor to the task failures on that executor. This is used for blacklisting - * within this taskset, and it is also relayed onto [[BlacklistTracker]] for app-level - * blacklisting if this taskset completes successfully. + * A map from each executor to the task failures on that executor. This is used for exclusion + * within this taskset, and it is also relayed onto [[HealthTracker]] for app-level + * exlucsion if this taskset completes successfully. */ val execToFailures = new HashMap[String, ExecutorFailuresInTaskSet]() @@ -61,9 +61,9 @@ private[scheduler] class TaskSetBlacklist( * node -> execs mapping in the usual case when there aren't any failures). */ private val nodeToExecsWithFailures = new HashMap[String, HashSet[String]]() - private val nodeToBlacklistedTaskIndexes = new HashMap[String, HashSet[Int]]() - private val blacklistedExecs = new HashSet[String]() - private val blacklistedNodes = new HashSet[String]() + private val nodeToExcludedTaskIndexes = new HashMap[String, HashSet[Int]]() + private val excludedExecs = new HashSet[String]() + private val excludedNodes = new HashSet[String]() private var latestFailureReason: String = null @@ -75,36 +75,36 @@ private[scheduler] class TaskSetBlacklist( } /** - * Return true if this executor is blacklisted for the given task. This does *not* - * need to return true if the executor is blacklisted for the entire stage, or blacklisted + * Return true if this executor is excluded for the given task. This does *not* + * need to return true if the executor is excluded for the entire stage, or excluded * for the entire application. That is to keep this method as fast as possible in the inner-loop * of the scheduler, where those filters will have already been applied. */ - def isExecutorBlacklistedForTask(executorId: String, index: Int): Boolean = { + def isExecutorExcludedForTask(executorId: String, index: Int): Boolean = { execToFailures.get(executorId).exists { execFailures => execFailures.getNumTaskFailures(index) >= MAX_TASK_ATTEMPTS_PER_EXECUTOR } } - def isNodeBlacklistedForTask(node: String, index: Int): Boolean = { - nodeToBlacklistedTaskIndexes.get(node).exists(_.contains(index)) + def isNodeExcludedForTask(node: String, index: Int): Boolean = { + nodeToExcludedTaskIndexes.get(node).exists(_.contains(index)) } /** - * Return true if this executor is blacklisted for the given stage. Completely ignores whether - * the executor is blacklisted for the entire application (or anything to do with the node the + * Return true if this executor is excluded for the given stage. Completely ignores whether + * the executor is excluded for the entire application (or anything to do with the node the * executor is on). That is to keep this method as fast as possible in the inner-loop of the * scheduler, where those filters will already have been applied. */ - def isExecutorBlacklistedForTaskSet(executorId: String): Boolean = { - blacklistedExecs.contains(executorId) + def isExecutorExcludedForTaskSet(executorId: String): Boolean = { + excludedExecs.contains(executorId) } - def isNodeBlacklistedForTaskSet(node: String): Boolean = { - blacklistedNodes.contains(node) + def isNodeExcludedForTaskSet(node: String): Boolean = { + excludedNodes.contains(node) } - private[scheduler] def updateBlacklistForFailedTask( + private[scheduler] def updateExcludedForFailedTask( host: String, exec: String, index: Int, @@ -114,7 +114,7 @@ private[scheduler] class TaskSetBlacklist( execFailures.updateWithFailure(index, clock.getTimeMillis()) // check if this task has also failed on other executors on the same host -- if its gone - // over the limit, blacklist this task from the entire host. + // over the limit, exclude this task from the entire host. val execsWithFailuresOnNode = nodeToExecsWithFailures.getOrElseUpdate(host, new HashSet()) execsWithFailuresOnNode += exec val failuresOnHost = execsWithFailuresOnNode.toIterator.flatMap { exec => @@ -127,27 +127,35 @@ private[scheduler] class TaskSetBlacklist( } }.sum if (failuresOnHost >= MAX_TASK_ATTEMPTS_PER_NODE) { - nodeToBlacklistedTaskIndexes.getOrElseUpdate(host, new HashSet()) += index + nodeToExcludedTaskIndexes.getOrElseUpdate(host, new HashSet()) += index } - // Check if enough tasks have failed on the executor to blacklist it for the entire stage. + // Check if enough tasks have failed on the executor to exclude it for the entire stage. val numFailures = execFailures.numUniqueTasksWithFailures if (numFailures >= MAX_FAILURES_PER_EXEC_STAGE) { - if (blacklistedExecs.add(exec)) { - logInfo(s"Blacklisting executor ${exec} for stage $stageId") - // This executor has been pushed into the blacklist for this stage. Let's check if it - // pushes the whole node into the blacklist. - val blacklistedExecutorsOnNode = - execsWithFailuresOnNode.filter(blacklistedExecs.contains(_)) + if (excludedExecs.add(exec)) { + logInfo(s"Excluding executor ${exec} for stage $stageId") + // This executor has been excluded for this stage. Let's check if it + // the whole node should be excluded. + val excludedExecutorsOnNode = + execsWithFailuresOnNode.filter(excludedExecs.contains(_)) val now = clock.getTimeMillis() + // SparkListenerExecutorBlacklistedForStage is deprecated but post both events + // to keep backward compatibility listenerBus.post( SparkListenerExecutorBlacklistedForStage(now, exec, numFailures, stageId, stageAttemptId)) - val numFailExec = blacklistedExecutorsOnNode.size + listenerBus.post( + SparkListenerExecutorExcludedForStage(now, exec, numFailures, stageId, stageAttemptId)) + val numFailExec = excludedExecutorsOnNode.size if (numFailExec >= MAX_FAILED_EXEC_PER_NODE_STAGE) { - if (blacklistedNodes.add(host)) { - logInfo(s"Blacklisting ${host} for stage $stageId") + if (excludedNodes.add(host)) { + logInfo(s"Excluding ${host} for stage $stageId") + // SparkListenerNodeBlacklistedForStage is deprecated but post both events + // to keep backward compatibility listenerBus.post( SparkListenerNodeBlacklistedForStage(now, host, numFailExec, stageId, stageAttemptId)) + listenerBus.post( + SparkListenerNodeExcludedForStage(now, host, numFailExec, stageId, stageAttemptId)) } } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 78fd412ef154c..0cfa76583bfbb 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -55,7 +55,7 @@ private[spark] class TaskSetManager( sched: TaskSchedulerImpl, val taskSet: TaskSet, val maxTaskFailures: Int, - blacklistTracker: Option[BlacklistTracker] = None, + healthTracker: Option[HealthTracker] = None, clock: Clock = new SystemClock()) extends Schedulable with Logging { private val conf = sched.sc.conf @@ -130,9 +130,9 @@ private[spark] class TaskSetManager( private var totalResultSize = 0L private var calculatedTasks = 0 - private[scheduler] val taskSetBlacklistHelperOpt: Option[TaskSetBlacklist] = { - blacklistTracker.map { _ => - new TaskSetBlacklist(sched.sc.listenerBus, conf, stageId, taskSet.stageAttemptId, clock) + private[scheduler] val taskSetExcludelistHelperOpt: Option[TaskSetExcludelist] = { + healthTracker.map { _ => + new TaskSetExcludelist(sched.sc.listenerBus, conf, stageId, taskSet.stageAttemptId, clock) } } @@ -294,7 +294,7 @@ private[spark] class TaskSetManager( while (indexOffset > 0) { indexOffset -= 1 val index = list(indexOffset) - if (!isTaskBlacklistedOnExecOrNode(index, execId, host) && + if (!isTaskExcludededOnExecOrNode(index, execId, host) && !(speculative && hasAttemptOnHost(index, host))) { // This should almost always be list.trimEnd(1) to remove tail list.remove(indexOffset) @@ -317,10 +317,10 @@ private[spark] class TaskSetManager( taskAttempts(taskIndex).exists(_.host == host) } - private def isTaskBlacklistedOnExecOrNode(index: Int, execId: String, host: String): Boolean = { - taskSetBlacklistHelperOpt.exists { blacklist => - blacklist.isNodeBlacklistedForTask(host, index) || - blacklist.isExecutorBlacklistedForTask(execId, index) + private def isTaskExcludededOnExecOrNode(index: Int, execId: String, host: String): Boolean = { + taskSetExcludelistHelperOpt.exists { excludeList => + excludeList.isNodeExcludedForTask(host, index) || + excludeList.isExecutorExcludedForTask(execId, index) } } @@ -421,11 +421,11 @@ private[spark] class TaskSetManager( taskResourceAssignments: Map[String, ResourceInformation] = Map.empty) : (Option[TaskDescription], Boolean) = { - val offerBlacklisted = taskSetBlacklistHelperOpt.exists { blacklist => - blacklist.isNodeBlacklistedForTaskSet(host) || - blacklist.isExecutorBlacklistedForTaskSet(execId) + val offerExcluded = taskSetExcludelistHelperOpt.exists { excludeList => + excludeList.isNodeExcludedForTaskSet(host) || + excludeList.isExecutorExcludedForTaskSet(execId) } - if (!isZombie && !offerBlacklisted) { + if (!isZombie && !offerExcluded) { val curTime = clock.getTimeMillis() var allowedLocality = maxLocality @@ -518,10 +518,10 @@ private[spark] class TaskSetManager( if (isZombie && runningTasks == 0) { sched.taskSetFinished(this) if (tasksSuccessful == numTasks) { - blacklistTracker.foreach(_.updateBlacklistForSuccessfulTaskSet( + healthTracker.foreach(_.updateExcludedForSuccessfulTaskSet( taskSet.stageId, taskSet.stageAttemptId, - taskSetBlacklistHelperOpt.get.execToFailures)) + taskSetExcludelistHelperOpt.get.execToFailures)) } } } @@ -606,12 +606,13 @@ private[spark] class TaskSetManager( } /** - * Check whether the given task set has been blacklisted to the point that it can't run anywhere. + * Check whether the given task set has been excluded to the point that it can't run anywhere. * * It is possible that this taskset has become impossible to schedule *anywhere* due to the - * blacklist. The most common scenario would be if there are fewer executors than - * spark.task.maxFailures. We need to detect this so we can avoid the job from being hung. - * We try to acquire new executor/s by killing an existing idle blacklisted executor. + * failures that lead executors being excluded from the ones we can run on. The most common + * scenario would be if there are fewer executors than spark.task.maxFailures. + * We need to detect this so we can avoid the job from being hung. We try to acquire new + * executor/s by killing an existing idle excluded executor. * * There's a tradeoff here: we could make sure all tasks in the task set are schedulable, but that * would add extra time to each iteration of the scheduling loop. Here, we take the approach of @@ -620,12 +621,12 @@ private[spark] class TaskSetManager( * method is faster in the typical case. In the worst case, this method can take * O(maxTaskFailures + numTasks) time, but it will be faster when there haven't been any task * failures (this is because the method picks one unscheduled task, and then iterates through each - * executor until it finds one that the task isn't blacklisted on). + * executor until it finds one that the task isn't excluded on). */ - private[scheduler] def getCompletelyBlacklistedTaskIfAny( + private[scheduler] def getCompletelyExcludedTaskIfAny( hostToExecutors: HashMap[String, HashSet[String]]): Option[Int] = { - taskSetBlacklistHelperOpt.flatMap { taskSetBlacklist => - val appBlacklist = blacklistTracker.get + taskSetExcludelistHelperOpt.flatMap { taskSetExcludelist => + val appHealthTracker = healthTracker.get // Only look for unschedulable tasks when at least one executor has registered. Otherwise, // task sets will be (unnecessarily) aborted in cases when no executors have registered yet. if (hostToExecutors.nonEmpty) { @@ -651,18 +652,18 @@ private[spark] class TaskSetManager( // when that unschedulable task is the last task remaining. hostToExecutors.forall { case (host, execsOnHost) => // Check if the task can run on the node - val nodeBlacklisted = - appBlacklist.isNodeBlacklisted(host) || - taskSetBlacklist.isNodeBlacklistedForTaskSet(host) || - taskSetBlacklist.isNodeBlacklistedForTask(host, indexInTaskSet) - if (nodeBlacklisted) { + val nodeExcluded = + appHealthTracker.isNodeExcluded(host) || + taskSetExcludelist.isNodeExcludedForTaskSet(host) || + taskSetExcludelist.isNodeExcludedForTask(host, indexInTaskSet) + if (nodeExcluded) { true } else { // Check if the task can run on any of the executors execsOnHost.forall { exec => - appBlacklist.isExecutorBlacklisted(exec) || - taskSetBlacklist.isExecutorBlacklistedForTaskSet(exec) || - taskSetBlacklist.isExecutorBlacklistedForTask(exec, indexInTaskSet) + appHealthTracker.isExecutorExcluded(exec) || + taskSetExcludelist.isExecutorExcludedForTaskSet(exec) || + taskSetExcludelist.isExecutorExcludedForTask(exec, indexInTaskSet) } } } @@ -673,16 +674,16 @@ private[spark] class TaskSetManager( } } - private[scheduler] def abortSinceCompletelyBlacklisted(indexInTaskSet: Int): Unit = { - taskSetBlacklistHelperOpt.foreach { taskSetBlacklist => + private[scheduler] def abortSinceCompletelyExcludedOnFailure(indexInTaskSet: Int): Unit = { + taskSetExcludelistHelperOpt.foreach { taskSetExcludelist => val partition = tasks(indexInTaskSet).partitionId abort(s""" |Aborting $taskSet because task $indexInTaskSet (partition $partition) - |cannot run anywhere due to node and executor blacklist. + |cannot run anywhere due to node and executor excludeOnFailure. |Most recent failure: - |${taskSetBlacklist.getLatestFailureReason} + |${taskSetExcludelist.getLatestFailureReason} | - |Blacklisting behavior can be configured via spark.blacklist.*. + |ExcludeOnFailure behavior can be configured via spark.excludeOnFailure.*. |""".stripMargin) } } @@ -821,7 +822,7 @@ private[spark] class TaskSetManager( isZombie = true if (fetchFailed.bmAddress != null) { - blacklistTracker.foreach(_.updateBlacklistForFetchFailure( + healthTracker.foreach(_.updateExcludedForFetchFailure( fetchFailed.bmAddress.host, fetchFailed.bmAddress.executorId)) } @@ -899,7 +900,7 @@ private[spark] class TaskSetManager( if (!isZombie && reason.countTowardsTaskFailures) { assert (null != failureReason) - taskSetBlacklistHelperOpt.foreach(_.updateBlacklistForFailedTask( + taskSetExcludelistHelperOpt.foreach(_.updateExcludedForFailedTask( info.host, info.executorId, index, failureReason)) numFailures(index) += 1 if (numFailures(index) >= maxTaskFailures) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala index d1b0e798c51be..eda1cb52d4abc 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala @@ -131,7 +131,7 @@ private[spark] object CoarseGrainedClusterMessages { resourceProfileToTotalExecs: Map[ResourceProfile, Int], numLocalityAwareTasksPerResourceProfileId: Map[Int, Int], hostToLocalTaskCount: Map[Int, Map[String, Int]], - nodeBlacklist: Set[String]) + excludedNodes: Set[String]) extends CoarseGrainedClusterMessage // Check if an executor was force-killed but for a reason unrelated to the running tasks. diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 1d2689034f1ff..2bd0b4cc4b7d0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -209,13 +209,14 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp attributes, resources, resourceProfileId) => if (executorDataMap.contains(executorId)) { context.sendFailure(new IllegalStateException(s"Duplicate executor ID: $executorId")) - } else if (scheduler.nodeBlacklist.contains(hostname) || - isBlacklisted(executorId, hostname)) { - // If the cluster manager gives us an executor on a blacklisted node (because it - // already started allocating those resources before we informed it of our blacklist, - // or if it ignored our blacklist), then we reject that executor immediately. - logInfo(s"Rejecting $executorId as it has been blacklisted.") - context.sendFailure(new IllegalStateException(s"Executor is blacklisted: $executorId")) + } else if (scheduler.excludedNodes.contains(hostname) || + isExecutorExcluded(executorId, hostname)) { + // If the cluster manager gives us an executor on an excluded node (because it + // already started allocating those resources before we informed it of our exclusion, + // or if it ignored our exclusion), then we reject that executor immediately. + logInfo(s"Rejecting $executorId as it has been excluded.") + context.sendFailure( + new IllegalStateException(s"Executor is excluded due to failures: $executorId")) } else { // If the executor's rpc env is not listening for incoming connections, `hostPort` // will be null, and the client connection should be used to contact the executor. @@ -852,7 +853,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp final override def killExecutorsOnHost(host: String): Boolean = { logInfo(s"Requesting to kill any and all executors on host ${host}") // A potential race exists if a new executor attempts to register on a host - // that is on the blacklist and is no no longer valid. To avoid this race, + // that is on the exclude list and is no no longer valid. To avoid this race, // all executor registration and killing happens in the event loop. This way, either // an executor will fail to register, or will be killed when all executors on a host // are killed. @@ -884,13 +885,13 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp protected def currentDelegationTokens: Array[Byte] = delegationTokens.get() /** - * Checks whether the executor is blacklisted. This is called when the executor tries to - * register with the scheduler, and will deny registration if this method returns true. + * Checks whether the executor is excluded due to failure(s). This is called when the executor + * tries to register with the scheduler, and will deny registration if this method returns true. * - * This is in addition to the blacklist kept by the task scheduler, so custom implementations + * This is in addition to the exclude list kept by the task scheduler, so custom implementations * don't need to check there. */ - protected def isBlacklisted(executorId: String, hostname: String): Boolean = false + protected def isExecutorExcluded(executorId: String, hostname: String): Boolean = false // SPARK-27112: We need to ensure that there is ordering of lock acquisition // between TaskSchedulerImpl and CoarseGrainedSchedulerBackend objects in order to fix diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala index 7ae9117137caa..5b0c1dc389af0 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala @@ -283,82 +283,141 @@ private[spark] class AppStatusListener( } } + // Note, the blacklisted functions are left here for backwards compatibility to allow + // new history server to properly read and display older event logs. override def onExecutorBlacklisted(event: SparkListenerExecutorBlacklisted): Unit = { - updateBlackListStatus(event.executorId, true) + updateExecExclusionStatus(event.executorId, true) + } + + override def onExecutorExcluded(event: SparkListenerExecutorExcluded): Unit = { + updateExecExclusionStatus(event.executorId, true) } override def onExecutorBlacklistedForStage( event: SparkListenerExecutorBlacklistedForStage): Unit = { - val now = System.nanoTime() + updateExclusionStatusForStage(event.stageId, event.stageAttemptId, event.executorId) + } - Option(liveStages.get((event.stageId, event.stageAttemptId))).foreach { stage => - setStageBlackListStatus(stage, now, event.executorId) - } - liveExecutors.get(event.executorId).foreach { exec => - addBlackListedStageTo(exec, event.stageId, now) - } + override def onExecutorExcludedForStage( + event: SparkListenerExecutorExcludedForStage): Unit = { + updateExclusionStatusForStage(event.stageId, event.stageAttemptId, event.executorId) } override def onNodeBlacklistedForStage(event: SparkListenerNodeBlacklistedForStage): Unit = { - val now = System.nanoTime() + updateNodeExclusionStatusForStage(event.stageId, event.stageAttemptId, event.hostId) + } - // Implicitly blacklist every available executor for the stage associated with this node - Option(liveStages.get((event.stageId, event.stageAttemptId))).foreach { stage => - val executorIds = liveExecutors.values.filter(_.host == event.hostId).map(_.executorId).toSeq - setStageBlackListStatus(stage, now, executorIds: _*) - } - liveExecutors.values.filter(_.hostname == event.hostId).foreach { exec => - addBlackListedStageTo(exec, event.stageId, now) - } + override def onNodeExcludedForStage(event: SparkListenerNodeExcludedForStage): Unit = { + updateNodeExclusionStatusForStage(event.stageId, event.stageAttemptId, event.hostId) } - private def addBlackListedStageTo(exec: LiveExecutor, stageId: Int, now: Long): Unit = { - exec.blacklistedInStages += stageId + private def addExcludedStageTo(exec: LiveExecutor, stageId: Int, now: Long): Unit = { + exec.excludedInStages += stageId liveUpdate(exec, now) } private def setStageBlackListStatus(stage: LiveStage, now: Long, executorIds: String*): Unit = { executorIds.foreach { executorId => val executorStageSummary = stage.executorSummary(executorId) - executorStageSummary.isBlacklisted = true + executorStageSummary.isExcluded = true maybeUpdate(executorStageSummary, now) } - stage.blackListedExecutors ++= executorIds + stage.excludedExecutors ++= executorIds + maybeUpdate(stage, now) + } + + private def setStageExcludedStatus(stage: LiveStage, now: Long, executorIds: String*): Unit = { + executorIds.foreach { executorId => + val executorStageSummary = stage.executorSummary(executorId) + executorStageSummary.isExcluded = true + maybeUpdate(executorStageSummary, now) + } + stage.excludedExecutors ++= executorIds maybeUpdate(stage, now) } override def onExecutorUnblacklisted(event: SparkListenerExecutorUnblacklisted): Unit = { - updateBlackListStatus(event.executorId, false) + updateExecExclusionStatus(event.executorId, false) + } + + override def onExecutorUnexcluded(event: SparkListenerExecutorUnexcluded): Unit = { + updateExecExclusionStatus(event.executorId, false) } override def onNodeBlacklisted(event: SparkListenerNodeBlacklisted): Unit = { - updateNodeBlackList(event.hostId, true) + updateNodeExcluded(event.hostId, true) + } + + override def onNodeExcluded(event: SparkListenerNodeExcluded): Unit = { + updateNodeExcluded(event.hostId, true) } override def onNodeUnblacklisted(event: SparkListenerNodeUnblacklisted): Unit = { - updateNodeBlackList(event.hostId, false) + updateNodeExcluded(event.hostId, false) + } + + override def onNodeUnexcluded(event: SparkListenerNodeUnexcluded): Unit = { + updateNodeExcluded(event.hostId, false) } - private def updateBlackListStatus(execId: String, blacklisted: Boolean): Unit = { + private def updateNodeExclusionStatusForStage(stageId: Int, stageAttemptId: Int, + hostId: String): Unit = { + val now = System.nanoTime() + + // Implicitly exclude every available executor for the stage associated with this node + Option(liveStages.get((stageId, stageAttemptId))).foreach { stage => + val executorIds = liveExecutors.values.filter(_.host == hostId).map(_.executorId).toSeq + setStageExcludedStatus(stage, now, executorIds: _*) + } + liveExecutors.values.filter(_.hostname == hostId).foreach { exec => + addExcludedStageTo(exec, stageId, now) + } + } + + private def updateExclusionStatusForStage(stageId: Int, stageAttemptId: Int, + execId: String): Unit = { + val now = System.nanoTime() + + Option(liveStages.get((stageId, stageAttemptId))).foreach { stage => + setStageExcludedStatus(stage, now, execId) + } + liveExecutors.get(execId).foreach { exec => + addExcludedStageTo(exec, stageId, now) + } + } + + private def updateExecExclusionStatus(execId: String, excluded: Boolean): Unit = { liveExecutors.get(execId).foreach { exec => - exec.isBlacklisted = blacklisted - if (blacklisted) { + updateExecExclusionStatus(exec, excluded, System.nanoTime()) + } + } + + private def updateExecExclusionStatus(exec: LiveExecutor, excluded: Boolean, now: Long): Unit = { + // Since we are sending both blacklisted and excluded events for backwards compatibility + // we need to protect against double counting so don't increment if already in + // that state. Also protects against executor being excluded and then node being + // separately excluded which could result in this being called twice for same + // executor. + if (exec.isExcluded != excluded) { + if (excluded) { appStatusSource.foreach(_.BLACKLISTED_EXECUTORS.inc()) + appStatusSource.foreach(_.EXCLUDED_EXECUTORS.inc()) } else { appStatusSource.foreach(_.UNBLACKLISTED_EXECUTORS.inc()) + appStatusSource.foreach(_.UNEXCLUDED_EXECUTORS.inc()) } - liveUpdate(exec, System.nanoTime()) + exec.isExcluded = excluded + liveUpdate(exec, now) } } - private def updateNodeBlackList(host: String, blacklisted: Boolean): Unit = { + private def updateNodeExcluded(host: String, excluded: Boolean): Unit = { val now = System.nanoTime() - // Implicitly (un)blacklist every executor associated with the node. + // Implicitly (un)exclude every executor associated with the node. liveExecutors.values.foreach { exec => if (exec.hostname == host) { - exec.isBlacklisted = blacklisted - liveUpdate(exec, now) + updateExecExclusionStatus(exec, excluded, now) } } } @@ -759,10 +818,10 @@ private[spark] class AppStatusListener( update(pool, now) } - val executorIdsForStage = stage.blackListedExecutors + val executorIdsForStage = stage.excludedExecutors executorIdsForStage.foreach { executorId => liveExecutors.get(executorId).foreach { exec => - removeBlackListedStageFrom(exec, event.stageInfo.stageId, now) + removeExcludedStageFrom(exec, event.stageInfo.stageId, now) } } @@ -782,8 +841,8 @@ private[spark] class AppStatusListener( deadExecutors.retain((execId, exec) => isExecutorActiveForLiveStages(exec)) } - private def removeBlackListedStageFrom(exec: LiveExecutor, stageId: Int, now: Long) = { - exec.blacklistedInStages -= stageId + private def removeExcludedStageFrom(exec: LiveExecutor, stageId: Int, now: Long) = { + exec.excludedInStages -= stageId liveUpdate(exec, now) } diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusSource.scala b/core/src/main/scala/org/apache/spark/status/AppStatusSource.scala index 20f171bd3c375..d19744db089ba 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusSource.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusSource.scala @@ -59,9 +59,25 @@ private[spark] class AppStatusSource extends Source { val SKIPPED_TASKS = getCounter("tasks", "skippedTasks") + // This is the count of how many executors have been blacklisted at the application level, + // does not include stage level blacklisting. + // this is private but user visible from metrics so just deprecate + @deprecated("use excludedExecutors instead", "3.1.0") val BLACKLISTED_EXECUTORS = getCounter("tasks", "blackListedExecutors") + // This is the count of how many executors have been unblacklisted at the application level, + // does not include stage level unblacklisting. + @deprecated("use unexcludedExecutors instead", "3.1.0") val UNBLACKLISTED_EXECUTORS = getCounter("tasks", "unblackListedExecutors") + + // This is the count of how many executors have been excluded at the application level, + // does not include stage level exclusion. + val EXCLUDED_EXECUTORS = getCounter("tasks", "excludedExecutors") + + // This is the count of how many executors have been unexcluded at the application level, + // does not include stage level unexclusion. + val UNEXCLUDED_EXECUTORS = getCounter("tasks", "unexcludedExecutors") + } private[spark] object AppStatusSource { diff --git a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala index 0fadd330a01ad..38f1f25f2fcaa 100644 --- a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala +++ b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala @@ -286,8 +286,8 @@ private[spark] class LiveExecutor(val executorId: String, _addTime: Long) extend var totalInputBytes = 0L var totalShuffleRead = 0L var totalShuffleWrite = 0L - var isBlacklisted = false - var blacklistedInStages: Set[Int] = TreeSet() + var isExcluded = false + var excludedInStages: Set[Int] = TreeSet() var executorLogs = Map[String, String]() var attributes = Map[String, String]() @@ -334,18 +334,20 @@ private[spark] class LiveExecutor(val executorId: String, _addTime: Long) extend totalInputBytes, totalShuffleRead, totalShuffleWrite, - isBlacklisted, + isExcluded, maxMemory, addTime, Option(removeTime), Option(removeReason), executorLogs, memoryMetrics, - blacklistedInStages, + excludedInStages, Some(peakExecutorMetrics).filter(_.isSet), attributes, resources, - resourceProfileId) + resourceProfileId, + isExcluded, + excludedInStages) new ExecutorSummaryWrapper(info) } } @@ -361,7 +363,7 @@ private class LiveExecutorStageSummary( var succeededTasks = 0 var failedTasks = 0 var killedTasks = 0 - var isBlacklisted = false + var isExcluded = false var metrics = createMetrics(default = 0L) @@ -383,8 +385,9 @@ private class LiveExecutorStageSummary( metrics.shuffleWriteMetrics.recordsWritten, metrics.memoryBytesSpilled, metrics.diskBytesSpilled, - isBlacklisted, - Some(peakExecutorMetrics).filter(_.isSet)) + isExcluded, + Some(peakExecutorMetrics).filter(_.isSet), + isExcluded) new ExecutorStageSummaryWrapper(stageId, attemptId, executorId, info) } @@ -421,7 +424,7 @@ private class LiveStage extends LiveEntity { val activeTasksPerExecutor = new HashMap[String, Int]().withDefaultValue(0) - var blackListedExecutors = new HashSet[String]() + var excludedExecutors = new HashSet[String]() val peakExecutorMetrics = new ExecutorMetrics() diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala index 5a8cf09e1cba6..96f5b7b5cf27e 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala @@ -82,10 +82,12 @@ class ExecutorStageSummary private[spark]( val shuffleWriteRecords : Long, val memoryBytesSpilled : Long, val diskBytesSpilled : Long, + @deprecated("use isExcludedForStage instead", "3.1.0") val isBlacklistedForStage: Boolean, @JsonSerialize(using = classOf[ExecutorMetricsJsonSerializer]) @JsonDeserialize(using = classOf[ExecutorMetricsJsonDeserializer]) - val peakMemoryMetrics: Option[ExecutorMetrics]) + val peakMemoryMetrics: Option[ExecutorMetrics], + val isExcludedForStage: Boolean) class ExecutorSummary private[spark]( val id: String, @@ -105,6 +107,7 @@ class ExecutorSummary private[spark]( val totalInputBytes: Long, val totalShuffleRead: Long, val totalShuffleWrite: Long, + @deprecated("use isExcluded instead", "3.1.0") val isBlacklisted: Boolean, val maxMemory: Long, val addTime: Date, @@ -112,13 +115,16 @@ class ExecutorSummary private[spark]( val removeReason: Option[String], val executorLogs: Map[String, String], val memoryMetrics: Option[MemoryMetrics], + @deprecated("use excludedInStages instead", "3.1.0") val blacklistedInStages: Set[Int], @JsonSerialize(using = classOf[ExecutorMetricsJsonSerializer]) @JsonDeserialize(using = classOf[ExecutorMetricsJsonDeserializer]) val peakMemoryMetrics: Option[ExecutorMetrics], val attributes: Map[String, String], val resources: Map[String, ResourceInformation], - val resourceProfileId: Int) + val resourceProfileId: Int, + val isExcluded: Boolean, + val excludedInStages: Set[Int]) class MemoryMetrics private[spark]( val usedOnHeapStorageMemory: Long, diff --git a/core/src/main/scala/org/apache/spark/ui/ToolTips.scala b/core/src/main/scala/org/apache/spark/ui/ToolTips.scala index aefd001e573f9..a7c42b86468b2 100644 --- a/core/src/main/scala/org/apache/spark/ui/ToolTips.scala +++ b/core/src/main/scala/org/apache/spark/ui/ToolTips.scala @@ -91,9 +91,6 @@ private[spark] object ToolTips { val TASK_TIME = "Shaded red when garbage collection (GC) time is over 10% of task time" - val BLACKLISTED = - "Shows if this executor has been blacklisted by the scheduler due to task failures." - val APPLICATION_EXECUTOR_LIMIT = """Maximum number of executors that this application will use. This limit is finite only when dynamic allocation is enabled. The number of granted executors may exceed the limit diff --git a/core/src/test/resources/HistoryServerExpectations/blacklisting_for_stage_expectation.json b/core/src/test/resources/HistoryServerExpectations/excludeOnFailure_for_stage_expectation.json similarity index 99% rename from core/src/test/resources/HistoryServerExpectations/blacklisting_for_stage_expectation.json rename to core/src/test/resources/HistoryServerExpectations/excludeOnFailure_for_stage_expectation.json index 0d197eab0e25d..a69940fa5a1a5 100644 --- a/core/src/test/resources/HistoryServerExpectations/blacklisting_for_stage_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/excludeOnFailure_for_stage_expectation.json @@ -697,7 +697,8 @@ "shuffleWriteRecords" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, - "isBlacklistedForStage" : true + "isBlacklistedForStage" : true, + "isExcludedForStage" : true }, "1" : { "taskTime" : 708, @@ -714,7 +715,8 @@ "shuffleWriteRecords" : 10, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, - "isBlacklistedForStage" : false + "isBlacklistedForStage" : false, + "isExcludedForStage" : false } }, "killedTasksSummary" : { }, diff --git a/core/src/test/resources/HistoryServerExpectations/blacklisting_node_for_stage_expectation.json b/core/src/test/resources/HistoryServerExpectations/excludeOnFailure_node_for_stage_expectation.json similarity index 98% rename from core/src/test/resources/HistoryServerExpectations/blacklisting_node_for_stage_expectation.json rename to core/src/test/resources/HistoryServerExpectations/excludeOnFailure_node_for_stage_expectation.json index 24d73faa45021..bda9caedbbe81 100644 --- a/core/src/test/resources/HistoryServerExpectations/blacklisting_node_for_stage_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/excludeOnFailure_node_for_stage_expectation.json @@ -805,7 +805,8 @@ "shuffleWriteRecords" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, - "isBlacklistedForStage" : true + "isBlacklistedForStage" : true, + "isExcludedForStage" : true }, "5" : { "taskTime" : 1579, @@ -822,7 +823,8 @@ "shuffleWriteRecords" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, - "isBlacklistedForStage" : true + "isBlacklistedForStage" : true, + "isExcludedForStage" : true }, "1" : { "taskTime" : 2411, @@ -839,7 +841,8 @@ "shuffleWriteRecords" : 12, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, - "isBlacklistedForStage" : false + "isBlacklistedForStage" : false, + "isExcludedForStage" : false }, "2" : { "taskTime" : 2446, @@ -856,7 +859,8 @@ "shuffleWriteRecords" : 15, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, - "isBlacklistedForStage" : false + "isBlacklistedForStage" : false, + "isExcludedForStage" : false }, "3" : { "taskTime" : 1774, @@ -873,7 +877,8 @@ "shuffleWriteRecords" : 3, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, - "isBlacklistedForStage" : true + "isBlacklistedForStage" : true, + "isExcludedForStage" : true } }, "killedTasksSummary" : { }, diff --git a/core/src/test/resources/HistoryServerExpectations/executor_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_list_json_expectation.json index 67425676a62d6..c18a2e31dff3c 100644 --- a/core/src/test/resources/HistoryServerExpectations/executor_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/executor_list_json_expectation.json @@ -23,5 +23,7 @@ "blacklistedInStages" : [ ], "attributes" : { }, "resources" : { }, - "resourceProfileId" : 0 + "resourceProfileId" : 0, + "isExcluded" : false, + "excludedInStages" : [ ] } ] diff --git a/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_metrics_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_metrics_json_expectation.json index d052a27385f66..bf3e93f3d3783 100644 --- a/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_metrics_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_metrics_json_expectation.json @@ -51,7 +51,9 @@ }, "attributes" : { }, "resources" : { }, - "resourceProfileId" : 0 + "resourceProfileId" : 0, + "isExcluded" : false, + "excludedInStages" : [ ] }, { "id" : "3", "hostPort" : "test-3.vpc.company.com:37641", @@ -118,7 +120,9 @@ "CONTAINER_ID" : "container_1553914137147_0018_01_000004" }, "resources" : { }, - "resourceProfileId" : 0 + "resourceProfileId" : 0, + "isExcluded" : false, + "excludedInStages" : [ ] }, { "id" : "2", "hostPort" : "test-4.vpc.company.com:33179", @@ -185,7 +189,9 @@ "CONTAINER_ID" : "container_1553914137147_0018_01_000003" }, "resources" : { }, - "resourceProfileId" : 0 + "resourceProfileId" : 0, + "isExcluded" : false, + "excludedInStages" : [ ] }, { "id" : "1", "hostPort" : "test-2.vpc.company.com:43764", @@ -252,5 +258,7 @@ "CONTAINER_ID" : "container_1553914137147_0018_01_000002" }, "resources" : { }, - "resourceProfileId" : 0 + "resourceProfileId" : 0, + "isExcluded" : false, + "excludedInStages" : [ ] } ] diff --git a/core/src/test/resources/HistoryServerExpectations/executor_memory_usage_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_memory_usage_expectation.json index 91574ca8266b2..9adda275b5609 100644 --- a/core/src/test/resources/HistoryServerExpectations/executor_memory_usage_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/executor_memory_usage_expectation.json @@ -29,7 +29,9 @@ "blacklistedInStages" : [ ], "attributes" : { }, "resources" : { }, - "resourceProfileId" : 0 + "resourceProfileId" : 0, + "isExcluded" : true, + "excludedInStages" : [ ] }, { "id" : "3", "hostPort" : "172.22.0.167:51485", @@ -64,7 +66,9 @@ "blacklistedInStages" : [ ], "attributes" : { }, "resources" : { }, - "resourceProfileId" : 0 + "resourceProfileId" : 0, + "isExcluded" : true, + "excludedInStages" : [ ] } ,{ "id" : "2", "hostPort" : "172.22.0.167:51487", @@ -99,7 +103,9 @@ "blacklistedInStages" : [ ], "attributes" : { }, "resources" : { }, - "resourceProfileId" : 0 + "resourceProfileId" : 0, + "isExcluded" : true, + "excludedInStages" : [ ] }, { "id" : "1", "hostPort" : "172.22.0.167:51490", @@ -134,7 +140,9 @@ "blacklistedInStages" : [ ], "attributes" : { }, "resources" : { }, - "resourceProfileId" : 0 + "resourceProfileId" : 0, + "isExcluded" : true, + "excludedInStages" : [ ] }, { "id" : "0", "hostPort" : "172.22.0.167:51491", @@ -169,5 +177,7 @@ "blacklistedInStages" : [ ], "attributes" : { }, "resources" : { }, - "resourceProfileId" : 0 + "resourceProfileId" : 0, + "isExcluded" : true, + "excludedInStages" : [ ] } ] diff --git a/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_node_excludeOnFailure_expectation.json similarity index 92% rename from core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_expectation.json rename to core/src/test/resources/HistoryServerExpectations/executor_node_excludeOnFailure_expectation.json index f14b9a5085a42..65bd309c1025e 100644 --- a/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/executor_node_excludeOnFailure_expectation.json @@ -29,7 +29,9 @@ "blacklistedInStages" : [ ], "attributes" : { }, "resources" : { }, - "resourceProfileId" : 0 + "resourceProfileId" : 0, + "isExcluded" : true, + "excludedInStages" : [ ] }, { "id" : "3", "hostPort" : "172.22.0.167:51485", @@ -64,7 +66,9 @@ "blacklistedInStages" : [ ], "attributes" : { }, "resources" : { }, - "resourceProfileId" : 0 + "resourceProfileId" : 0, + "isExcluded" : true, + "excludedInStages" : [ ] }, { "id" : "2", "hostPort" : "172.22.0.167:51487", @@ -99,7 +103,9 @@ "blacklistedInStages" : [ ], "attributes" : { }, "resources" : { }, - "resourceProfileId" : 0 + "resourceProfileId" : 0, + "isExcluded" : true, + "excludedInStages" : [ ] }, { "id" : "1", "hostPort" : "172.22.0.167:51490", @@ -134,7 +140,9 @@ "blacklistedInStages" : [ ], "attributes" : { }, "resources" : { }, - "resourceProfileId" : 0 + "resourceProfileId" : 0, + "isExcluded" : true, + "excludedInStages" : [ ] }, { "id" : "0", "hostPort" : "172.22.0.167:51491", @@ -169,5 +177,7 @@ "blacklistedInStages" : [ ], "attributes" : { }, "resources" : { }, - "resourceProfileId" : 0 + "resourceProfileId" : 0, + "isExcluded" : true, + "excludedInStages" : [ ] } ] diff --git a/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_unblacklisting_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_node_excludeOnFailure_unexcluding_expectation.json similarity index 90% rename from core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_unblacklisting_expectation.json rename to core/src/test/resources/HistoryServerExpectations/executor_node_excludeOnFailure_unexcluding_expectation.json index 3645387317ca1..46e8f81d0e245 100644 --- a/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_unblacklisting_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/executor_node_excludeOnFailure_unexcluding_expectation.json @@ -23,7 +23,9 @@ "blacklistedInStages" : [ ], "attributes" : { }, "resources" : { }, - "resourceProfileId" : 0 + "resourceProfileId" : 0, + "isExcluded" : false, + "excludedInStages" : [ ] }, { "id" : "3", "hostPort" : "172.22.0.111:64543", @@ -52,7 +54,9 @@ "blacklistedInStages" : [ ], "attributes" : { }, "resources" : { }, - "resourceProfileId" : 0 + "resourceProfileId" : 0, + "isExcluded" : false, + "excludedInStages" : [ ] }, { "id" : "2", "hostPort" : "172.22.0.111:64539", @@ -81,7 +85,9 @@ "blacklistedInStages" : [ ], "attributes" : { }, "resources" : { }, - "resourceProfileId" : 0 + "resourceProfileId" : 0, + "isExcluded" : false, + "excludedInStages" : [ ] }, { "id" : "1", "hostPort" : "172.22.0.111:64541", @@ -110,7 +116,9 @@ "blacklistedInStages" : [ ], "attributes" : { }, "resources" : { }, - "resourceProfileId" : 0 + "resourceProfileId" : 0, + "isExcluded" : false, + "excludedInStages" : [ ] }, { "id" : "0", "hostPort" : "172.22.0.111:64540", @@ -139,5 +147,7 @@ "blacklistedInStages" : [ ], "attributes" : { }, "resources" : { }, - "resourceProfileId" : 0 + "resourceProfileId" : 0, + "isExcluded" : false, + "excludedInStages" : [ ] } ] diff --git a/core/src/test/resources/HistoryServerExpectations/executor_resource_information_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_resource_information_expectation.json index 165389cf25027..53ae9a0c7909e 100644 --- a/core/src/test/resources/HistoryServerExpectations/executor_resource_information_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/executor_resource_information_expectation.json @@ -29,7 +29,9 @@ "blacklistedInStages" : [ ], "attributes" : { }, "resources" : { }, - "resourceProfileId" : 0 + "resourceProfileId" : 0, + "isExcluded" : false, + "excludedInStages" : [ ] }, { "id" : "2", "hostPort" : "tomg-test:46005", @@ -79,7 +81,9 @@ "addresses" : [ "0", "1", "2", "3", "4", "5", "6", "7", "8", "9", "10", "11", "12" ] } }, - "resourceProfileId" : 0 + "resourceProfileId" : 0, + "isExcluded" : false, + "excludedInStages" : [ ] }, { "id" : "1", "hostPort" : "tomg-test:44873", @@ -129,5 +133,7 @@ "addresses" : [ "0", "1", "2", "3", "4", "5", "6", "7", "8", "9", "10", "11", "12" ] } }, - "resourceProfileId" : 0 + "resourceProfileId" : 0, + "isExcluded" : false, + "excludedInStages" : [ ] } ] diff --git a/core/src/test/resources/HistoryServerExpectations/one_stage_attempt_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/one_stage_attempt_json_expectation.json index 3db7d551b6130..41e54c68858ad 100644 --- a/core/src/test/resources/HistoryServerExpectations/one_stage_attempt_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/one_stage_attempt_json_expectation.json @@ -459,7 +459,8 @@ "shuffleWriteRecords" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, - "isBlacklistedForStage" : false + "isBlacklistedForStage" : false, + "isExcludedForStage" : false } }, "killedTasksSummary" : { }, diff --git a/core/src/test/resources/HistoryServerExpectations/one_stage_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/one_stage_json_expectation.json index 8ef3769c1ca6b..7a6685a609523 100644 --- a/core/src/test/resources/HistoryServerExpectations/one_stage_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/one_stage_json_expectation.json @@ -459,7 +459,8 @@ "shuffleWriteRecords" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, - "isBlacklistedForStage" : false + "isBlacklistedForStage" : false, + "isExcludedForStage" : false } }, "killedTasksSummary" : { }, diff --git a/core/src/test/resources/HistoryServerExpectations/stage_with_accumulable_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_with_accumulable_json_expectation.json index 3b5476ae8b160..066b6a4f884a7 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_with_accumulable_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_with_accumulable_json_expectation.json @@ -503,7 +503,8 @@ "shuffleWriteRecords" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, - "isBlacklistedForStage" : false + "isBlacklistedForStage" : false, + "isExcludedForStage" : false } }, "killedTasksSummary" : { }, diff --git a/core/src/test/resources/HistoryServerExpectations/stage_with_peak_metrics_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_with_peak_metrics_expectation.json index 373510d23058e..20a958073245a 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_with_peak_metrics_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_with_peak_metrics_expectation.json @@ -929,7 +929,8 @@ "MinorGCTime" : 0, "MajorGCCount" : 0, "MajorGCTime" : 0 - } + }, + "isExcludedForStage" : false }, "driver" : { "taskTime" : 0, @@ -968,7 +969,8 @@ "MinorGCTime" : 115, "MajorGCCount" : 4, "MajorGCTime" : 339 - } + }, + "isExcludedForStage" : false } }, "killedTasksSummary" : { }, diff --git a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala index 6a38bba5dd0e5..d1edb80e40b21 100644 --- a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala @@ -524,7 +524,7 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { assert(numExecutorsTarget(manager, defaultProfile.id) === 1) assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) == 1) - // Stage 0 becomes unschedulable due to blacklisting + // Stage 0 becomes unschedulable due to excludeOnFailure post(SparkListenerUnschedulableTaskSetAdded(0, 0)) clock.advance(1000) manager invokePrivate _updateAndSyncNumExecutorsTarget(clock.nanoTime()) @@ -580,7 +580,7 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { post(SparkListenerTaskEnd(0, 0, null, Success, t2Info, new ExecutorMetrics, null)) post(SparkListenerStageCompleted(createStageInfo(0, 2))) - // Stage 1 and 2 becomes unschedulable now due to blacklisting + // Stage 1 and 2 becomes unschedulable now due to excludeOnFailure post(SparkListenerUnschedulableTaskSetAdded(1, 0)) post(SparkListenerUnschedulableTaskSetAdded(2, 0)) @@ -637,7 +637,7 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { (0 to 3).foreach { i => assert(removeExecutorDefaultProfile(manager, i.toString)) } (0 to 3).foreach { i => onExecutorRemoved(manager, i.toString) } - // Now due to blacklisting, the task becomes unschedulable + // Now due to executor being excluded, the task becomes unschedulable post(SparkListenerUnschedulableTaskSetAdded(0, 0)) clock.advance(1000) manager invokePrivate _updateAndSyncNumExecutorsTarget(clock.nanoTime()) diff --git a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala index a2e70b23a3e5d..c9d43f517afba 100644 --- a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala +++ b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala @@ -76,7 +76,7 @@ class HeartbeatReceiverSuite sc = spy(new SparkContext(conf)) scheduler = mock(classOf[TaskSchedulerImpl]) when(sc.taskScheduler).thenReturn(scheduler) - when(scheduler.nodeBlacklist).thenReturn(Predef.Set[String]()) + when(scheduler.excludedNodes).thenReturn(Predef.Set[String]()) when(scheduler.sc).thenReturn(sc) heartbeatReceiverClock = new ManualClock heartbeatReceiver = new HeartbeatReceiver(sc, heartbeatReceiverClock) diff --git a/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala b/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala index c7c3ad27675fa..e1d4eff0a62cb 100644 --- a/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala @@ -497,19 +497,19 @@ class StandaloneDynamicAllocationSuite } } - test("executor registration on a blacklisted host must fail") { + test("executor registration on a excluded host must fail") { // The context isn't really used by the test, but it helps with creating a test scheduler, // since CoarseGrainedSchedulerBackend makes a lot of calls to the context instance. - sc = new SparkContext(appConf.set(config.BLACKLIST_ENABLED.key, "true")) + sc = new SparkContext(appConf.set(config.EXCLUDE_ON_FAILURE_ENABLED.key, "true")) val endpointRef = mock(classOf[RpcEndpointRef]) val mockAddress = mock(classOf[RpcAddress]) when(endpointRef.address).thenReturn(mockAddress) - val message = RegisterExecutor("one", endpointRef, "blacklisted-host", 10, Map.empty, + val message = RegisterExecutor("one", endpointRef, "excluded-host", 10, Map.empty, Map.empty, Map.empty, ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) val taskScheduler = mock(classOf[TaskSchedulerImpl]) - when(taskScheduler.nodeBlacklist()).thenReturn(Set("blacklisted-host")) + when(taskScheduler.excludedNodes()).thenReturn(Set("excluded-host")) when(taskScheduler.resourceOffers(any(), any[Boolean])).thenReturn(Nil) when(taskScheduler.sc).thenReturn(sc) diff --git a/core/src/test/scala/org/apache/spark/deploy/history/BasicEventFilterSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/BasicEventFilterSuite.scala index 2da40dccba53e..5d40a0610eb6c 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/BasicEventFilterSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/BasicEventFilterSuite.scala @@ -135,6 +135,8 @@ class BasicEventFilterSuite extends SparkFunSuite { SparkListenerStageExecutorMetrics(1.toString, 0, 0, new ExecutorMetrics))) assert(Some(false) === acceptFn(SparkListenerExecutorBlacklisted(0, 1.toString, 1))) assert(Some(false) === acceptFn(SparkListenerExecutorUnblacklisted(0, 1.toString))) + assert(Some(false) === acceptFn(SparkListenerExecutorExcluded(0, 1.toString, 1))) + assert(Some(false) === acceptFn(SparkListenerExecutorUnexcluded(0, 1.toString))) assert(Some(false) === acceptFn(createExecutorRemovedEvent(1))) val bmId = BlockManagerId(1.toString, "host1", 1) assert(Some(false) === acceptFn(SparkListenerBlockManagerAdded(0, bmId, 1))) @@ -148,6 +150,10 @@ class BasicEventFilterSuite extends SparkFunSuite { SparkListenerStageExecutorMetrics(2.toString, 0, 0, new ExecutorMetrics))) assert(Some(true) === acceptFn(SparkListenerExecutorBlacklisted(0, 2.toString, 1))) assert(Some(true) === acceptFn(SparkListenerExecutorUnblacklisted(0, 2.toString))) + assert(None === acceptFn(SparkListenerNodeBlacklisted(0, "host1", 1))) + assert(None === acceptFn(SparkListenerNodeUnblacklisted(0, "host1"))) + assert(Some(true) === acceptFn(SparkListenerExecutorExcluded(0, 2.toString, 1))) + assert(Some(true) === acceptFn(SparkListenerExecutorUnexcluded(0, 2.toString))) assert(Some(true) === acceptFn(createExecutorRemovedEvent(2))) val bmId2 = BlockManagerId(2.toString, "host1", 1) assert(Some(true) === acceptFn(SparkListenerBlockManagerAdded(0, bmId2, 1))) @@ -164,8 +170,8 @@ class BasicEventFilterSuite extends SparkFunSuite { assert(None === acceptFn(SparkListenerEnvironmentUpdate(Map.empty))) assert(None === acceptFn(SparkListenerApplicationStart("1", Some("1"), 0, "user", None))) assert(None === acceptFn(SparkListenerApplicationEnd(1))) - assert(None === acceptFn(SparkListenerNodeBlacklisted(0, "host1", 1))) - assert(None === acceptFn(SparkListenerNodeUnblacklisted(0, "host1"))) + assert(None === acceptFn(SparkListenerNodeExcluded(0, "host1", 1))) + assert(None === acceptFn(SparkListenerNodeUnexcluded(0, "host1"))) assert(None === acceptFn(SparkListenerLogStart("testVersion"))) } diff --git a/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileCompactorSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileCompactorSuite.scala index 2a914023ec821..ac39f022d5ca6 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileCompactorSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileCompactorSuite.scala @@ -219,10 +219,10 @@ class EventLogFileCompactorSuite extends SparkFunSuite { override def acceptFn(): PartialFunction[SparkListenerEvent, Boolean] = { case _: SparkListenerApplicationEnd => true case _: SparkListenerEnvironmentUpdate => true - case _: SparkListenerNodeBlacklisted => true + case _: SparkListenerNodeExcluded => true case _: SparkListenerBlockManagerAdded => false case _: SparkListenerApplicationStart => false - case _: SparkListenerNodeUnblacklisted => false + case _: SparkListenerNodeUnexcluded => false } override def statistics(): Option[EventFilter.FilterStatistics] = None @@ -254,11 +254,11 @@ class EventLogFileCompactorSuite extends SparkFunSuite { // filterApplicationStart: Some(false) & Some(false) => filter out writeEventToWriter(writer, SparkListenerApplicationStart("app", None, 0, "user", None)) - // filterNodeBlacklisted: None & Some(true) => filter in - expectedLines += writeEventToWriter(writer, SparkListenerNodeBlacklisted(0, "host1", 1)) + // filterNodeExcluded: None & Some(true) => filter in + expectedLines += writeEventToWriter(writer, SparkListenerNodeExcluded(0, "host1", 1)) - // filterNodeUnblacklisted: None & Some(false) => filter out - writeEventToWriter(writer, SparkListenerNodeUnblacklisted(0, "host1")) + // filterNodeUnexcluded: None & Some(false) => filter out + writeEventToWriter(writer, SparkListenerNodeUnexcluded(0, "host1")) // other events: None & None => filter in expectedLines += writeEventToWriter(writer, SparkListenerUnpersistRDD(0)) diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala index e4c23d3d1b1c3..08b2118065521 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala @@ -169,12 +169,13 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers "applications/local-1426533911241/1/stages/0/0/taskList", "stage task list from multi-attempt app json(2)" -> "applications/local-1426533911241/2/stages/0/0/taskList", - "blacklisting for stage" -> "applications/app-20180109111548-0000/stages/0/0", - "blacklisting node for stage" -> "applications/application_1516285256255_0012/stages/0/0", + "excludeOnFailure for stage" -> "applications/app-20180109111548-0000/stages/0/0", + "excludeOnFailure node for stage" -> "applications/application_1516285256255_0012/stages/0/0", "rdd list storage json" -> "applications/local-1422981780767/storage/rdd", - "executor node blacklisting" -> "applications/app-20161116163331-0000/executors", - "executor node blacklisting unblacklisting" -> "applications/app-20161115172038-0000/executors", + "executor node excludeOnFailure" -> "applications/app-20161116163331-0000/executors", + "executor node excludeOnFailure unexcluding" -> + "applications/app-20161115172038-0000/executors", "executor memory usage" -> "applications/app-20161116163331-0000/executors", "executor resource information" -> "applications/application_1555004656427_0144/executors", "multiple resource profiles" -> "applications/application_1578436911597_0052/environment", diff --git a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala deleted file mode 100644 index a1671a58f0d9b..0000000000000 --- a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala +++ /dev/null @@ -1,608 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.scheduler - -import org.mockito.ArgumentMatchers.any -import org.mockito.Mockito.{never, verify, when} -import org.mockito.invocation.InvocationOnMock -import org.scalatest.BeforeAndAfterEach -import org.scalatestplus.mockito.MockitoSugar - -import org.apache.spark._ -import org.apache.spark.internal.config -import org.apache.spark.util.ManualClock - -class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with MockitoSugar - with LocalSparkContext { - - private val clock = new ManualClock(0) - - private var blacklist: BlacklistTracker = _ - private var listenerBusMock: LiveListenerBus = _ - private var scheduler: TaskSchedulerImpl = _ - private var conf: SparkConf = _ - - override def beforeEach(): Unit = { - conf = new SparkConf().setAppName("test").setMaster("local") - .set(config.BLACKLIST_ENABLED.key, "true") - scheduler = mockTaskSchedWithConf(conf) - - clock.setTime(0) - - listenerBusMock = mock[LiveListenerBus] - blacklist = new BlacklistTracker(listenerBusMock, conf, None, clock) - } - - override def afterEach(): Unit = { - if (blacklist != null) { - blacklist = null - } - if (scheduler != null) { - scheduler.stop() - scheduler = null - } - super.afterEach() - } - - // All executors and hosts used in tests should be in this set, so that [[assertEquivalentToSet]] - // works. Its OK if its got extraneous entries - val allExecutorAndHostIds = { - (('A' to 'Z')++ (1 to 100).map(_.toString)) - .flatMap{ suffix => - Seq(s"host$suffix", s"host-$suffix") - } - }.toSet - - /** - * Its easier to write our tests as if we could directly look at the sets of nodes & executors in - * the blacklist. However the api doesn't expose a set, so this is a simple way to test - * something similar, since we know the universe of values that might appear in these sets. - */ - def assertEquivalentToSet(f: String => Boolean, expected: Set[String]): Unit = { - allExecutorAndHostIds.foreach { id => - val actual = f(id) - val exp = expected.contains(id) - assert(actual === exp, raw"""for string "$id" """) - } - } - - def mockTaskSchedWithConf(conf: SparkConf): TaskSchedulerImpl = { - sc = new SparkContext(conf) - val scheduler = mock[TaskSchedulerImpl] - when(scheduler.sc).thenReturn(sc) - when(scheduler.mapOutputTracker).thenReturn( - SparkEnv.get.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster]) - scheduler - } - - def createTaskSetBlacklist(stageId: Int = 0): TaskSetBlacklist = { - new TaskSetBlacklist(listenerBusMock, conf, stageId, stageAttemptId = 0, clock = clock) - } - - test("executors can be blacklisted with only a few failures per stage") { - // For many different stages, executor 1 fails a task, then executor 2 succeeds the task, - // and then the task set is done. Not enough failures to blacklist the executor *within* - // any particular taskset, but we still blacklist the executor overall eventually. - // Also, we intentionally have a mix of task successes and failures -- there are even some - // successes after the executor is blacklisted. The idea here is those tasks get scheduled - // before the executor is blacklisted. We might get successes after blacklisting (because the - // executor might be flaky but not totally broken). But successes should not unblacklist the - // executor. - val failuresUntilBlacklisted = conf.get(config.MAX_FAILURES_PER_EXEC) - var failuresSoFar = 0 - (0 until failuresUntilBlacklisted * 10).foreach { stageId => - val taskSetBlacklist = createTaskSetBlacklist(stageId) - if (stageId % 2 == 0) { - // fail one task in every other taskset - taskSetBlacklist.updateBlacklistForFailedTask( - "hostA", exec = "1", index = 0, failureReason = "testing") - failuresSoFar += 1 - } - blacklist.updateBlacklistForSuccessfulTaskSet(stageId, 0, taskSetBlacklist.execToFailures) - assert(failuresSoFar == stageId / 2 + 1) - if (failuresSoFar < failuresUntilBlacklisted) { - assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set()) - } else { - assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1")) - verify(listenerBusMock).post( - SparkListenerExecutorBlacklisted(0, "1", failuresUntilBlacklisted)) - } - } - } - - // If an executor has many task failures, but the task set ends up failing, it shouldn't be - // counted against the executor. - test("executors aren't blacklisted as a result of tasks in failed task sets") { - val failuresUntilBlacklisted = conf.get(config.MAX_FAILURES_PER_EXEC) - // for many different stages, executor 1 fails a task, and then the taskSet fails. - (0 until failuresUntilBlacklisted * 10).foreach { stage => - val taskSetBlacklist = createTaskSetBlacklist(stage) - taskSetBlacklist.updateBlacklistForFailedTask( - "hostA", exec = "1", index = 0, failureReason = "testing") - } - assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set()) - } - - Seq(true, false).foreach { succeedTaskSet => - val label = if (succeedTaskSet) "success" else "failure" - test(s"stage blacklist updates correctly on stage $label") { - // Within one taskset, an executor fails a few times, so it's blacklisted for the taskset. - // But if the taskset fails, we shouldn't blacklist the executor after the stage. - val taskSetBlacklist = createTaskSetBlacklist(0) - // We trigger enough failures for both the taskset blacklist, and the application blacklist. - val numFailures = math.max(conf.get(config.MAX_FAILURES_PER_EXEC), - conf.get(config.MAX_FAILURES_PER_EXEC_STAGE)) - (0 until numFailures).foreach { index => - taskSetBlacklist.updateBlacklistForFailedTask( - "hostA", exec = "1", index = index, failureReason = "testing") - } - assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("1")) - assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set()) - if (succeedTaskSet) { - // The task set succeeded elsewhere, so we should count those failures against our executor, - // and it should be blacklisted for the entire application. - blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, taskSetBlacklist.execToFailures) - assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1")) - verify(listenerBusMock).post(SparkListenerExecutorBlacklisted(0, "1", numFailures)) - } else { - // The task set failed, so we don't count these failures against the executor for other - // stages. - assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set()) - } - } - } - - test("blacklisted executors and nodes get recovered with time") { - val taskSetBlacklist0 = createTaskSetBlacklist(stageId = 0) - // Fail 4 tasks in one task set on executor 1, so that executor gets blacklisted for the whole - // application. - (0 until 4).foreach { partition => - taskSetBlacklist0.updateBlacklistForFailedTask( - "hostA", exec = "1", index = partition, failureReason = "testing") - } - blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, taskSetBlacklist0.execToFailures) - assert(blacklist.nodeBlacklist() === Set()) - assertEquivalentToSet(blacklist.isNodeBlacklisted(_), Set()) - assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1")) - verify(listenerBusMock).post(SparkListenerExecutorBlacklisted(0, "1", 4)) - - val taskSetBlacklist1 = createTaskSetBlacklist(stageId = 1) - // Fail 4 tasks in one task set on executor 2, so that executor gets blacklisted for the whole - // application. Since that's the second executor that is blacklisted on the same node, we also - // blacklist that node. - (0 until 4).foreach { partition => - taskSetBlacklist1.updateBlacklistForFailedTask( - "hostA", exec = "2", index = partition, failureReason = "testing") - } - blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, taskSetBlacklist1.execToFailures) - assert(blacklist.nodeBlacklist() === Set("hostA")) - assertEquivalentToSet(blacklist.isNodeBlacklisted(_), Set("hostA")) - verify(listenerBusMock).post(SparkListenerNodeBlacklisted(0, "hostA", 2)) - assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1", "2")) - verify(listenerBusMock).post(SparkListenerExecutorBlacklisted(0, "2", 4)) - - // Advance the clock and then make sure hostA and executors 1 and 2 have been removed from the - // blacklist. - val timeout = blacklist.BLACKLIST_TIMEOUT_MILLIS + 1 - clock.advance(timeout) - blacklist.applyBlacklistTimeout() - assert(blacklist.nodeBlacklist() === Set()) - assertEquivalentToSet(blacklist.isNodeBlacklisted(_), Set()) - assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set()) - verify(listenerBusMock).post(SparkListenerExecutorUnblacklisted(timeout, "2")) - verify(listenerBusMock).post(SparkListenerExecutorUnblacklisted(timeout, "1")) - verify(listenerBusMock).post(SparkListenerNodeUnblacklisted(timeout, "hostA")) - - // Fail one more task, but executor isn't put back into blacklist since the count of failures - // on that executor should have been reset to 0. - val taskSetBlacklist2 = createTaskSetBlacklist(stageId = 2) - taskSetBlacklist2.updateBlacklistForFailedTask( - "hostA", exec = "1", index = 0, failureReason = "testing") - blacklist.updateBlacklistForSuccessfulTaskSet(2, 0, taskSetBlacklist2.execToFailures) - assert(blacklist.nodeBlacklist() === Set()) - assertEquivalentToSet(blacklist.isNodeBlacklisted(_), Set()) - assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set()) - } - - test("blacklist can handle lost executors") { - // The blacklist should still work if an executor is killed completely. We should still - // be able to blacklist the entire node. - val taskSetBlacklist0 = createTaskSetBlacklist(stageId = 0) - // Lets say that executor 1 dies completely. We get some task failures, but - // the taskset then finishes successfully (elsewhere). - (0 until 4).foreach { partition => - taskSetBlacklist0.updateBlacklistForFailedTask( - "hostA", exec = "1", index = partition, failureReason = "testing") - } - blacklist.handleRemovedExecutor("1") - blacklist.updateBlacklistForSuccessfulTaskSet( - stageId = 0, - stageAttemptId = 0, - taskSetBlacklist0.execToFailures) - assert(blacklist.isExecutorBlacklisted("1")) - verify(listenerBusMock).post(SparkListenerExecutorBlacklisted(0, "1", 4)) - val t1 = blacklist.BLACKLIST_TIMEOUT_MILLIS / 2 - clock.advance(t1) - - // Now another executor gets spun up on that host, but it also dies. - val taskSetBlacklist1 = createTaskSetBlacklist(stageId = 1) - (0 until 4).foreach { partition => - taskSetBlacklist1.updateBlacklistForFailedTask( - "hostA", exec = "2", index = partition, failureReason = "testing") - } - blacklist.handleRemovedExecutor("2") - blacklist.updateBlacklistForSuccessfulTaskSet( - stageId = 1, - stageAttemptId = 0, - taskSetBlacklist1.execToFailures) - // We've now had two bad executors on the hostA, so we should blacklist the entire node. - assert(blacklist.isExecutorBlacklisted("1")) - assert(blacklist.isExecutorBlacklisted("2")) - verify(listenerBusMock).post(SparkListenerExecutorBlacklisted(t1, "2", 4)) - assert(blacklist.isNodeBlacklisted("hostA")) - verify(listenerBusMock).post(SparkListenerNodeBlacklisted(t1, "hostA", 2)) - - // Advance the clock so that executor 1 should no longer be explicitly blacklisted, but - // everything else should still be blacklisted. - val t2 = blacklist.BLACKLIST_TIMEOUT_MILLIS / 2 + 1 - clock.advance(t2) - blacklist.applyBlacklistTimeout() - assert(!blacklist.isExecutorBlacklisted("1")) - verify(listenerBusMock).post(SparkListenerExecutorUnblacklisted(t1 + t2, "1")) - assert(blacklist.isExecutorBlacklisted("2")) - assert(blacklist.isNodeBlacklisted("hostA")) - // make sure we don't leak memory - assert(!blacklist.executorIdToBlacklistStatus.contains("1")) - assert(!blacklist.nodeToBlacklistedExecs("hostA").contains("1")) - // Advance the timeout again so now hostA should be removed from the blacklist. - clock.advance(t1) - blacklist.applyBlacklistTimeout() - assert(!blacklist.nodeIdToBlacklistExpiryTime.contains("hostA")) - verify(listenerBusMock).post(SparkListenerNodeUnblacklisted(t1 + t2 + t1, "hostA")) - // Even though unblacklisting a node implicitly unblacklists all of its executors, - // there will be no SparkListenerExecutorUnblacklisted sent here. - } - - test("task failures expire with time") { - // Verifies that 2 failures within the timeout period cause an executor to be blacklisted, but - // if task failures are spaced out by more than the timeout period, the first failure is timed - // out, and the executor isn't blacklisted. - var stageId = 0 - - def failOneTaskInTaskSet(exec: String): Unit = { - val taskSetBlacklist = createTaskSetBlacklist(stageId = stageId) - taskSetBlacklist.updateBlacklistForFailedTask("host-" + exec, exec, 0, "testing") - blacklist.updateBlacklistForSuccessfulTaskSet(stageId, 0, taskSetBlacklist.execToFailures) - stageId += 1 - } - - failOneTaskInTaskSet(exec = "1") - // We have one sporadic failure on exec 2, but that's it. Later checks ensure that we never - // blacklist executor 2 despite this one failure. - failOneTaskInTaskSet(exec = "2") - assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set()) - assert(blacklist.nextExpiryTime === Long.MaxValue) - - // We advance the clock past the expiry time. - clock.advance(blacklist.BLACKLIST_TIMEOUT_MILLIS + 1) - val t0 = clock.getTimeMillis() - blacklist.applyBlacklistTimeout() - assert(blacklist.nextExpiryTime === Long.MaxValue) - failOneTaskInTaskSet(exec = "1") - - // Because the 2nd failure on executor 1 happened past the expiry time, nothing should have been - // blacklisted. - assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set()) - - // Now we add one more failure, within the timeout, and it should be counted. - clock.setTime(t0 + blacklist.BLACKLIST_TIMEOUT_MILLIS - 1) - val t1 = clock.getTimeMillis() - failOneTaskInTaskSet(exec = "1") - blacklist.applyBlacklistTimeout() - assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1")) - verify(listenerBusMock).post(SparkListenerExecutorBlacklisted(t1, "1", 2)) - assert(blacklist.nextExpiryTime === t1 + blacklist.BLACKLIST_TIMEOUT_MILLIS) - - // Add failures on executor 3, make sure it gets put on the blacklist. - clock.setTime(t1 + blacklist.BLACKLIST_TIMEOUT_MILLIS - 1) - val t2 = clock.getTimeMillis() - failOneTaskInTaskSet(exec = "3") - failOneTaskInTaskSet(exec = "3") - blacklist.applyBlacklistTimeout() - assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1", "3")) - verify(listenerBusMock).post(SparkListenerExecutorBlacklisted(t2, "3", 2)) - assert(blacklist.nextExpiryTime === t1 + blacklist.BLACKLIST_TIMEOUT_MILLIS) - - // Now we go past the timeout for executor 1, so it should be dropped from the blacklist. - clock.setTime(t1 + blacklist.BLACKLIST_TIMEOUT_MILLIS + 1) - blacklist.applyBlacklistTimeout() - assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("3")) - verify(listenerBusMock).post(SparkListenerExecutorUnblacklisted(clock.getTimeMillis(), "1")) - assert(blacklist.nextExpiryTime === t2 + blacklist.BLACKLIST_TIMEOUT_MILLIS) - - // Make sure that we update correctly when we go from having blacklisted executors to - // just having tasks with timeouts. - clock.setTime(t2 + blacklist.BLACKLIST_TIMEOUT_MILLIS - 1) - failOneTaskInTaskSet(exec = "4") - blacklist.applyBlacklistTimeout() - assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("3")) - assert(blacklist.nextExpiryTime === t2 + blacklist.BLACKLIST_TIMEOUT_MILLIS) - - clock.setTime(t2 + blacklist.BLACKLIST_TIMEOUT_MILLIS + 1) - blacklist.applyBlacklistTimeout() - assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set()) - verify(listenerBusMock).post(SparkListenerExecutorUnblacklisted(clock.getTimeMillis(), "3")) - // we've got one task failure still, but we don't bother setting nextExpiryTime to it, to - // avoid wasting time checking for expiry of individual task failures. - assert(blacklist.nextExpiryTime === Long.MaxValue) - } - - test("task failure timeout works as expected for long-running tasksets") { - // This ensures that we don't trigger spurious blacklisting for long tasksets, when the taskset - // finishes long after the task failures. We create two tasksets, each with one failure. - // Individually they shouldn't cause any blacklisting since there is only one failure. - // Furthermore, we space the failures out so far that even when both tasksets have completed, - // we still don't trigger any blacklisting. - val taskSetBlacklist1 = createTaskSetBlacklist(stageId = 1) - val taskSetBlacklist2 = createTaskSetBlacklist(stageId = 2) - // Taskset1 has one failure immediately - taskSetBlacklist1.updateBlacklistForFailedTask("host-1", "1", 0, "testing") - // Then we have a *long* delay, much longer than the timeout, before any other failures or - // taskset completion - clock.advance(blacklist.BLACKLIST_TIMEOUT_MILLIS * 5) - // After the long delay, we have one failure on taskset 2, on the same executor - taskSetBlacklist2.updateBlacklistForFailedTask("host-1", "1", 0, "testing") - // Finally, we complete both tasksets. Its important here to complete taskset2 *first*. We - // want to make sure that when taskset 1 finishes, even though we've now got two task failures, - // we realize that the task failure we just added was well before the timeout. - clock.advance(1) - blacklist.updateBlacklistForSuccessfulTaskSet(stageId = 2, 0, taskSetBlacklist2.execToFailures) - clock.advance(1) - blacklist.updateBlacklistForSuccessfulTaskSet(stageId = 1, 0, taskSetBlacklist1.execToFailures) - - // Make sure nothing was blacklisted - assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set()) - } - - test("only blacklist nodes for the application when enough executors have failed on that " + - "specific host") { - // we blacklist executors on two different hosts -- make sure that doesn't lead to any - // node blacklisting - val taskSetBlacklist0 = createTaskSetBlacklist(stageId = 0) - taskSetBlacklist0.updateBlacklistForFailedTask( - "hostA", exec = "1", index = 0, failureReason = "testing") - taskSetBlacklist0.updateBlacklistForFailedTask( - "hostA", exec = "1", index = 1, failureReason = "testing") - blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, taskSetBlacklist0.execToFailures) - assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1")) - verify(listenerBusMock).post(SparkListenerExecutorBlacklisted(0, "1", 2)) - assertEquivalentToSet(blacklist.isNodeBlacklisted(_), Set()) - - val taskSetBlacklist1 = createTaskSetBlacklist(stageId = 1) - taskSetBlacklist1.updateBlacklistForFailedTask( - "hostB", exec = "2", index = 0, failureReason = "testing") - taskSetBlacklist1.updateBlacklistForFailedTask( - "hostB", exec = "2", index = 1, failureReason = "testing") - blacklist.updateBlacklistForSuccessfulTaskSet(1, 0, taskSetBlacklist1.execToFailures) - assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1", "2")) - verify(listenerBusMock).post(SparkListenerExecutorBlacklisted(0, "2", 2)) - assertEquivalentToSet(blacklist.isNodeBlacklisted(_), Set()) - - // Finally, blacklist another executor on the same node as the original blacklisted executor, - // and make sure this time we *do* blacklist the node. - val taskSetBlacklist2 = createTaskSetBlacklist(stageId = 0) - taskSetBlacklist2.updateBlacklistForFailedTask( - "hostA", exec = "3", index = 0, failureReason = "testing") - taskSetBlacklist2.updateBlacklistForFailedTask( - "hostA", exec = "3", index = 1, failureReason = "testing") - blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, taskSetBlacklist2.execToFailures) - assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1", "2", "3")) - verify(listenerBusMock).post(SparkListenerExecutorBlacklisted(0, "3", 2)) - assertEquivalentToSet(blacklist.isNodeBlacklisted(_), Set("hostA")) - verify(listenerBusMock).post(SparkListenerNodeBlacklisted(0, "hostA", 2)) - } - - test("blacklist still respects legacy configs") { - val conf = new SparkConf().setMaster("local") - assert(!BlacklistTracker.isBlacklistEnabled(conf)) - conf.set(config.BLACKLIST_LEGACY_TIMEOUT_CONF, 5000L) - assert(BlacklistTracker.isBlacklistEnabled(conf)) - assert(5000 === BlacklistTracker.getBlacklistTimeout(conf)) - // the new conf takes precedence, though - conf.set(config.BLACKLIST_TIMEOUT_CONF, 1000L) - assert(1000 === BlacklistTracker.getBlacklistTimeout(conf)) - - // if you explicitly set the legacy conf to 0, that also would disable blacklisting - conf.set(config.BLACKLIST_LEGACY_TIMEOUT_CONF, 0L) - assert(!BlacklistTracker.isBlacklistEnabled(conf)) - // but again, the new conf takes precedence - conf.set(config.BLACKLIST_ENABLED, true) - assert(BlacklistTracker.isBlacklistEnabled(conf)) - assert(1000 === BlacklistTracker.getBlacklistTimeout(conf)) - } - - test("check blacklist configuration invariants") { - val conf = new SparkConf().setMaster("yarn").set(config.SUBMIT_DEPLOY_MODE, "cluster") - Seq( - (2, 2), - (2, 3) - ).foreach { case (maxTaskFailures, maxNodeAttempts) => - conf.set(config.TASK_MAX_FAILURES, maxTaskFailures) - conf.set(config.MAX_TASK_ATTEMPTS_PER_NODE.key, maxNodeAttempts.toString) - val excMsg = intercept[IllegalArgumentException] { - BlacklistTracker.validateBlacklistConfs(conf) - }.getMessage() - assert(excMsg === s"${config.MAX_TASK_ATTEMPTS_PER_NODE.key} " + - s"( = ${maxNodeAttempts}) was >= ${config.TASK_MAX_FAILURES.key} " + - s"( = ${maxTaskFailures} ). Though blacklisting is enabled, with this configuration, " + - s"Spark will not be robust to one bad node. Decrease " + - s"${config.MAX_TASK_ATTEMPTS_PER_NODE.key}, increase ${config.TASK_MAX_FAILURES.key}, " + - s"or disable blacklisting with ${config.BLACKLIST_ENABLED.key}") - } - - conf.remove(config.TASK_MAX_FAILURES) - conf.remove(config.MAX_TASK_ATTEMPTS_PER_NODE) - - Seq( - config.MAX_TASK_ATTEMPTS_PER_EXECUTOR, - config.MAX_TASK_ATTEMPTS_PER_NODE, - config.MAX_FAILURES_PER_EXEC_STAGE, - config.MAX_FAILED_EXEC_PER_NODE_STAGE, - config.MAX_FAILURES_PER_EXEC, - config.MAX_FAILED_EXEC_PER_NODE, - config.BLACKLIST_TIMEOUT_CONF - ).foreach { config => - conf.set(config.key, "0") - val excMsg = intercept[IllegalArgumentException] { - BlacklistTracker.validateBlacklistConfs(conf) - }.getMessage() - assert(excMsg.contains(s"${config.key} was 0, but must be > 0.")) - conf.remove(config) - } - } - - test("blacklisting kills executors, configured by BLACKLIST_KILL_ENABLED") { - val allocationClientMock = mock[ExecutorAllocationClient] - when(allocationClientMock.killExecutors(any(), any(), any(), any())).thenReturn(Seq("called")) - when(allocationClientMock.killExecutorsOnHost("hostA")).thenAnswer { (_: InvocationOnMock) => - // 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. - if (blacklist.nodeBlacklist.contains("hostA")) { - true - } else { - throw new IllegalStateException("hostA should be on the blacklist") - } - } - 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) - - val taskSetBlacklist0 = createTaskSetBlacklist(stageId = 0) - // Fail 4 tasks in one task set on executor 1, so that executor gets blacklisted for the whole - // application. - (0 until 4).foreach { partition => - taskSetBlacklist0.updateBlacklistForFailedTask( - "hostA", exec = "1", index = partition, failureReason = "testing") - } - blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, taskSetBlacklist0.execToFailures) - - verify(allocationClientMock, never).killExecutor(any()) - - val taskSetBlacklist1 = createTaskSetBlacklist(stageId = 1) - // Fail 4 tasks in one task set on executor 2, so that executor gets blacklisted for the whole - // application. Since that's the second executor that is blacklisted on the same node, we also - // blacklist that node. - (0 until 4).foreach { partition => - taskSetBlacklist1.updateBlacklistForFailedTask( - "hostA", exec = "2", index = partition, failureReason = "testing") - } - blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, taskSetBlacklist1.execToFailures) - - verify(allocationClientMock, never).killExecutors(any(), 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) - - val taskSetBlacklist2 = createTaskSetBlacklist(stageId = 0) - // Fail 4 tasks in one task set on executor 1, so that executor gets blacklisted for the whole - // application. - (0 until 4).foreach { partition => - taskSetBlacklist2.updateBlacklistForFailedTask( - "hostA", exec = "1", index = partition, failureReason = "testing") - } - blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, taskSetBlacklist2.execToFailures) - - verify(allocationClientMock).killExecutors(Seq("1"), false, false, true) - - val taskSetBlacklist3 = createTaskSetBlacklist(stageId = 1) - // Fail 4 tasks in one task set on executor 2, so that executor gets blacklisted for the whole - // application. Since that's the second executor that is blacklisted on the same node, we also - // blacklist that node. - (0 until 4).foreach { partition => - taskSetBlacklist3.updateBlacklistForFailedTask( - "hostA", exec = "2", index = partition, failureReason = "testing") - } - blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, taskSetBlacklist3.execToFailures) - - verify(allocationClientMock).killExecutors(Seq("2"), false, false, 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(), any())).thenReturn(Seq("called")) - when(allocationClientMock.killExecutorsOnHost("hostA")).thenAnswer { (_: InvocationOnMock) => - // 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. - if (blacklist.nodeBlacklist.contains("hostA")) { - true - } else { - throw new IllegalStateException("hostA should be on the blacklist") - } - } - - 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(), any()) - verify(allocationClientMock, never).killExecutorsOnHost(any()) - - assert(blacklist.nodeToBlacklistedExecs.contains("hostA")) - assert(blacklist.nodeToBlacklistedExecs("hostA").contains("1")) - - // 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"), false, false, 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) - assert(blacklist.nodeToBlacklistedExecs.contains("hostA")) - assert(blacklist.nodeToBlacklistedExecs("hostA").contains("1")) - - // 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) - } -} diff --git a/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala index d648293fdbe06..47e37fc55cefe 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala @@ -300,7 +300,7 @@ private class CSMockExternalClusterManager extends ExternalClusterManager { when(ts.applicationId()).thenReturn("appid1") when(ts.applicationAttemptId()).thenReturn(Some("attempt1")) when(ts.schedulingMode).thenReturn(SchedulingMode.FIFO) - when(ts.nodeBlacklist()).thenReturn(Set.empty[String]) + when(ts.excludedNodes()).thenReturn(Set.empty[String]) ts } diff --git a/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/HealthTrackerIntegrationSuite.scala similarity index 86% rename from core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala rename to core/src/test/scala/org/apache/spark/scheduler/HealthTrackerIntegrationSuite.scala index 246d4b2f56ec9..29a8f4be8b72b 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/HealthTrackerIntegrationSuite.scala @@ -20,7 +20,7 @@ import org.apache.spark._ import org.apache.spark.internal.config import org.apache.spark.internal.config.Tests._ -class BlacklistIntegrationSuite extends SchedulerIntegrationSuite[MultiExecutorMockBackend]{ +class HealthTrackerIntegrationSuite extends SchedulerIntegrationSuite[MultiExecutorMockBackend]{ val badHost = "host-0" @@ -40,9 +40,9 @@ class BlacklistIntegrationSuite extends SchedulerIntegrationSuite[MultiExecutorM // Test demonstrating the issue -- without a config change, the scheduler keeps scheduling // according to locality preferences, and so the job fails - testScheduler("If preferred node is bad, without blacklist job will fail", + testScheduler("If preferred node is bad, without excludeOnFailure job will fail", extraConfs = Seq( - config.BLACKLIST_ENABLED.key -> "false" + config.EXCLUDE_ON_FAILURE_ENABLED.key -> "false" )) { val rdd = new MockRDDWithLocalityPrefs(sc, 10, Nil, badHost) withBackend(badHostBackend _) { @@ -55,19 +55,19 @@ class BlacklistIntegrationSuite extends SchedulerIntegrationSuite[MultiExecutorM testScheduler( "With default settings, job can succeed despite multiple bad executors on node", extraConfs = Seq( - config.BLACKLIST_ENABLED.key -> "true", + config.EXCLUDE_ON_FAILURE_ENABLED.key -> "true", config.TASK_MAX_FAILURES.key -> "4", TEST_N_HOSTS.key -> "2", TEST_N_EXECUTORS_HOST.key -> "5", TEST_N_CORES_EXECUTOR.key -> "10" ) ) { - // To reliably reproduce the failure that would occur without blacklisting, we have to use 1 + // To reliably reproduce the failure that would occur without exludeOnFailure, we have to use 1 // task. That way, we ensure this 1 task gets rotated through enough bad executors on the host // to fail the taskSet, before we have a bunch of different tasks fail in the executors so we - // blacklist them. - // But the point here is -- without blacklisting, we would never schedule anything on the good - // host-1 before we hit too many failures trying our preferred host-0. + // exclude them. + // But the point here is -- without excludeOnFailure, we would never schedule anything on the + // good host-1 before we hit too many failures trying our preferred host-0. val rdd = new MockRDDWithLocalityPrefs(sc, 1, Nil, badHost) withBackend(badHostBackend _) { val jobFuture = submit(rdd, (0 until 1).toArray) @@ -76,12 +76,12 @@ class BlacklistIntegrationSuite extends SchedulerIntegrationSuite[MultiExecutorM assertDataStructuresEmpty(noFailure = true) } - // Here we run with the blacklist on, and the default config takes care of having this + // Here we run with the excludeOnFailure on, and the default config takes care of having this // robust to one bad node. testScheduler( "Bad node with multiple executors, job will still succeed with the right confs", extraConfs = Seq( - config.BLACKLIST_ENABLED.key -> "true", + config.EXCLUDE_ON_FAILURE_ENABLED.key -> "true", // just to avoid this test taking too long config.LOCALITY_WAIT.key -> "10ms" ) @@ -100,7 +100,7 @@ class BlacklistIntegrationSuite extends SchedulerIntegrationSuite[MultiExecutorM testScheduler( "SPARK-15865 Progress with fewer executors than maxTaskFailures", extraConfs = Seq( - config.BLACKLIST_ENABLED.key -> "true", + config.EXCLUDE_ON_FAILURE_ENABLED.key -> "true", TEST_N_HOSTS.key -> "2", TEST_N_EXECUTORS_HOST.key -> "1", TEST_N_CORES_EXECUTOR.key -> "1", @@ -116,7 +116,7 @@ class BlacklistIntegrationSuite extends SchedulerIntegrationSuite[MultiExecutorM awaitJobTermination(jobFuture, duration) val pattern = ( s"""|Aborting TaskSet 0.0 because task .* - |cannot run anywhere due to node and executor blacklist""".stripMargin).r + |cannot run anywhere due to node and executor excludeOnFailure""".stripMargin).r assert(pattern.findFirstIn(failure.getMessage).isDefined, s"Couldn't find $pattern in ${failure.getMessage()}") } diff --git a/core/src/test/scala/org/apache/spark/scheduler/HealthTrackerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/HealthTrackerSuite.scala new file mode 100644 index 0000000000000..7ecc1f51ce236 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/scheduler/HealthTrackerSuite.scala @@ -0,0 +1,615 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.scheduler + +import org.mockito.ArgumentMatchers.any +import org.mockito.Mockito.{never, verify, when} +import org.mockito.invocation.InvocationOnMock +import org.scalatest.BeforeAndAfterEach +import org.scalatestplus.mockito.MockitoSugar + +import org.apache.spark._ +import org.apache.spark.internal.config +import org.apache.spark.util.ManualClock + +class HealthTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with MockitoSugar + with LocalSparkContext { + + private val clock = new ManualClock(0) + + private var healthTracker: HealthTracker = _ + private var listenerBusMock: LiveListenerBus = _ + private var scheduler: TaskSchedulerImpl = _ + private var conf: SparkConf = _ + + override def beforeEach(): Unit = { + conf = new SparkConf().setAppName("test").setMaster("local") + .set(config.EXCLUDE_ON_FAILURE_ENABLED.key, "true") + scheduler = mockTaskSchedWithConf(conf) + + clock.setTime(0) + + listenerBusMock = mock[LiveListenerBus] + healthTracker = new HealthTracker(listenerBusMock, conf, None, clock) + } + + override def afterEach(): Unit = { + if (healthTracker != null) { + healthTracker = null + } + if (scheduler != null) { + scheduler.stop() + scheduler = null + } + super.afterEach() + } + + // All executors and hosts used in tests should be in this set, so that [[assertEquivalentToSet]] + // works. Its OK if its got extraneous entries + val allExecutorAndHostIds = { + (('A' to 'Z')++ (1 to 100).map(_.toString)) + .flatMap{ suffix => + Seq(s"host$suffix", s"host-$suffix") + } + }.toSet + + /** + * Its easier to write our tests as if we could directly look at the sets of nodes & executors in + * the exclude. However the api doesn't expose a set, so this is a simple way to test + * something similar, since we know the universe of values that might appear in these sets. + */ + def assertEquivalentToSet(f: String => Boolean, expected: Set[String]): Unit = { + allExecutorAndHostIds.foreach { id => + val actual = f(id) + val exp = expected.contains(id) + assert(actual === exp, raw"""for string "$id" """) + } + } + + def mockTaskSchedWithConf(conf: SparkConf): TaskSchedulerImpl = { + sc = new SparkContext(conf) + val scheduler = mock[TaskSchedulerImpl] + when(scheduler.sc).thenReturn(sc) + when(scheduler.mapOutputTracker).thenReturn( + SparkEnv.get.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster]) + scheduler + } + + def createTaskSetExcludelist(stageId: Int = 0): TaskSetExcludelist = { + new TaskSetExcludelist(listenerBusMock, conf, stageId, stageAttemptId = 0, clock = clock) + } + + test("executors can be excluded with only a few failures per stage") { + // For many different stages, executor 1 fails a task, then executor 2 succeeds the task, + // and then the task set is done. Not enough failures to exclude the executor *within* + // any particular taskset, but we still exclude the executor overall eventually. + // Also, we intentionally have a mix of task successes and failures -- there are even some + // successes after the executor is excluded. The idea here is those tasks get scheduled + // before the executor is excluded. We might get successes after excluding (because the + // executor might be flaky but not totally broken). But successes should not unexclude the + // executor. + val failuresUntilExcludeed = conf.get(config.MAX_FAILURES_PER_EXEC) + var failuresSoFar = 0 + (0 until failuresUntilExcludeed * 10).foreach { stageId => + val taskSetExclude = createTaskSetExcludelist(stageId) + if (stageId % 2 == 0) { + // fail one task in every other taskset + taskSetExclude.updateExcludedForFailedTask( + "hostA", exec = "1", index = 0, failureReason = "testing") + failuresSoFar += 1 + } + healthTracker.updateExcludedForSuccessfulTaskSet(stageId, 0, taskSetExclude.execToFailures) + assert(failuresSoFar == stageId / 2 + 1) + if (failuresSoFar < failuresUntilExcludeed) { + assertEquivalentToSet(healthTracker.isExecutorExcluded(_), Set()) + } else { + assertEquivalentToSet(healthTracker.isExecutorExcluded(_), Set("1")) + verify(listenerBusMock).post( + SparkListenerExecutorExcluded(0, "1", failuresUntilExcludeed)) + verify(listenerBusMock).post( + SparkListenerExecutorBlacklisted(0, "1", failuresUntilExcludeed)) + } + } + } + + // If an executor has many task failures, but the task set ends up failing, it shouldn't be + // counted against the executor. + test("executors aren't excluded as a result of tasks in failed task sets") { + val failuresUntilExcludeed = conf.get(config.MAX_FAILURES_PER_EXEC) + // for many different stages, executor 1 fails a task, and then the taskSet fails. + (0 until failuresUntilExcludeed * 10).foreach { stage => + val taskSetExclude = createTaskSetExcludelist(stage) + taskSetExclude.updateExcludedForFailedTask( + "hostA", exec = "1", index = 0, failureReason = "testing") + } + assertEquivalentToSet(healthTracker.isExecutorExcluded(_), Set()) + } + + Seq(true, false).foreach { succeedTaskSet => + val label = if (succeedTaskSet) "success" else "failure" + test(s"stage exclude updates correctly on stage $label") { + // Within one taskset, an executor fails a few times, so it's excluded for the taskset. + // But if the taskset fails, we shouldn't exclude the executor after the stage. + val taskSetExclude = createTaskSetExcludelist(0) + // We trigger enough failures for both the taskset exclude, and the application exclude. + val numFailures = math.max(conf.get(config.MAX_FAILURES_PER_EXEC), + conf.get(config.MAX_FAILURES_PER_EXEC_STAGE)) + (0 until numFailures).foreach { index => + taskSetExclude.updateExcludedForFailedTask( + "hostA", exec = "1", index = index, failureReason = "testing") + } + assert(taskSetExclude.isExecutorExcludedForTaskSet("1")) + assertEquivalentToSet(healthTracker.isExecutorExcluded(_), Set()) + if (succeedTaskSet) { + // The task set succeeded elsewhere, so we should count those failures against our executor, + // and it should be excluded for the entire application. + healthTracker.updateExcludedForSuccessfulTaskSet(0, 0, taskSetExclude.execToFailures) + assertEquivalentToSet(healthTracker.isExecutorExcluded(_), Set("1")) + verify(listenerBusMock).post(SparkListenerExecutorExcluded(0, "1", numFailures)) + } else { + // The task set failed, so we don't count these failures against the executor for other + // stages. + assertEquivalentToSet(healthTracker.isExecutorExcluded(_), Set()) + } + } + } + + test("excluded executors and nodes get recovered with time") { + val taskSetExclude0 = createTaskSetExcludelist(stageId = 0) + // Fail 4 tasks in one task set on executor 1, so that executor gets excluded for the whole + // application. + (0 until 4).foreach { partition => + taskSetExclude0.updateExcludedForFailedTask( + "hostA", exec = "1", index = partition, failureReason = "testing") + } + healthTracker.updateExcludedForSuccessfulTaskSet(0, 0, taskSetExclude0.execToFailures) + assert(healthTracker.excludedNodeList() === Set()) + assertEquivalentToSet(healthTracker.isNodeExcluded(_), Set()) + assertEquivalentToSet(healthTracker.isExecutorExcluded(_), Set("1")) + verify(listenerBusMock).post(SparkListenerExecutorExcluded(0, "1", 4)) + verify(listenerBusMock).post(SparkListenerExecutorBlacklisted(0, "1", 4)) + + val taskSetExclude1 = createTaskSetExcludelist(stageId = 1) + // Fail 4 tasks in one task set on executor 2, so that executor gets excluded for the whole + // application. Since that's the second executor that is excluded on the same node, we also + // exclude that node. + (0 until 4).foreach { partition => + taskSetExclude1.updateExcludedForFailedTask( + "hostA", exec = "2", index = partition, failureReason = "testing") + } + healthTracker.updateExcludedForSuccessfulTaskSet(0, 0, taskSetExclude1.execToFailures) + assert(healthTracker.excludedNodeList() === Set("hostA")) + assertEquivalentToSet(healthTracker.isNodeExcluded(_), Set("hostA")) + verify(listenerBusMock).post(SparkListenerNodeExcluded(0, "hostA", 2)) + verify(listenerBusMock).post(SparkListenerNodeBlacklisted(0, "hostA", 2)) + assertEquivalentToSet(healthTracker.isExecutorExcluded(_), Set("1", "2")) + verify(listenerBusMock).post(SparkListenerExecutorExcluded(0, "2", 4)) + verify(listenerBusMock).post(SparkListenerExecutorBlacklisted(0, "2", 4)) + + // Advance the clock and then make sure hostA and executors 1 and 2 have been removed from the + // exclude. + val timeout = healthTracker.EXCLUDE_ON_FAILURE_TIMEOUT_MILLIS + 1 + clock.advance(timeout) + healthTracker.applyExcludeOnFailureTimeout() + assert(healthTracker.excludedNodeList() === Set()) + assertEquivalentToSet(healthTracker.isNodeExcluded(_), Set()) + assertEquivalentToSet(healthTracker.isExecutorExcluded(_), Set()) + verify(listenerBusMock).post(SparkListenerExecutorUnexcluded(timeout, "2")) + verify(listenerBusMock).post(SparkListenerExecutorUnexcluded(timeout, "1")) + verify(listenerBusMock).post(SparkListenerExecutorUnblacklisted(timeout, "2")) + verify(listenerBusMock).post(SparkListenerExecutorUnblacklisted(timeout, "1")) + verify(listenerBusMock).post(SparkListenerNodeUnexcluded(timeout, "hostA")) + + // Fail one more task, but executor isn't put back into exclude since the count of failures + // on that executor should have been reset to 0. + val taskSetExclude2 = createTaskSetExcludelist(stageId = 2) + taskSetExclude2.updateExcludedForFailedTask( + "hostA", exec = "1", index = 0, failureReason = "testing") + healthTracker.updateExcludedForSuccessfulTaskSet(2, 0, taskSetExclude2.execToFailures) + assert(healthTracker.excludedNodeList() === Set()) + assertEquivalentToSet(healthTracker.isNodeExcluded(_), Set()) + assertEquivalentToSet(healthTracker.isExecutorExcluded(_), Set()) + } + + test("exclude can handle lost executors") { + // The exclude should still work if an executor is killed completely. We should still + // be able to exclude the entire node. + val taskSetExclude0 = createTaskSetExcludelist(stageId = 0) + // Lets say that executor 1 dies completely. We get some task failures, but + // the taskset then finishes successfully (elsewhere). + (0 until 4).foreach { partition => + taskSetExclude0.updateExcludedForFailedTask( + "hostA", exec = "1", index = partition, failureReason = "testing") + } + healthTracker.handleRemovedExecutor("1") + healthTracker.updateExcludedForSuccessfulTaskSet( + stageId = 0, + stageAttemptId = 0, + taskSetExclude0.execToFailures) + assert(healthTracker.isExecutorExcluded("1")) + verify(listenerBusMock).post(SparkListenerExecutorExcluded(0, "1", 4)) + val t1 = healthTracker.EXCLUDE_ON_FAILURE_TIMEOUT_MILLIS / 2 + clock.advance(t1) + + // Now another executor gets spun up on that host, but it also dies. + val taskSetExclude1 = createTaskSetExcludelist(stageId = 1) + (0 until 4).foreach { partition => + taskSetExclude1.updateExcludedForFailedTask( + "hostA", exec = "2", index = partition, failureReason = "testing") + } + healthTracker.handleRemovedExecutor("2") + healthTracker.updateExcludedForSuccessfulTaskSet( + stageId = 1, + stageAttemptId = 0, + taskSetExclude1.execToFailures) + // We've now had two bad executors on the hostA, so we should exclude the entire node. + assert(healthTracker.isExecutorExcluded("1")) + assert(healthTracker.isExecutorExcluded("2")) + verify(listenerBusMock).post(SparkListenerExecutorExcluded(t1, "2", 4)) + assert(healthTracker.isNodeExcluded("hostA")) + verify(listenerBusMock).post(SparkListenerNodeExcluded(t1, "hostA", 2)) + + // Advance the clock so that executor 1 should no longer be explicitly excluded, but + // everything else should still be excluded. + val t2 = healthTracker.EXCLUDE_ON_FAILURE_TIMEOUT_MILLIS / 2 + 1 + clock.advance(t2) + healthTracker.applyExcludeOnFailureTimeout() + assert(!healthTracker.isExecutorExcluded("1")) + verify(listenerBusMock).post(SparkListenerExecutorUnexcluded(t1 + t2, "1")) + assert(healthTracker.isExecutorExcluded("2")) + assert(healthTracker.isNodeExcluded("hostA")) + // make sure we don't leak memory + assert(!healthTracker.executorIdToExcludedStatus.contains("1")) + assert(!healthTracker.nodeToExcludedExecs("hostA").contains("1")) + // Advance the timeout again so now hostA should be removed from the exclude. + clock.advance(t1) + healthTracker.applyExcludeOnFailureTimeout() + assert(!healthTracker.nodeIdToExcludedExpiryTime.contains("hostA")) + verify(listenerBusMock).post(SparkListenerNodeUnexcluded(t1 + t2 + t1, "hostA")) + // Even though unexcluding a node implicitly unexcludes all of its executors, + // there will be no SparkListenerExecutorUnexcluded sent here. + } + + test("task failures expire with time") { + // Verifies that 2 failures within the timeout period cause an executor to be excluded, but + // if task failures are spaced out by more than the timeout period, the first failure is timed + // out, and the executor isn't excluded. + var stageId = 0 + + def failOneTaskInTaskSet(exec: String): Unit = { + val taskSetExclude = createTaskSetExcludelist(stageId = stageId) + taskSetExclude.updateExcludedForFailedTask("host-" + exec, exec, 0, "testing") + healthTracker.updateExcludedForSuccessfulTaskSet(stageId, 0, taskSetExclude.execToFailures) + stageId += 1 + } + + failOneTaskInTaskSet(exec = "1") + // We have one sporadic failure on exec 2, but that's it. Later checks ensure that we never + // exclude executor 2 despite this one failure. + failOneTaskInTaskSet(exec = "2") + assertEquivalentToSet(healthTracker.isExecutorExcluded(_), Set()) + assert(healthTracker.nextExpiryTime === Long.MaxValue) + + // We advance the clock past the expiry time. + clock.advance(healthTracker.EXCLUDE_ON_FAILURE_TIMEOUT_MILLIS + 1) + val t0 = clock.getTimeMillis() + healthTracker.applyExcludeOnFailureTimeout() + assert(healthTracker.nextExpiryTime === Long.MaxValue) + failOneTaskInTaskSet(exec = "1") + + // Because the 2nd failure on executor 1 happened past the expiry time, nothing should have been + // excluded. + assertEquivalentToSet(healthTracker.isExecutorExcluded(_), Set()) + + // Now we add one more failure, within the timeout, and it should be counted. + clock.setTime(t0 + healthTracker.EXCLUDE_ON_FAILURE_TIMEOUT_MILLIS - 1) + val t1 = clock.getTimeMillis() + failOneTaskInTaskSet(exec = "1") + healthTracker.applyExcludeOnFailureTimeout() + assertEquivalentToSet(healthTracker.isExecutorExcluded(_), Set("1")) + verify(listenerBusMock).post(SparkListenerExecutorExcluded(t1, "1", 2)) + assert(healthTracker.nextExpiryTime === t1 + healthTracker.EXCLUDE_ON_FAILURE_TIMEOUT_MILLIS) + + // Add failures on executor 3, make sure it gets put on the exclude. + clock.setTime(t1 + healthTracker.EXCLUDE_ON_FAILURE_TIMEOUT_MILLIS - 1) + val t2 = clock.getTimeMillis() + failOneTaskInTaskSet(exec = "3") + failOneTaskInTaskSet(exec = "3") + healthTracker.applyExcludeOnFailureTimeout() + assertEquivalentToSet(healthTracker.isExecutorExcluded(_), Set("1", "3")) + verify(listenerBusMock).post(SparkListenerExecutorExcluded(t2, "3", 2)) + assert(healthTracker.nextExpiryTime === t1 + healthTracker.EXCLUDE_ON_FAILURE_TIMEOUT_MILLIS) + + // Now we go past the timeout for executor 1, so it should be dropped from the exclude. + clock.setTime(t1 + healthTracker.EXCLUDE_ON_FAILURE_TIMEOUT_MILLIS + 1) + healthTracker.applyExcludeOnFailureTimeout() + assertEquivalentToSet(healthTracker.isExecutorExcluded(_), Set("3")) + verify(listenerBusMock).post(SparkListenerExecutorUnexcluded(clock.getTimeMillis(), "1")) + assert(healthTracker.nextExpiryTime === t2 + healthTracker.EXCLUDE_ON_FAILURE_TIMEOUT_MILLIS) + + // Make sure that we update correctly when we go from having excluded executors to + // just having tasks with timeouts. + clock.setTime(t2 + healthTracker.EXCLUDE_ON_FAILURE_TIMEOUT_MILLIS - 1) + failOneTaskInTaskSet(exec = "4") + healthTracker.applyExcludeOnFailureTimeout() + assertEquivalentToSet(healthTracker.isExecutorExcluded(_), Set("3")) + assert(healthTracker.nextExpiryTime === t2 + healthTracker.EXCLUDE_ON_FAILURE_TIMEOUT_MILLIS) + + clock.setTime(t2 + healthTracker.EXCLUDE_ON_FAILURE_TIMEOUT_MILLIS + 1) + healthTracker.applyExcludeOnFailureTimeout() + assertEquivalentToSet(healthTracker.isExecutorExcluded(_), Set()) + verify(listenerBusMock).post(SparkListenerExecutorUnexcluded(clock.getTimeMillis(), "3")) + // we've got one task failure still, but we don't bother setting nextExpiryTime to it, to + // avoid wasting time checking for expiry of individual task failures. + assert(healthTracker.nextExpiryTime === Long.MaxValue) + } + + test("task failure timeout works as expected for long-running tasksets") { + // This ensures that we don't trigger spurious excluding for long tasksets, when the taskset + // finishes long after the task failures. We create two tasksets, each with one failure. + // Individually they shouldn't cause any excluding since there is only one failure. + // Furthermore, we space the failures out so far that even when both tasksets have completed, + // we still don't trigger any excluding. + val taskSetExclude1 = createTaskSetExcludelist(stageId = 1) + val taskSetExclude2 = createTaskSetExcludelist(stageId = 2) + // Taskset1 has one failure immediately + taskSetExclude1.updateExcludedForFailedTask("host-1", "1", 0, "testing") + // Then we have a *long* delay, much longer than the timeout, before any other failures or + // taskset completion + clock.advance(healthTracker.EXCLUDE_ON_FAILURE_TIMEOUT_MILLIS * 5) + // After the long delay, we have one failure on taskset 2, on the same executor + taskSetExclude2.updateExcludedForFailedTask("host-1", "1", 0, "testing") + // Finally, we complete both tasksets. Its important here to complete taskset2 *first*. We + // want to make sure that when taskset 1 finishes, even though we've now got two task failures, + // we realize that the task failure we just added was well before the timeout. + clock.advance(1) + healthTracker.updateExcludedForSuccessfulTaskSet(stageId = 2, 0, taskSetExclude2.execToFailures) + clock.advance(1) + healthTracker.updateExcludedForSuccessfulTaskSet(stageId = 1, 0, taskSetExclude1.execToFailures) + + // Make sure nothing was excluded + assertEquivalentToSet(healthTracker.isExecutorExcluded(_), Set()) + } + + test("only exclude nodes for the application when enough executors have failed on that " + + "specific host") { + // we exclude executors on two different hosts -- make sure that doesn't lead to any + // node excluding + val taskSetExclude0 = createTaskSetExcludelist(stageId = 0) + taskSetExclude0.updateExcludedForFailedTask( + "hostA", exec = "1", index = 0, failureReason = "testing") + taskSetExclude0.updateExcludedForFailedTask( + "hostA", exec = "1", index = 1, failureReason = "testing") + healthTracker.updateExcludedForSuccessfulTaskSet(0, 0, taskSetExclude0.execToFailures) + assertEquivalentToSet(healthTracker.isExecutorExcluded(_), Set("1")) + verify(listenerBusMock).post(SparkListenerExecutorExcluded(0, "1", 2)) + assertEquivalentToSet(healthTracker.isNodeExcluded(_), Set()) + + val taskSetExclude1 = createTaskSetExcludelist(stageId = 1) + taskSetExclude1.updateExcludedForFailedTask( + "hostB", exec = "2", index = 0, failureReason = "testing") + taskSetExclude1.updateExcludedForFailedTask( + "hostB", exec = "2", index = 1, failureReason = "testing") + healthTracker.updateExcludedForSuccessfulTaskSet(1, 0, taskSetExclude1.execToFailures) + assertEquivalentToSet(healthTracker.isExecutorExcluded(_), Set("1", "2")) + verify(listenerBusMock).post(SparkListenerExecutorExcluded(0, "2", 2)) + assertEquivalentToSet(healthTracker.isNodeExcluded(_), Set()) + + // Finally, exclude another executor on the same node as the original excluded executor, + // and make sure this time we *do* exclude the node. + val taskSetExclude2 = createTaskSetExcludelist(stageId = 0) + taskSetExclude2.updateExcludedForFailedTask( + "hostA", exec = "3", index = 0, failureReason = "testing") + taskSetExclude2.updateExcludedForFailedTask( + "hostA", exec = "3", index = 1, failureReason = "testing") + healthTracker.updateExcludedForSuccessfulTaskSet(0, 0, taskSetExclude2.execToFailures) + assertEquivalentToSet(healthTracker.isExecutorExcluded(_), Set("1", "2", "3")) + verify(listenerBusMock).post(SparkListenerExecutorExcluded(0, "3", 2)) + assertEquivalentToSet(healthTracker.isNodeExcluded(_), Set("hostA")) + verify(listenerBusMock).post(SparkListenerNodeExcluded(0, "hostA", 2)) + } + + test("exclude still respects legacy configs") { + val conf = new SparkConf().setMaster("local") + assert(!HealthTracker.isExcludeOnFailureEnabled(conf)) + conf.set(config.EXCLUDE_ON_FAILURE_LEGACY_TIMEOUT_CONF, 5000L) + assert(HealthTracker.isExcludeOnFailureEnabled(conf)) + assert(5000 === HealthTracker.getExludeOnFailureTimeout(conf)) + // the new conf takes precedence, though + conf.set(config.EXCLUDE_ON_FAILURE_TIMEOUT_CONF, 1000L) + assert(1000 === HealthTracker.getExludeOnFailureTimeout(conf)) + + // if you explicitly set the legacy conf to 0, that also would disable excluding + conf.set(config.EXCLUDE_ON_FAILURE_LEGACY_TIMEOUT_CONF, 0L) + assert(!HealthTracker.isExcludeOnFailureEnabled(conf)) + // but again, the new conf takes precedence + conf.set(config.EXCLUDE_ON_FAILURE_ENABLED, true) + assert(HealthTracker.isExcludeOnFailureEnabled(conf)) + assert(1000 === HealthTracker.getExludeOnFailureTimeout(conf)) + } + + test("check exclude configuration invariants") { + val conf = new SparkConf().setMaster("yarn").set(config.SUBMIT_DEPLOY_MODE, "cluster") + Seq( + (2, 2), + (2, 3) + ).foreach { case (maxTaskFailures, maxNodeAttempts) => + conf.set(config.TASK_MAX_FAILURES, maxTaskFailures) + conf.set(config.MAX_TASK_ATTEMPTS_PER_NODE.key, maxNodeAttempts.toString) + val excMsg = intercept[IllegalArgumentException] { + HealthTracker.validateExcludeOnFailureConfs(conf) + }.getMessage() + assert(excMsg === s"${config.MAX_TASK_ATTEMPTS_PER_NODE.key} " + + s"( = ${maxNodeAttempts}) was >= ${config.TASK_MAX_FAILURES.key} " + + s"( = ${maxTaskFailures} ). Though excludeOnFailure is enabled, with this " + + s"configuration, Spark will not be robust to one bad node. Decrease " + + s"${config.MAX_TASK_ATTEMPTS_PER_NODE.key}, increase ${config.TASK_MAX_FAILURES.key}, " + + s"or disable excludeOnFailure with ${config.EXCLUDE_ON_FAILURE_ENABLED.key}") + } + + conf.remove(config.TASK_MAX_FAILURES) + conf.remove(config.MAX_TASK_ATTEMPTS_PER_NODE) + + Seq( + config.MAX_TASK_ATTEMPTS_PER_EXECUTOR, + config.MAX_TASK_ATTEMPTS_PER_NODE, + config.MAX_FAILURES_PER_EXEC_STAGE, + config.MAX_FAILED_EXEC_PER_NODE_STAGE, + config.MAX_FAILURES_PER_EXEC, + config.MAX_FAILED_EXEC_PER_NODE, + config.EXCLUDE_ON_FAILURE_TIMEOUT_CONF + ).foreach { config => + conf.set(config.key, "0") + val excMsg = intercept[IllegalArgumentException] { + HealthTracker.validateExcludeOnFailureConfs(conf) + }.getMessage() + assert(excMsg.contains(s"${config.key} was 0, but must be > 0.")) + conf.remove(config) + } + } + + test("excluding kills executors, configured by EXCLUDE_ON_FAILURE_KILL_ENABLED") { + val allocationClientMock = mock[ExecutorAllocationClient] + when(allocationClientMock.killExecutors(any(), any(), any(), any())).thenReturn(Seq("called")) + when(allocationClientMock.killExecutorsOnHost("hostA")).thenAnswer { (_: InvocationOnMock) => + // To avoid a race between excluding and killing, it is important that the nodeExclude + // is updated before we ask the executor allocation client to kill all the executors + // on a particular host. + if (healthTracker.excludedNodeList().contains("hostA")) { + true + } else { + throw new IllegalStateException("hostA should be on the exclude") + } + } + healthTracker = new HealthTracker(listenerBusMock, conf, Some(allocationClientMock), clock) + + // Disable auto-kill. Exclude an executor and make sure killExecutors is not called. + conf.set(config.EXCLUDE_ON_FAILURE_KILL_ENABLED, false) + + val taskSetExclude0 = createTaskSetExcludelist(stageId = 0) + // Fail 4 tasks in one task set on executor 1, so that executor gets excluded for the whole + // application. + (0 until 4).foreach { partition => + taskSetExclude0.updateExcludedForFailedTask( + "hostA", exec = "1", index = partition, failureReason = "testing") + } + healthTracker.updateExcludedForSuccessfulTaskSet(0, 0, taskSetExclude0.execToFailures) + + verify(allocationClientMock, never).killExecutor(any()) + + val taskSetExclude1 = createTaskSetExcludelist(stageId = 1) + // Fail 4 tasks in one task set on executor 2, so that executor gets excluded for the whole + // application. Since that's the second executor that is excluded on the same node, we also + // exclude that node. + (0 until 4).foreach { partition => + taskSetExclude1.updateExcludedForFailedTask( + "hostA", exec = "2", index = partition, failureReason = "testing") + } + healthTracker.updateExcludedForSuccessfulTaskSet(0, 0, taskSetExclude1.execToFailures) + + verify(allocationClientMock, never).killExecutors(any(), any(), any(), any()) + verify(allocationClientMock, never).killExecutorsOnHost(any()) + + // Enable auto-kill. Exclude an executor and make sure killExecutors is called. + conf.set(config.EXCLUDE_ON_FAILURE_KILL_ENABLED, true) + healthTracker = new HealthTracker(listenerBusMock, conf, Some(allocationClientMock), clock) + + val taskSetExclude2 = createTaskSetExcludelist(stageId = 0) + // Fail 4 tasks in one task set on executor 1, so that executor gets excluded for the whole + // application. + (0 until 4).foreach { partition => + taskSetExclude2.updateExcludedForFailedTask( + "hostA", exec = "1", index = partition, failureReason = "testing") + } + healthTracker.updateExcludedForSuccessfulTaskSet(0, 0, taskSetExclude2.execToFailures) + + verify(allocationClientMock).killExecutors(Seq("1"), false, false, true) + + val taskSetExclude3 = createTaskSetExcludelist(stageId = 1) + // Fail 4 tasks in one task set on executor 2, so that executor gets excluded for the whole + // application. Since that's the second executor that is excluded on the same node, we also + // exclude that node. + (0 until 4).foreach { partition => + taskSetExclude3.updateExcludedForFailedTask( + "hostA", exec = "2", index = partition, failureReason = "testing") + } + healthTracker.updateExcludedForSuccessfulTaskSet(0, 0, taskSetExclude3.execToFailures) + + verify(allocationClientMock).killExecutors(Seq("2"), false, false, true) + verify(allocationClientMock).killExecutorsOnHost("hostA") + } + + test("fetch failure excluding kills executors, configured by EXCLUDE_ON_FAILURE_KILL_ENABLED") { + val allocationClientMock = mock[ExecutorAllocationClient] + when(allocationClientMock.killExecutors(any(), any(), any(), any())).thenReturn(Seq("called")) + when(allocationClientMock.killExecutorsOnHost("hostA")).thenAnswer { (_: InvocationOnMock) => + // To avoid a race between excluding and killing, it is important that the nodeExclude + // is updated before we ask the executor allocation client to kill all the executors + // on a particular host. + if (healthTracker.excludedNodeList().contains("hostA")) { + true + } else { + throw new IllegalStateException("hostA should be on the exclude") + } + } + + conf.set(config.EXCLUDE_ON_FAILURE_FETCH_FAILURE_ENABLED, true) + healthTracker = new HealthTracker(listenerBusMock, conf, Some(allocationClientMock), clock) + + // Disable auto-kill. Exclude an executor and make sure killExecutors is not called. + conf.set(config.EXCLUDE_ON_FAILURE_KILL_ENABLED, false) + healthTracker.updateExcludedForFetchFailure("hostA", exec = "1") + + verify(allocationClientMock, never).killExecutors(any(), any(), any(), any()) + verify(allocationClientMock, never).killExecutorsOnHost(any()) + + assert(healthTracker.nodeToExcludedExecs.contains("hostA")) + assert(healthTracker.nodeToExcludedExecs("hostA").contains("1")) + + // Enable auto-kill. Exclude an executor and make sure killExecutors is called. + conf.set(config.EXCLUDE_ON_FAILURE_KILL_ENABLED, true) + healthTracker = new HealthTracker(listenerBusMock, conf, Some(allocationClientMock), clock) + clock.advance(1000) + healthTracker.updateExcludedForFetchFailure("hostA", exec = "1") + + verify(allocationClientMock).killExecutors(Seq("1"), false, false, true) + verify(allocationClientMock, never).killExecutorsOnHost(any()) + + assert(healthTracker.executorIdToExcludedStatus.contains("1")) + assert(healthTracker.executorIdToExcludedStatus("1").node === "hostA") + assert(healthTracker.executorIdToExcludedStatus("1").expiryTime === + 1000 + healthTracker.EXCLUDE_ON_FAILURE_TIMEOUT_MILLIS) + assert(healthTracker.nextExpiryTime === 1000 + healthTracker.EXCLUDE_ON_FAILURE_TIMEOUT_MILLIS) + assert(healthTracker.nodeIdToExcludedExpiryTime.isEmpty) + assert(healthTracker.nodeToExcludedExecs.contains("hostA")) + assert(healthTracker.nodeToExcludedExecs("hostA").contains("1")) + + // 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) + healthTracker.updateExcludedForFetchFailure("hostA", exec = "2") + + verify(allocationClientMock, never).killExecutors(Seq("2"), true, true) + verify(allocationClientMock).killExecutorsOnHost("hostA") + + assert(healthTracker.nodeIdToExcludedExpiryTime.contains("hostA")) + assert(healthTracker.nodeIdToExcludedExpiryTime("hostA") === + 2000 + healthTracker.EXCLUDE_ON_FAILURE_TIMEOUT_MILLIS) + assert(healthTracker.nextExpiryTime === 1000 + healthTracker.EXCLUDE_ON_FAILURE_TIMEOUT_MILLIS) + } +} diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala index f29eb70eb3628..0c60c42c054cf 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -51,11 +51,11 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B var failedTaskSetReason: String = null var failedTaskSet = false - var blacklist: BlacklistTracker = null + var healthTracker: HealthTracker = null var taskScheduler: TaskSchedulerImpl = null var dagScheduler: DAGScheduler = null - val stageToMockTaskSetBlacklist = new HashMap[Int, TaskSetBlacklist]() + val stageToMockTaskSetExcludelist = new HashMap[Int, TaskSetExcludelist]() val stageToMockTaskSetManager = new HashMap[Int, TaskSetManager]() override def beforeEach(): Unit = { @@ -63,7 +63,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B failedTaskSet = false failedTaskSetException = None failedTaskSetReason = null - stageToMockTaskSetBlacklist.clear() + stageToMockTaskSetExcludelist.clear() stageToMockTaskSetManager.clear() } @@ -95,10 +95,10 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B setupHelper() } - def setupSchedulerWithMockTaskSetBlacklist(confs: (String, String)*): TaskSchedulerImpl = { - blacklist = mock[BlacklistTracker] + def setupSchedulerWithMockTaskSetExcludelist(confs: (String, String)*): TaskSchedulerImpl = { + healthTracker = mock[HealthTracker] val conf = new SparkConf().setMaster("local").setAppName("TaskSchedulerImplSuite") - conf.set(config.BLACKLIST_ENABLED, true) + conf.set(config.EXCLUDE_ON_FAILURE_ENABLED, true) confs.foreach { case (k, v) => conf.set(k, v) } sc = new SparkContext(conf) @@ -106,16 +106,16 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B new TaskSchedulerImpl(sc, sc.conf.get(config.TASK_MAX_FAILURES)) { override def createTaskSetManager(taskSet: TaskSet, maxFailures: Int): TaskSetManager = { val tsm = super.createTaskSetManager(taskSet, maxFailures) - // we need to create a spied tsm just so we can set the TaskSetBlacklist + // we need to create a spied tsm just so we can set the TaskSetExcludelist val tsmSpy = spy(tsm) - val taskSetBlacklist = mock[TaskSetBlacklist] - when(tsmSpy.taskSetBlacklistHelperOpt).thenReturn(Some(taskSetBlacklist)) + val taskSetExcludelist = mock[TaskSetExcludelist] + when(tsmSpy.taskSetExcludelistHelperOpt).thenReturn(Some(taskSetExcludelist)) stageToMockTaskSetManager(taskSet.stageId) = tsmSpy - stageToMockTaskSetBlacklist(taskSet.stageId) = taskSetBlacklist + stageToMockTaskSetExcludelist(taskSet.stageId) = taskSetExcludelist tsmSpy } - override private[scheduler] lazy val blacklistTrackerOpt = Some(blacklist) + override private[scheduler] lazy val healthTrackerOpt = Some(healthTracker) } setupHelper() } @@ -230,7 +230,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B sc.conf.get(config.TASK_MAX_FAILURES), clock = clock) { override def createTaskSetManager(taskSet: TaskSet, maxTaskFailures: Int): TaskSetManager = { - new TaskSetManager(this, taskSet, maxTaskFailures, blacklistTrackerOpt, clock) + new TaskSetManager(this, taskSet, maxTaskFailures, healthTrackerOpt, clock) } override def shuffleOffers(offers: IndexedSeq[WorkerOffer]): IndexedSeq[WorkerOffer] = { // Don't shuffle the offers around for this test. Instead, we'll just pass in all @@ -678,22 +678,22 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B assert(!failedTaskSet) } - test("scheduled tasks obey task and stage blacklists") { - taskScheduler = setupSchedulerWithMockTaskSetBlacklist() + test("scheduled tasks obey task and stage excludelist") { + taskScheduler = setupSchedulerWithMockTaskSetExcludelist() (0 to 2).foreach {stageId => val taskSet = FakeTask.createTaskSet(numTasks = 2, stageId = stageId, stageAttemptId = 0) taskScheduler.submitTasks(taskSet) } - // Setup our mock blacklist: - // * stage 0 is blacklisted on node "host1" - // * stage 1 is blacklisted on executor "executor3" - // * stage 0, partition 0 is blacklisted on executor 0 - // (mocked methods default to returning false, ie. no blacklisting) - when(stageToMockTaskSetBlacklist(0).isNodeBlacklistedForTaskSet("host1")).thenReturn(true) - when(stageToMockTaskSetBlacklist(1).isExecutorBlacklistedForTaskSet("executor3")) + // Setup our mock excludelist: + // * stage 0 is excluded on node "host1" + // * stage 1 is excluded on executor "executor3" + // * stage 0, partition 0 is excluded on executor 0 + // (mocked methods default to returning false, ie. no excluding) + when(stageToMockTaskSetExcludelist(0).isNodeExcludedForTaskSet("host1")).thenReturn(true) + when(stageToMockTaskSetExcludelist(1).isExecutorExcludedForTaskSet("executor3")) .thenReturn(true) - when(stageToMockTaskSetBlacklist(0).isExecutorBlacklistedForTask("executor0", 0)) + when(stageToMockTaskSetExcludelist(0).isExecutorExcludedForTask("executor0", 0)) .thenReturn(true) val offers = IndexedSeq( @@ -705,21 +705,21 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B val firstTaskAttempts = taskScheduler.resourceOffers(offers).flatten // We should schedule all tasks. assert(firstTaskAttempts.size === 6) - // Whenever we schedule a task, we must consult the node and executor blacklist. (The test + // Whenever we schedule a task, we must consult the node and executor excludelist. (The test // doesn't check exactly what checks are made because the offers get shuffled.) (0 to 2).foreach { stageId => - verify(stageToMockTaskSetBlacklist(stageId), atLeast(1)) - .isNodeBlacklistedForTaskSet(anyString()) - verify(stageToMockTaskSetBlacklist(stageId), atLeast(1)) - .isExecutorBlacklistedForTaskSet(anyString()) + verify(stageToMockTaskSetExcludelist(stageId), atLeast(1)) + .isNodeExcludedForTaskSet(anyString()) + verify(stageToMockTaskSetExcludelist(stageId), atLeast(1)) + .isExecutorExcludedForTaskSet(anyString()) } def tasksForStage(stageId: Int): Seq[TaskDescription] = { firstTaskAttempts.filter{_.name.contains(s"stage $stageId")} } tasksForStage(0).foreach { task => - // executors 1 & 2 blacklisted for node - // executor 0 blacklisted just for partition 0 + // executors 1 & 2 excluded for node + // executor 0 excluded just for partition 0 if (task.index == 0) { assert(task.executorId === "executor3") } else { @@ -727,12 +727,12 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B } } tasksForStage(1).foreach { task => - // executor 3 blacklisted + // executor 3 excluded assert("executor3" != task.executorId) } // no restrictions on stage 2 - // Finally, just make sure that we can still complete tasks as usual with blacklisting + // Finally, just make sure that we can still complete tasks as usual with exclusion // in effect. Finish each of the tasksets -- taskset 0 & 1 complete successfully, taskset 2 // fails. (0 to 2).foreach { stageId => @@ -770,23 +770,23 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B } // the tasksSets complete, so the tracker should be notified of the successful ones - verify(blacklist, times(1)).updateBlacklistForSuccessfulTaskSet( + verify(healthTracker, times(1)).updateExcludedForSuccessfulTaskSet( stageId = 0, stageAttemptId = 0, - failuresByExec = stageToMockTaskSetBlacklist(0).execToFailures) - verify(blacklist, times(1)).updateBlacklistForSuccessfulTaskSet( + failuresByExec = stageToMockTaskSetExcludelist(0).execToFailures) + verify(healthTracker, times(1)).updateExcludedForSuccessfulTaskSet( stageId = 1, stageAttemptId = 0, - failuresByExec = stageToMockTaskSetBlacklist(1).execToFailures) + failuresByExec = stageToMockTaskSetExcludelist(1).execToFailures) // but we shouldn't update for the failed taskset - verify(blacklist, never).updateBlacklistForSuccessfulTaskSet( + verify(healthTracker, never).updateExcludedForSuccessfulTaskSet( stageId = meq(2), stageAttemptId = anyInt(), failuresByExec = any()) } - test("scheduled tasks obey node and executor blacklists") { - taskScheduler = setupSchedulerWithMockTaskSetBlacklist() + test("scheduled tasks obey node and executor excludelists") { + taskScheduler = setupSchedulerWithMockTaskSetExcludelist() (0 to 2).foreach { stageId => val taskSet = FakeTask.createTaskSet(numTasks = 2, stageId = stageId, stageAttemptId = 0) taskScheduler.submitTasks(taskSet) @@ -800,13 +800,13 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B new WorkerOffer("executor4", "host3", 1) ) - // setup our mock blacklist: - // host1, executor0 & executor3 are completely blacklisted + // setup our mock excludelist: + // host1, executor0 & executor3 are completely excluded // This covers everything *except* one core on executor4 / host3, so that everything is still // schedulable. - when(blacklist.isNodeBlacklisted("host1")).thenReturn(true) - when(blacklist.isExecutorBlacklisted("executor0")).thenReturn(true) - when(blacklist.isExecutorBlacklisted("executor3")).thenReturn(true) + when(healthTracker.isNodeExcluded("host1")).thenReturn(true) + when(healthTracker.isExecutorExcluded("executor0")).thenReturn(true) + when(healthTracker.isExecutorExcluded("executor3")).thenReturn(true) val stageToTsm = (0 to 2).map { stageId => val tsm = taskScheduler.taskSetManagerForAttempt(stageId, 0).get @@ -818,12 +818,12 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B assert(firstTaskAttempts.size === 1) assert(firstTaskAttempts.head.executorId === "executor4") ('0' until '2').foreach { hostNum => - verify(blacklist, atLeast(1)).isNodeBlacklisted("host" + hostNum) + verify(healthTracker, atLeast(1)).isNodeExcluded("host" + hostNum) } } - test("abort stage when all executors are blacklisted and we cannot acquire new executor") { - taskScheduler = setupSchedulerWithMockTaskSetBlacklist() + test("abort stage when all executors are excluded and we cannot acquire new executor") { + taskScheduler = setupSchedulerWithMockTaskSetExcludelist() val taskSet = FakeTask.createTaskSet(numTasks = 10) taskScheduler.submitTasks(taskSet) val tsm = stageToMockTaskSetManager(0) @@ -836,11 +836,11 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B WorkerOffer("executor3", "host1", 2) )) - // now say our blacklist updates to blacklist a bunch of resources, but *not* everything - when(blacklist.isNodeBlacklisted("host1")).thenReturn(true) - when(blacklist.isExecutorBlacklisted("executor0")).thenReturn(true) + // now say our health tracker updates to exclude a bunch of resources, but *not* everything + when(healthTracker.isNodeExcluded("host1")).thenReturn(true) + when(healthTracker.isExecutorExcluded("executor0")).thenReturn(true) - // make an offer on the blacklisted resources. We won't schedule anything, but also won't + // make an offer on the excluded resources. We won't schedule anything, but also won't // abort yet, since we know of other resources that work assert(taskScheduler.resourceOffers(IndexedSeq( WorkerOffer("executor0", "host0", 2), @@ -848,9 +848,9 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B )).flatten.size === 0) assert(!tsm.isZombie) - // now update the blacklist so that everything really is blacklisted - when(blacklist.isExecutorBlacklisted("executor1")).thenReturn(true) - when(blacklist.isExecutorBlacklisted("executor2")).thenReturn(true) + // now update the health tracker so that everything really is excluded + when(healthTracker.isExecutorExcluded("executor1")).thenReturn(true) + when(healthTracker.isExecutorExcluded("executor2")).thenReturn(true) assert(taskScheduler.resourceOffers(IndexedSeq( WorkerOffer("executor0", "host0", 2), WorkerOffer("executor3", "host1", 2) @@ -859,10 +859,10 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B verify(tsm).abort(anyString(), any()) } - test("SPARK-22148 abort timer should kick in when task is completely blacklisted & no new " + + test("SPARK-22148 abort timer should kick in when task is completely excluded & no new " + "executor can be acquired") { // set the abort timer to fail immediately - taskScheduler = setupSchedulerWithMockTaskSetBlacklist( + taskScheduler = setupSchedulerWithMockTaskSetExcludelist( config.UNSCHEDULABLE_TASKSET_TIMEOUT.key -> "0") // We have only 1 task remaining with 1 executor @@ -878,10 +878,10 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B // Fail the running task val failedTask = firstTaskAttempts.find(_.executorId == "executor0").get failTask(failedTask.taskId, TaskState.FAILED, UnknownReason, tsm) - when(tsm.taskSetBlacklistHelperOpt.get.isExecutorBlacklistedForTask( + when(tsm.taskSetExcludelistHelperOpt.get.isExecutorExcludedForTask( "executor0", failedTask.index)).thenReturn(true) - // make an offer on the blacklisted executor. We won't schedule anything, and set the abort + // make an offer on the excluded executor. We won't schedule anything, and set the abort // timer to kick in immediately assert(taskScheduler.resourceOffers(IndexedSeq( WorkerOffer("executor0", "host0", 1) @@ -894,7 +894,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B } test("SPARK-22148 try to acquire a new executor when task is unschedulable with 1 executor") { - taskScheduler = setupSchedulerWithMockTaskSetBlacklist( + taskScheduler = setupSchedulerWithMockTaskSetExcludelist( config.UNSCHEDULABLE_TASKSET_TIMEOUT.key -> "10") // We have only 1 task remaining with 1 executor @@ -910,11 +910,11 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B // Fail the running task val failedTask = firstTaskAttempts.head failTask(failedTask.taskId, TaskState.FAILED, UnknownReason, tsm) - when(tsm.taskSetBlacklistHelperOpt.get.isExecutorBlacklistedForTask( + when(tsm.taskSetExcludelistHelperOpt.get.isExecutorExcludedForTask( "executor0", failedTask.index)).thenReturn(true) - // make an offer on the blacklisted executor. We won't schedule anything, and set the abort - // timer to expire if no new executors could be acquired. We kill the existing idle blacklisted + // make an offer on the excluded executor. We won't schedule anything, and set the abort + // timer to expire if no new executors could be acquired. We kill the existing idle excluded // executor and try to acquire a new one. assert(taskScheduler.resourceOffers(IndexedSeq( WorkerOffer("executor0", "host0", 1) @@ -930,12 +930,12 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B assert(!tsm.isZombie) } - // This is to test a scenario where we have two taskSets completely blacklisted and on acquiring + // This is to test a scenario where we have two taskSets completely excluded and on acquiring // a new executor we don't want the abort timer for the second taskSet to expire and abort the job test("SPARK-22148 abort timer should clear unschedulableTaskSetToExpiryTime for all TaskSets") { - taskScheduler = setupSchedulerWithMockTaskSetBlacklist() + taskScheduler = setupSchedulerWithMockTaskSetExcludelist() - // We have 2 taskSets with 1 task remaining in each with 1 executor completely blacklisted + // We have 2 taskSets with 1 task remaining in each with 1 executor completely excluded val taskSet1 = FakeTask.createTaskSet(numTasks = 1, stageId = 0, stageAttemptId = 0) taskScheduler.submitTasks(taskSet1) val taskSet2 = FakeTask.createTaskSet(numTasks = 1, stageId = 1, stageAttemptId = 0) @@ -952,7 +952,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B // Fail the running task val failedTask = firstTaskAttempts.head failTask(failedTask.taskId, TaskState.FAILED, UnknownReason, tsm) - when(tsm.taskSetBlacklistHelperOpt.get.isExecutorBlacklistedForTask( + when(tsm.taskSetExcludelistHelperOpt.get.isExecutorExcludedForTask( "executor0", failedTask.index)).thenReturn(true) // make an offer. We will schedule the task from the second taskSet. Since a task was scheduled @@ -966,10 +966,10 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B val tsm2 = stageToMockTaskSetManager(1) val failedTask2 = secondTaskAttempts.head failTask(failedTask2.taskId, TaskState.FAILED, UnknownReason, tsm2) - when(tsm2.taskSetBlacklistHelperOpt.get.isExecutorBlacklistedForTask( + when(tsm2.taskSetExcludelistHelperOpt.get.isExecutorExcludedForTask( "executor0", failedTask2.index)).thenReturn(true) - // make an offer on the blacklisted executor. We won't schedule anything, and set the abort + // make an offer on the excluded executor. We won't schedule anything, and set the abort // timer for taskSet1 and taskSet2 assert(taskScheduler.resourceOffers(IndexedSeq( WorkerOffer("executor0", "host0", 1) @@ -991,9 +991,9 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B // this test is to check that we don't abort a taskSet which is not being scheduled on other // executors as it is waiting on locality timeout and not being aborted because it is still not - // completely blacklisted. - test("SPARK-22148 Ensure we don't abort the taskSet if we haven't been completely blacklisted") { - taskScheduler = setupSchedulerWithMockTaskSetBlacklist( + // completely excluded. + test("SPARK-22148 Ensure we don't abort the taskSet if we haven't been completely excluded") { + taskScheduler = setupSchedulerWithMockTaskSetExcludelist( config.UNSCHEDULABLE_TASKSET_TIMEOUT.key -> "0", // This is to avoid any potential flakiness in the test because of large pauses in jenkins config.LOCALITY_WAIT.key -> "30s" @@ -1014,7 +1014,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B // Fail the running task val failedTask = taskAttempts.head failTask(failedTask.taskId, TaskState.FAILED, UnknownReason, tsm) - when(tsm.taskSetBlacklistHelperOpt.get.isExecutorBlacklistedForTask( + when(tsm.taskSetExcludelistHelperOpt.get.isExecutorExcludedForTask( "executor0", failedTask.index)).thenReturn(true) // make an offer but we won't schedule anything yet as scheduler locality is still PROCESS_LOCAL @@ -1027,10 +1027,10 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B assert(!tsm.isZombie) } - test("SPARK-31418 abort timer should kick in when task is completely blacklisted &" + + test("SPARK-31418 abort timer should kick in when task is completely excluded &" + "allocation manager could not acquire a new executor before the timeout") { // set the abort timer to fail immediately - taskScheduler = setupSchedulerWithMockTaskSetBlacklist( + taskScheduler = setupSchedulerWithMockTaskSetExcludelist( config.UNSCHEDULABLE_TASKSET_TIMEOUT.key -> "0", config.DYN_ALLOCATION_ENABLED.key -> "true") @@ -1044,14 +1044,14 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B // Fail the running task failTask(0, TaskState.FAILED, UnknownReason, tsm) - when(tsm.taskSetBlacklistHelperOpt.get.isExecutorBlacklistedForTask( + when(tsm.taskSetExcludelistHelperOpt.get.isExecutorExcludedForTask( "executor0", 0)).thenReturn(true) // If the executor is busy, then dynamic allocation should kick in and try - // to acquire additional executors to schedule the blacklisted task + // to acquire additional executors to schedule the excluded task assert(taskScheduler.isExecutorBusy("executor0")) - // make an offer on the blacklisted executor. We won't schedule anything, and set the abort + // make an offer on the excluded executor. We won't schedule anything, and set the abort // timer to kick in immediately assert(taskScheduler.resourceOffers(IndexedSeq( WorkerOffer("executor0", "host0", 1) @@ -1064,31 +1064,31 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B } /** - * Helper for performance tests. Takes the explicitly blacklisted nodes and executors; verifies - * that the blacklists are used efficiently to ensure scheduling is not O(numPendingTasks). + * Helper for performance tests. Takes the explicitly excluded nodes and executors; verifies + * that the excluded are used efficiently to ensure scheduling is not O(numPendingTasks). * Creates 1 offer on executor[1-3]. Executor1 & 2 are on host1, executor3 is on host2. Passed * in nodes and executors should be on that list. */ - private def testBlacklistPerformance( + private def testExcludelistPerformance( testName: String, - nodeBlacklist: Seq[String], - execBlacklist: Seq[String]): Unit = { + nodeExcludelist: Seq[String], + execExcludelist: Seq[String]): Unit = { // Because scheduling involves shuffling the order of offers around, we run this test a few // times to cover more possibilities. There are only 3 offers, which means 6 permutations, // so 10 iterations is pretty good. (0 until 10).foreach { testItr => test(s"$testName: iteration $testItr") { - // When an executor or node is blacklisted, we want to make sure that we don't try - // scheduling each pending task, one by one, to discover they are all blacklisted. This is + // When an executor or node is excluded, we want to make sure that we don't try + // scheduling each pending task, one by one, to discover they are all excluded. This is // important for performance -- if we did check each task one-by-one, then responding to a // resource offer (which is usually O(1)-ish) would become O(numPendingTasks), which would // slow down scheduler throughput and slow down scheduling even on healthy executors. // Here, we check a proxy for the runtime -- we make sure the scheduling is short-circuited - // at the node or executor blacklist, so we never check the per-task blacklist. We also - // make sure we don't check the node & executor blacklist for the entire taskset + // at the node or executor excludelist, so we never check the per-task excludelist. We also + // make sure we don't check the node & executor excludelist for the entire taskset // O(numPendingTasks) times. - taskScheduler = setupSchedulerWithMockTaskSetBlacklist() + taskScheduler = setupSchedulerWithMockTaskSetExcludelist() // we schedule 500 tasks so we can clearly distinguish anything that is O(numPendingTasks) val taskSet = FakeTask.createTaskSet(numTasks = 500, stageId = 0, stageAttemptId = 0) taskScheduler.submitTasks(taskSet) @@ -1098,91 +1098,92 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B new WorkerOffer("executor2", "host1", 1), new WorkerOffer("executor3", "host2", 1) ) - // We should check the node & exec blacklists, but only O(numOffers), not O(numPendingTasks) - // times. In the worst case, after shuffling, we offer our blacklisted resource first, and - // then offer other resources which do get used. The taskset blacklist is consulted - // repeatedly as we offer resources to the taskset -- each iteration either schedules - // something, or it terminates that locality level, so the maximum number of checks is - // numCores + numLocalityLevels + // We should check the node & exec excludelists, but only O(numOffers), + // not O(numPendingTasks) times. In the worst case, after shuffling, + // we offer our excluded resource first, and then offer other resources + // which do get used. The taskset excludelist is consulted repeatedly + // as we offer resources to the taskset -- each iteration either schedules + // something, or it terminates that locality level, so the maximum number of + // checks is numCores + numLocalityLevels val numCoresOnAllOffers = offers.map(_.cores).sum val numLocalityLevels = TaskLocality.values.size - val maxBlacklistChecks = numCoresOnAllOffers + numLocalityLevels + val maxExcludelistChecks = numCoresOnAllOffers + numLocalityLevels - // Setup the blacklist - nodeBlacklist.foreach { node => - when(stageToMockTaskSetBlacklist(0).isNodeBlacklistedForTaskSet(node)).thenReturn(true) + // Setup the excludelist + nodeExcludelist.foreach { node => + when(stageToMockTaskSetExcludelist(0).isNodeExcludedForTaskSet(node)).thenReturn(true) } - execBlacklist.foreach { exec => - when(stageToMockTaskSetBlacklist(0).isExecutorBlacklistedForTaskSet(exec)) + execExcludelist.foreach { exec => + when(stageToMockTaskSetExcludelist(0).isExecutorExcludedForTaskSet(exec)) .thenReturn(true) } - // Figure out which nodes have any effective blacklisting on them. This means all nodes - // that are explicitly blacklisted, plus those that have *any* executors blacklisted. - val nodesForBlacklistedExecutors = offers.filter { offer => - execBlacklist.contains(offer.executorId) + // Figure out which nodes have any effective exclusions on them. This means all nodes + // that are explicitly excluded, plus those that have *any* executors excluded. + val nodesForExcludedExecutors = offers.filter { offer => + execExcludelist.contains(offer.executorId) }.map(_.host).distinct - val nodesWithAnyBlacklisting = (nodeBlacklist ++ nodesForBlacklistedExecutors).toSet - // Similarly, figure out which executors have any blacklisting. This means all executors - // that are explicitly blacklisted, plus all executors on nodes that are blacklisted. - val execsForBlacklistedNodes = offers.filter { offer => - nodeBlacklist.contains(offer.host) + val nodesWithAnyExclusions = (nodeExcludelist ++ nodesForExcludedExecutors).toSet + // Similarly, figure out which executors have any exclusions. This means all executors + // that are explicitly excluded, plus all executors on nodes that are excluded. + val execsForExcludedNodes = offers.filter { offer => + nodeExcludelist.contains(offer.host) }.map(_.executorId).toSeq - val executorsWithAnyBlacklisting = (execBlacklist ++ execsForBlacklistedNodes).toSet + val executorsWithAnyExclusions = (execExcludelist ++ execsForExcludedNodes).toSet // Schedule a taskset, and make sure our test setup is correct -- we are able to schedule - // a task on all executors that aren't blacklisted (whether that executor is a explicitly - // blacklisted, or implicitly blacklisted via the node blacklist). + // a task on all executors that aren't excluded (whether that executor is a explicitly + // excluded, or implicitly excluded via the node excludeOnFailures). val firstTaskAttempts = taskScheduler.resourceOffers(offers).flatten - assert(firstTaskAttempts.size === offers.size - executorsWithAnyBlacklisting.size) + assert(firstTaskAttempts.size === offers.size - executorsWithAnyExclusions.size) - // Now check that we haven't made too many calls to any of the blacklist methods. - // We should be checking our node blacklist, but it should be within the bound we defined + // Now check that we haven't made too many calls to any of the excludelist methods. + // We should be checking our node excludelist, but it should be within the bound we defined // above. - verify(stageToMockTaskSetBlacklist(0), atMost(maxBlacklistChecks)) - .isNodeBlacklistedForTaskSet(anyString()) - // We shouldn't ever consult the per-task blacklist for the nodes that have been blacklisted - // for the entire taskset, since the taskset level blacklisting should prevent scheduling + verify(stageToMockTaskSetExcludelist(0), atMost(maxExcludelistChecks)) + .isNodeExcludedForTaskSet(anyString()) + // We shouldn't ever consult the per-task excludelist for the nodes that have been excluded + // for the entire taskset, since the taskset level exclusions should prevent scheduling // from ever looking at specific tasks. - nodesWithAnyBlacklisting.foreach { node => - verify(stageToMockTaskSetBlacklist(0), never) - .isNodeBlacklistedForTask(meq(node), anyInt()) + nodesWithAnyExclusions.foreach { node => + verify(stageToMockTaskSetExcludelist(0), never) + .isNodeExcludedForTask(meq(node), anyInt()) } - executorsWithAnyBlacklisting.foreach { exec => - // We should be checking our executor blacklist, but it should be within the bound defined - // above. Its possible that this will be significantly fewer calls, maybe even 0, if - // there is also a node-blacklist which takes effect first. But this assert is all we - // need to avoid an O(numPendingTask) slowdown. - verify(stageToMockTaskSetBlacklist(0), atMost(maxBlacklistChecks)) - .isExecutorBlacklistedForTaskSet(exec) - // We shouldn't ever consult the per-task blacklist for executors that have been - // blacklisted for the entire taskset, since the taskset level blacklisting should prevent + executorsWithAnyExclusions.foreach { exec => + // We should be checking our executor excludelist, but it should be within the bound + // defined above. Its possible that this will be significantly fewer calls, maybe even + // 0, if there is also a node-excludelist which takes effect first. But this assert is + // all we need to avoid an O(numPendingTask) slowdown. + verify(stageToMockTaskSetExcludelist(0), atMost(maxExcludelistChecks)) + .isExecutorExcludedForTaskSet(exec) + // We shouldn't ever consult the per-task excludelist for executors that have been + // excluded for the entire taskset, since the taskset level exclusions should prevent // scheduling from ever looking at specific tasks. - verify(stageToMockTaskSetBlacklist(0), never) - .isExecutorBlacklistedForTask(meq(exec), anyInt()) + verify(stageToMockTaskSetExcludelist(0), never) + .isExecutorExcludedForTask(meq(exec), anyInt()) } } } } - testBlacklistPerformance( - testName = "Blacklisted node for entire task set prevents per-task blacklist checks", - nodeBlacklist = Seq("host1"), - execBlacklist = Seq()) + testExcludelistPerformance( + testName = "Excluded node for entire task set prevents per-task exclusion checks", + nodeExcludelist = Seq("host1"), + execExcludelist = Seq()) - testBlacklistPerformance( - testName = "Blacklisted executor for entire task set prevents per-task blacklist checks", - nodeBlacklist = Seq(), - execBlacklist = Seq("executor3") + testExcludelistPerformance( + testName = "Excluded executor for entire task set prevents per-task exclusion checks", + nodeExcludelist = Seq(), + execExcludelist = Seq("executor3") ) test("abort stage if executor loss results in unschedulability from previously failed tasks") { - // Make sure we can detect when a taskset becomes unschedulable from a blacklisting. This + // Make sure we can detect when a taskset becomes unschedulable from excludeOnFailure. This // test explores a particular corner case -- you may have one task fail, but still be // schedulable on another executor. However, that executor may fail later on, leaving the // first task with no place to run. val taskScheduler = setupScheduler( - config.BLACKLIST_ENABLED.key -> "true" + config.EXCLUDE_ON_FAILURE_ENABLED.key -> "true" ) val taskSet = FakeTask.createTaskSet(2) @@ -1215,7 +1216,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B assert(nextTaskAttempts.head.index != failedTask.index) // Now we should definitely realize that our task set is unschedulable, because the only - // task left can't be scheduled on any executors due to the blacklist. + // task left can't be scheduled on any executors due to the excludelist. taskScheduler.resourceOffers(IndexedSeq(new WorkerOffer("executor0", "host0", 1))) sc.listenerBus.waitUntilEmpty(100000) assert(tsm.isZombie) @@ -1223,11 +1224,11 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B val idx = failedTask.index assert(failedTaskSetReason === s""" |Aborting $taskSet because task $idx (partition $idx) - |cannot run anywhere due to node and executor blacklist. + |cannot run anywhere due to node and executor excludeOnFailure. |Most recent failure: - |${tsm.taskSetBlacklistHelperOpt.get.getLatestFailureReason} + |${tsm.taskSetExcludelistHelperOpt.get.getLatestFailureReason} | - |Blacklisting behavior can be configured via spark.blacklist.*. + |ExcludeOnFailure behavior can be configured via spark.excludeOnFailure.*. |""".stripMargin) } @@ -1238,7 +1239,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B // available and not bail on the job val taskScheduler = setupScheduler( - config.BLACKLIST_ENABLED.key -> "true" + config.EXCLUDE_ON_FAILURE_ENABLED.key -> "true" ) val taskSet = FakeTask.createTaskSet(2, (0 until 2).map { _ => Seq(TaskLocation("host0")) }: _*) @@ -1306,7 +1307,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B assert(taskScheduler.getExecutorsAliveOnHost("host1") === Some(Set("executor1", "executor3"))) } - test("scheduler checks for executors that can be expired from blacklist") { + test("scheduler checks for executors that can be expired from excludeOnFailure") { taskScheduler = setupScheduler() taskScheduler.submitTasks(FakeTask.createTaskSet(1, stageId = 0, stageAttemptId = 0)) @@ -1314,7 +1315,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B new WorkerOffer("executor0", "host0", 1) )).flatten - verify(blacklist).applyBlacklistTimeout() + verify(healthTracker).applyExcludeOnFailureTimeout() } test("if an executor is lost then the state for its running tasks is cleaned up (SPARK-18553)") { @@ -1400,7 +1401,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B offers } override def createTaskSetManager(taskSet: TaskSet, maxTaskFailures: Int): TaskSetManager = { - new TaskSetManager(this, taskSet, maxTaskFailures, blacklistTrackerOpt, clock) + new TaskSetManager(this, taskSet, maxTaskFailures, healthTrackerOpt, clock) } } // Need to initialize a DAGScheduler for the taskScheduler to use for callbacks. @@ -1440,7 +1441,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B val clock = new ManualClock() val taskScheduler = new TaskSchedulerImpl(sc) { override def createTaskSetManager(taskSet: TaskSet, maxTaskFailures: Int): TaskSetManager = { - new TaskSetManager(this, taskSet, maxTaskFailures, blacklistTrackerOpt, clock) + new TaskSetManager(this, taskSet, maxTaskFailures, healthTrackerOpt, clock) } } // Need to initialize a DAGScheduler for the taskScheduler to use for callbacks. diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetBlacklistSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetBlacklistSuite.scala deleted file mode 100644 index ed97a4c206ca3..0000000000000 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetBlacklistSuite.scala +++ /dev/null @@ -1,287 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.scheduler - -import org.mockito.ArgumentMatchers.isA -import org.mockito.Mockito.{never, verify} -import org.scalatest.BeforeAndAfterEach -import org.scalatestplus.mockito.MockitoSugar - -import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.internal.config -import org.apache.spark.util.ManualClock - -class TaskSetBlacklistSuite extends SparkFunSuite with BeforeAndAfterEach with MockitoSugar { - - private var listenerBusMock: LiveListenerBus = _ - - override def beforeEach(): Unit = { - listenerBusMock = mock[LiveListenerBus] - super.beforeEach() - } - - test("Blacklisting tasks, executors, and nodes") { - val conf = new SparkConf().setAppName("test").setMaster("local") - .set(config.BLACKLIST_ENABLED.key, "true") - val clock = new ManualClock - val attemptId = 0 - val taskSetBlacklist = new TaskSetBlacklist( - listenerBusMock, conf, stageId = 0, stageAttemptId = attemptId, clock = clock) - - clock.setTime(0) - // We will mark task 0 & 1 failed on both executor 1 & 2. - // We should blacklist all executors on that host, for all tasks for the stage. Note the API - // will return false for isExecutorBacklistedForTaskSet even when the node is blacklisted, so - // the executor is implicitly blacklisted (this makes sense with how the scheduler uses the - // blacklist) - - // First, mark task 0 as failed on exec1. - // task 0 should be blacklisted on exec1, and nowhere else - taskSetBlacklist.updateBlacklistForFailedTask( - "hostA", exec = "exec1", index = 0, failureReason = "testing") - for { - executor <- (1 to 4).map(_.toString) - index <- 0 until 10 - } { - val shouldBeBlacklisted = (executor == "exec1" && index == 0) - assert(taskSetBlacklist.isExecutorBlacklistedForTask(executor, index) === shouldBeBlacklisted) - } - - assert(!taskSetBlacklist.isExecutorBlacklistedForTaskSet("exec1")) - verify(listenerBusMock, never()) - .post(isA(classOf[SparkListenerExecutorBlacklistedForStage])) - - assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) - verify(listenerBusMock, never()) - .post(isA(classOf[SparkListenerNodeBlacklistedForStage])) - - // Mark task 1 failed on exec1 -- this pushes the executor into the blacklist - taskSetBlacklist.updateBlacklistForFailedTask( - "hostA", exec = "exec1", index = 1, failureReason = "testing") - - assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("exec1")) - verify(listenerBusMock).post( - SparkListenerExecutorBlacklistedForStage(0, "exec1", 2, 0, attemptId)) - - assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) - verify(listenerBusMock, never()) - .post(isA(classOf[SparkListenerNodeBlacklistedForStage])) - - // Mark one task as failed on exec2 -- not enough for any further blacklisting yet. - taskSetBlacklist.updateBlacklistForFailedTask( - "hostA", exec = "exec2", index = 0, failureReason = "testing") - assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("exec1")) - - assert(!taskSetBlacklist.isExecutorBlacklistedForTaskSet("exec2")) - - assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) - verify(listenerBusMock, never()) - .post(isA(classOf[SparkListenerNodeBlacklistedForStage])) - - // Mark another task as failed on exec2 -- now we blacklist exec2, which also leads to - // blacklisting the entire node. - taskSetBlacklist.updateBlacklistForFailedTask( - "hostA", exec = "exec2", index = 1, failureReason = "testing") - - assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("exec1")) - - assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("exec2")) - verify(listenerBusMock).post( - SparkListenerExecutorBlacklistedForStage(0, "exec2", 2, 0, attemptId)) - - assert(taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) - verify(listenerBusMock).post( - SparkListenerNodeBlacklistedForStage(0, "hostA", 2, 0, attemptId)) - - // Make sure the blacklist has the correct per-task && per-executor responses, over a wider - // range of inputs. - for { - executor <- (1 to 4).map(e => s"exec$e") - index <- 0 until 10 - } { - withClue(s"exec = $executor; index = $index") { - val badExec = (executor == "exec1" || executor == "exec2") - val badIndex = (index == 0 || index == 1) - assert( - // this ignores whether the executor is blacklisted entirely for the taskset -- that is - // intentional, it keeps it fast and is sufficient for usage in the scheduler. - taskSetBlacklist.isExecutorBlacklistedForTask(executor, index) === (badExec && badIndex)) - assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet(executor) === badExec) - if (badExec) { - verify(listenerBusMock).post( - SparkListenerExecutorBlacklistedForStage(0, executor, 2, 0, attemptId)) - } - } - } - assert(taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) - val execToFailures = taskSetBlacklist.execToFailures - assert(execToFailures.keySet === Set("exec1", "exec2")) - - Seq("exec1", "exec2").foreach { exec => - assert( - execToFailures(exec).taskToFailureCountAndFailureTime === Map( - 0 -> ((1, 0)), - 1 -> ((1, 0)) - ) - ) - } - } - - test("multiple attempts for the same task count once") { - // Make sure that for blacklisting tasks, the node counts task attempts, not executors. But for - // stage-level blacklisting, we count unique tasks. The reason for this difference is, with - // task-attempt blacklisting, we want to make it easy to configure so that you ensure a node - // is blacklisted before the taskset is completely aborted because of spark.task.maxFailures. - // But with stage-blacklisting, we want to make sure we're not just counting one bad task - // that has failed many times. - - val conf = new SparkConf().setMaster("local").setAppName("test") - .set(config.MAX_TASK_ATTEMPTS_PER_EXECUTOR, 2) - .set(config.MAX_TASK_ATTEMPTS_PER_NODE, 3) - .set(config.MAX_FAILURES_PER_EXEC_STAGE, 2) - .set(config.MAX_FAILED_EXEC_PER_NODE_STAGE, 3) - val clock = new ManualClock - - val attemptId = 0 - val taskSetBlacklist = new TaskSetBlacklist( - listenerBusMock, conf, stageId = 0, stageAttemptId = attemptId, clock = clock) - - var time = 0 - clock.setTime(time) - // Fail a task twice on hostA, exec:1 - taskSetBlacklist.updateBlacklistForFailedTask( - "hostA", exec = "1", index = 0, failureReason = "testing") - taskSetBlacklist.updateBlacklistForFailedTask( - "hostA", exec = "1", index = 0, failureReason = "testing") - assert(taskSetBlacklist.isExecutorBlacklistedForTask("1", 0)) - assert(!taskSetBlacklist.isNodeBlacklistedForTask("hostA", 0)) - - assert(!taskSetBlacklist.isExecutorBlacklistedForTaskSet("1")) - verify(listenerBusMock, never()).post( - SparkListenerExecutorBlacklistedForStage(time, "1", 2, 0, attemptId)) - - assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) - verify(listenerBusMock, never()).post( - SparkListenerNodeBlacklistedForStage(time, "hostA", 2, 0, attemptId)) - - // Fail the same task once more on hostA, exec:2 - time += 1 - clock.setTime(time) - taskSetBlacklist.updateBlacklistForFailedTask( - "hostA", exec = "2", index = 0, failureReason = "testing") - assert(taskSetBlacklist.isNodeBlacklistedForTask("hostA", 0)) - - assert(!taskSetBlacklist.isExecutorBlacklistedForTaskSet("2")) - verify(listenerBusMock, never()).post( - SparkListenerExecutorBlacklistedForStage(time, "2", 2, 0, attemptId)) - - assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) - verify(listenerBusMock, never()).post( - SparkListenerNodeBlacklistedForStage(time, "hostA", 2, 0, attemptId)) - - // Fail another task on hostA, exec:1. Now that executor has failures on two different tasks, - // so its blacklisted - time += 1 - clock.setTime(time) - taskSetBlacklist.updateBlacklistForFailedTask( - "hostA", exec = "1", index = 1, failureReason = "testing") - - assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("1")) - verify(listenerBusMock) - .post(SparkListenerExecutorBlacklistedForStage(time, "1", 2, 0, attemptId)) - - assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) - verify(listenerBusMock, never()) - .post(isA(classOf[SparkListenerNodeBlacklistedForStage])) - - // Fail a third task on hostA, exec:2, so that exec is blacklisted for the whole task set - time += 1 - clock.setTime(time) - taskSetBlacklist.updateBlacklistForFailedTask( - "hostA", exec = "2", index = 2, failureReason = "testing") - - assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("2")) - verify(listenerBusMock) - .post(SparkListenerExecutorBlacklistedForStage(time, "2", 2, 0, attemptId)) - - assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) - verify(listenerBusMock, never()) - .post(isA(classOf[SparkListenerNodeBlacklistedForStage])) - - // Fail a fourth & fifth task on hostA, exec:3. Now we've got three executors that are - // blacklisted for the taskset, so blacklist the whole node. - time += 1 - clock.setTime(time) - taskSetBlacklist.updateBlacklistForFailedTask( - "hostA", exec = "3", index = 3, failureReason = "testing") - taskSetBlacklist.updateBlacklistForFailedTask( - "hostA", exec = "3", index = 4, failureReason = "testing") - - assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("3")) - verify(listenerBusMock) - .post(SparkListenerExecutorBlacklistedForStage(time, "3", 2, 0, attemptId)) - - assert(taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) - verify(listenerBusMock).post( - SparkListenerNodeBlacklistedForStage(time, "hostA", 3, 0, attemptId)) - } - - test("only blacklist nodes for the task set when all the blacklisted executors are all on " + - "same host") { - // we blacklist executors on two different hosts within one taskSet -- make sure that doesn't - // lead to any node blacklisting - val conf = new SparkConf().setAppName("test").setMaster("local") - .set(config.BLACKLIST_ENABLED.key, "true") - val clock = new ManualClock - - val attemptId = 0 - val taskSetBlacklist = new TaskSetBlacklist( - listenerBusMock, conf, stageId = 0, stageAttemptId = attemptId, clock = clock) - var time = 0 - clock.setTime(time) - taskSetBlacklist.updateBlacklistForFailedTask( - "hostA", exec = "1", index = 0, failureReason = "testing") - taskSetBlacklist.updateBlacklistForFailedTask( - "hostA", exec = "1", index = 1, failureReason = "testing") - - assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("1")) - verify(listenerBusMock) - .post(SparkListenerExecutorBlacklistedForStage(time, "1", 2, 0, attemptId)) - - assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) - verify(listenerBusMock, never()).post( - SparkListenerNodeBlacklistedForStage(time, "hostA", 2, 0, attemptId)) - - time += 1 - clock.setTime(time) - taskSetBlacklist.updateBlacklistForFailedTask( - "hostB", exec = "2", index = 0, failureReason = "testing") - taskSetBlacklist.updateBlacklistForFailedTask( - "hostB", exec = "2", index = 1, failureReason = "testing") - assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("1")) - - assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("2")) - verify(listenerBusMock) - .post(SparkListenerExecutorBlacklistedForStage(time, "2", 2, 0, attemptId)) - - assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) - assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostB")) - verify(listenerBusMock, never()) - .post(isA(classOf[SparkListenerNodeBlacklistedForStage])) - } - -} diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetExcludelistSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetExcludelistSuite.scala new file mode 100644 index 0000000000000..d20768d7cd12b --- /dev/null +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetExcludelistSuite.scala @@ -0,0 +1,310 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.scheduler + +import org.mockito.ArgumentMatchers.isA +import org.mockito.Mockito.{never, verify} +import org.scalatest.BeforeAndAfterEach +import org.scalatestplus.mockito.MockitoSugar + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.internal.config +import org.apache.spark.util.ManualClock + +class TaskSetExcludelistSuite extends SparkFunSuite with BeforeAndAfterEach with MockitoSugar { + + private var listenerBusMock: LiveListenerBus = _ + + override def beforeEach(): Unit = { + listenerBusMock = mock[LiveListenerBus] + super.beforeEach() + } + + test("Excluding tasks, executors, and nodes") { + val conf = new SparkConf().setAppName("test").setMaster("local") + .set(config.EXCLUDE_ON_FAILURE_ENABLED.key, "true") + val clock = new ManualClock + val attemptId = 0 + val taskSetExcludelist = new TaskSetExcludelist( + listenerBusMock, conf, stageId = 0, stageAttemptId = attemptId, clock = clock) + + clock.setTime(0) + // We will mark task 0 & 1 failed on both executor 1 & 2. + // We should exclude all executors on that host, for all tasks for the stage. Note the API + // will return false for isExecutorBacklistedForTaskSet even when the node is excluded, so + // the executor is implicitly excluded (this makes sense with how the scheduler uses the + // exclude) + + // First, mark task 0 as failed on exec1. + // task 0 should be excluded on exec1, and nowhere else + taskSetExcludelist.updateExcludedForFailedTask( + "hostA", exec = "exec1", index = 0, failureReason = "testing") + for { + executor <- (1 to 4).map(_.toString) + index <- 0 until 10 + } { + val shouldBeExcluded = (executor == "exec1" && index == 0) + assert(taskSetExcludelist.isExecutorExcludedForTask(executor, index) === shouldBeExcluded) + } + + assert(!taskSetExcludelist.isExecutorExcludedForTaskSet("exec1")) + verify(listenerBusMock, never()) + .post(isA(classOf[SparkListenerExecutorExcludedForStage])) + verify(listenerBusMock, never()) + .post(isA(classOf[SparkListenerExecutorBlacklistedForStage])) + + assert(!taskSetExcludelist.isNodeExcludedForTaskSet("hostA")) + verify(listenerBusMock, never()) + .post(isA(classOf[SparkListenerNodeExcludedForStage])) + + // Mark task 1 failed on exec1 -- this pushes the executor into the exclude + taskSetExcludelist.updateExcludedForFailedTask( + "hostA", exec = "exec1", index = 1, failureReason = "testing") + + assert(taskSetExcludelist.isExecutorExcludedForTaskSet("exec1")) + verify(listenerBusMock).post( + SparkListenerExecutorExcludedForStage(0, "exec1", 2, 0, attemptId)) + verify(listenerBusMock).post( + SparkListenerExecutorBlacklistedForStage(0, "exec1", 2, 0, attemptId)) + + + assert(!taskSetExcludelist.isNodeExcludedForTaskSet("hostA")) + verify(listenerBusMock, never()) + .post(isA(classOf[SparkListenerNodeExcludedForStage])) + verify(listenerBusMock, never()) + .post(isA(classOf[SparkListenerNodeBlacklistedForStage])) + + // Mark one task as failed on exec2 -- not enough for any further excluding yet. + taskSetExcludelist.updateExcludedForFailedTask( + "hostA", exec = "exec2", index = 0, failureReason = "testing") + assert(taskSetExcludelist.isExecutorExcludedForTaskSet("exec1")) + + assert(!taskSetExcludelist.isExecutorExcludedForTaskSet("exec2")) + + assert(!taskSetExcludelist.isNodeExcludedForTaskSet("hostA")) + verify(listenerBusMock, never()) + .post(isA(classOf[SparkListenerNodeExcludedForStage])) + verify(listenerBusMock, never()) + .post(isA(classOf[SparkListenerNodeBlacklistedForStage])) + + // Mark another task as failed on exec2 -- now we exclude exec2, which also leads to + // excluding the entire node. + taskSetExcludelist.updateExcludedForFailedTask( + "hostA", exec = "exec2", index = 1, failureReason = "testing") + + assert(taskSetExcludelist.isExecutorExcludedForTaskSet("exec1")) + + assert(taskSetExcludelist.isExecutorExcludedForTaskSet("exec2")) + verify(listenerBusMock).post( + SparkListenerExecutorExcludedForStage(0, "exec2", 2, 0, attemptId)) + verify(listenerBusMock).post( + SparkListenerExecutorBlacklistedForStage(0, "exec2", 2, 0, attemptId)) + + assert(taskSetExcludelist.isNodeExcludedForTaskSet("hostA")) + verify(listenerBusMock).post( + SparkListenerNodeExcludedForStage(0, "hostA", 2, 0, attemptId)) + verify(listenerBusMock).post( + SparkListenerNodeBlacklistedForStage(0, "hostA", 2, 0, attemptId)) + + // Make sure the exclude has the correct per-task && per-executor responses, over a wider + // range of inputs. + for { + executor <- (1 to 4).map(e => s"exec$e") + index <- 0 until 10 + } { + withClue(s"exec = $executor; index = $index") { + val badExec = (executor == "exec1" || executor == "exec2") + val badIndex = (index == 0 || index == 1) + assert( + // this ignores whether the executor is excluded entirely for the taskset -- that is + // intentional, it keeps it fast and is sufficient for usage in the scheduler. + taskSetExcludelist.isExecutorExcludedForTask(executor, index) === (badExec && badIndex)) + assert(taskSetExcludelist.isExecutorExcludedForTaskSet(executor) === badExec) + if (badExec) { + verify(listenerBusMock).post( + SparkListenerExecutorExcludedForStage(0, executor, 2, 0, attemptId)) + verify(listenerBusMock).post( + SparkListenerExecutorBlacklistedForStage(0, executor, 2, 0, attemptId)) + } + } + } + assert(taskSetExcludelist.isNodeExcludedForTaskSet("hostA")) + val execToFailures = taskSetExcludelist.execToFailures + assert(execToFailures.keySet === Set("exec1", "exec2")) + + Seq("exec1", "exec2").foreach { exec => + assert( + execToFailures(exec).taskToFailureCountAndFailureTime === Map( + 0 -> ((1, 0)), + 1 -> ((1, 0)) + ) + ) + } + } + + test("multiple attempts for the same task count once") { + // Make sure that for excluding tasks, the node counts task attempts, not executors. But for + // stage-level excluding, we count unique tasks. The reason for this difference is, with + // task-attempt excluding, we want to make it easy to configure so that you ensure a node + // is excluded before the taskset is completely aborted because of spark.task.maxFailures. + // But with stage-excluding, we want to make sure we're not just counting one bad task + // that has failed many times. + + val conf = new SparkConf().setMaster("local").setAppName("test") + .set(config.MAX_TASK_ATTEMPTS_PER_EXECUTOR, 2) + .set(config.MAX_TASK_ATTEMPTS_PER_NODE, 3) + .set(config.MAX_FAILURES_PER_EXEC_STAGE, 2) + .set(config.MAX_FAILED_EXEC_PER_NODE_STAGE, 3) + val clock = new ManualClock + + val attemptId = 0 + val taskSetExcludlist = new TaskSetExcludelist( + listenerBusMock, conf, stageId = 0, stageAttemptId = attemptId, clock = clock) + + var time = 0 + clock.setTime(time) + // Fail a task twice on hostA, exec:1 + taskSetExcludlist.updateExcludedForFailedTask( + "hostA", exec = "1", index = 0, failureReason = "testing") + taskSetExcludlist.updateExcludedForFailedTask( + "hostA", exec = "1", index = 0, failureReason = "testing") + assert(taskSetExcludlist.isExecutorExcludedForTask("1", 0)) + assert(!taskSetExcludlist.isNodeExcludedForTask("hostA", 0)) + + assert(!taskSetExcludlist.isExecutorExcludedForTaskSet("1")) + verify(listenerBusMock, never()).post( + SparkListenerExecutorExcludedForStage(time, "1", 2, 0, attemptId)) + + assert(!taskSetExcludlist.isNodeExcludedForTaskSet("hostA")) + verify(listenerBusMock, never()).post( + SparkListenerNodeExcludedForStage(time, "hostA", 2, 0, attemptId)) + + // Fail the same task once more on hostA, exec:2 + time += 1 + clock.setTime(time) + taskSetExcludlist.updateExcludedForFailedTask( + "hostA", exec = "2", index = 0, failureReason = "testing") + assert(taskSetExcludlist.isNodeExcludedForTask("hostA", 0)) + + assert(!taskSetExcludlist.isExecutorExcludedForTaskSet("2")) + verify(listenerBusMock, never()).post( + SparkListenerExecutorExcludedForStage(time, "2", 2, 0, attemptId)) + + assert(!taskSetExcludlist.isNodeExcludedForTaskSet("hostA")) + verify(listenerBusMock, never()).post( + SparkListenerNodeExcludedForStage(time, "hostA", 2, 0, attemptId)) + + // Fail another task on hostA, exec:1. Now that executor has failures on two different tasks, + // so its excluded + time += 1 + clock.setTime(time) + taskSetExcludlist.updateExcludedForFailedTask( + "hostA", exec = "1", index = 1, failureReason = "testing") + + assert(taskSetExcludlist.isExecutorExcludedForTaskSet("1")) + verify(listenerBusMock) + .post(SparkListenerExecutorExcludedForStage(time, "1", 2, 0, attemptId)) + + assert(!taskSetExcludlist.isNodeExcludedForTaskSet("hostA")) + verify(listenerBusMock, never()) + .post(isA(classOf[SparkListenerNodeExcludedForStage])) + + // Fail a third task on hostA, exec:2, so that exec is excluded for the whole task set + time += 1 + clock.setTime(time) + taskSetExcludlist.updateExcludedForFailedTask( + "hostA", exec = "2", index = 2, failureReason = "testing") + + assert(taskSetExcludlist.isExecutorExcludedForTaskSet("2")) + verify(listenerBusMock) + .post(SparkListenerExecutorExcludedForStage(time, "2", 2, 0, attemptId)) + + assert(!taskSetExcludlist.isNodeExcludedForTaskSet("hostA")) + verify(listenerBusMock, never()) + .post(isA(classOf[SparkListenerNodeExcludedForStage])) + + // Fail a fourth & fifth task on hostA, exec:3. Now we've got three executors that are + // excluded for the taskset, so exclude the whole node. + time += 1 + clock.setTime(time) + taskSetExcludlist.updateExcludedForFailedTask( + "hostA", exec = "3", index = 3, failureReason = "testing") + taskSetExcludlist.updateExcludedForFailedTask( + "hostA", exec = "3", index = 4, failureReason = "testing") + + assert(taskSetExcludlist.isExecutorExcludedForTaskSet("3")) + verify(listenerBusMock) + .post(SparkListenerExecutorExcludedForStage(time, "3", 2, 0, attemptId)) + + assert(taskSetExcludlist.isNodeExcludedForTaskSet("hostA")) + verify(listenerBusMock).post( + SparkListenerNodeExcludedForStage(time, "hostA", 3, 0, attemptId)) + } + + test("only exclude nodes for the task set when all the excluded executors are all on " + + "same host") { + // we exclude executors on two different hosts within one taskSet -- make sure that doesn't + // lead to any node excluding + val conf = new SparkConf().setAppName("test").setMaster("local") + .set(config.EXCLUDE_ON_FAILURE_ENABLED.key, "true") + val clock = new ManualClock + + val attemptId = 0 + val taskSetExcludlist = new TaskSetExcludelist( + listenerBusMock, conf, stageId = 0, stageAttemptId = attemptId, clock = clock) + var time = 0 + clock.setTime(time) + taskSetExcludlist.updateExcludedForFailedTask( + "hostA", exec = "1", index = 0, failureReason = "testing") + taskSetExcludlist.updateExcludedForFailedTask( + "hostA", exec = "1", index = 1, failureReason = "testing") + + assert(taskSetExcludlist.isExecutorExcludedForTaskSet("1")) + verify(listenerBusMock) + .post(SparkListenerExecutorExcludedForStage(time, "1", 2, 0, attemptId)) + verify(listenerBusMock) + .post(SparkListenerExecutorBlacklistedForStage(time, "1", 2, 0, attemptId)) + + assert(!taskSetExcludlist.isNodeExcludedForTaskSet("hostA")) + verify(listenerBusMock, never()).post( + SparkListenerNodeExcludedForStage(time, "hostA", 2, 0, attemptId)) + verify(listenerBusMock, never()).post( + SparkListenerNodeBlacklistedForStage(time, "hostA", 2, 0, attemptId)) + + time += 1 + clock.setTime(time) + taskSetExcludlist.updateExcludedForFailedTask( + "hostB", exec = "2", index = 0, failureReason = "testing") + taskSetExcludlist.updateExcludedForFailedTask( + "hostB", exec = "2", index = 1, failureReason = "testing") + assert(taskSetExcludlist.isExecutorExcludedForTaskSet("1")) + + assert(taskSetExcludlist.isExecutorExcludedForTaskSet("2")) + verify(listenerBusMock) + .post(SparkListenerExecutorExcludedForStage(time, "2", 2, 0, attemptId)) + verify(listenerBusMock) + .post(SparkListenerExecutorBlacklistedForStage(time, "2", 2, 0, attemptId)) + + assert(!taskSetExcludlist.isNodeExcludedForTaskSet("hostA")) + assert(!taskSetExcludlist.isNodeExcludedForTaskSet("hostB")) + verify(listenerBusMock, never()) + .post(isA(classOf[SparkListenerNodeExcludedForStage])) + verify(listenerBusMock, never()) + .post(isA(classOf[SparkListenerNodeBlacklistedForStage])) + } + +} diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index c389fd2ffa8b1..e01e278f60205 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -382,14 +382,14 @@ class TaskSetManagerSuite assert(delayReject === false) manager.isZombie = false - // offers not accepted due to blacklisting are not delay schedule rejects + // offers not accepted due to excludelist are not delay schedule rejects val tsmSpy = spy(manager) - val blacklist = mock(classOf[TaskSetBlacklist]) - when(tsmSpy.taskSetBlacklistHelperOpt).thenReturn(Some(blacklist)) - when(blacklist.isNodeBlacklistedForTaskSet(any())).thenReturn(true) - val (blacklistTask, blackListReject) = tsmSpy.resourceOffer("exec2", "host2", ANY) - assert(blacklistTask.isEmpty) - assert(blackListReject === false) + val excludelist = mock(classOf[TaskSetExcludelist]) + when(tsmSpy.taskSetExcludelistHelperOpt).thenReturn(Some(excludelist)) + when(excludelist.isNodeExcludedForTaskSet(any())).thenReturn(true) + val (task, taskReject) = tsmSpy.resourceOffer("exec2", "host2", ANY) + assert(task.isEmpty) + assert(taskReject === false) // After another delay, we can go ahead and launch that task non-locally assert(manager.resourceOffer("exec2", "host2", ANY)._1.get.index === 3) @@ -479,11 +479,11 @@ class TaskSetManagerSuite } } - test("executors should be blacklisted after task failure, in spite of locality preferences") { + test("executors should be excluded after task failure, in spite of locality preferences") { val rescheduleDelay = 300L val conf = new SparkConf(). - set(config.BLACKLIST_ENABLED, true). - set(config.BLACKLIST_TIMEOUT_CONF, rescheduleDelay). + set(config.EXCLUDE_ON_FAILURE_ENABLED, true). + set(config.EXCLUDE_ON_FAILURE_TIMEOUT_CONF, rescheduleDelay). // don't wait to jump locality levels in this test set(config.LOCALITY_WAIT.key, "0") @@ -495,11 +495,11 @@ class TaskSetManagerSuite val taskSet = FakeTask.createTaskSet(1, Seq(TaskLocation("host1", "exec1"))) val clock = new ManualClock clock.advance(1) - // We don't directly use the application blacklist, but its presence triggers blacklisting + // We don't directly use the application excludelist, but its presence triggers exclusion // within the taskset. val mockListenerBus = mock(classOf[LiveListenerBus]) - val blacklistTrackerOpt = Some(new BlacklistTracker(mockListenerBus, conf, None, clock)) - val manager = new TaskSetManager(sched, taskSet, 4, blacklistTrackerOpt, clock) + val healthTrackerOpt = Some(new HealthTracker(mockListenerBus, conf, None, clock)) + val manager = new TaskSetManager(sched, taskSet, 4, healthTrackerOpt, clock) { val offerResult = manager.resourceOffer("exec1", "host1", PROCESS_LOCAL)._1 @@ -512,7 +512,7 @@ class TaskSetManagerSuite manager.handleFailedTask(offerResult.get.taskId, TaskState.FINISHED, TaskResultLost) assert(!sched.taskSetsFailed.contains(taskSet.id)) - // Ensure scheduling on exec1 fails after failure 1 due to blacklist + // Ensure scheduling on exec1 fails after failure 1 due to executor being excluded assert(manager.resourceOffer("exec1", "host1", PROCESS_LOCAL)._1.isEmpty) assert(manager.resourceOffer("exec1", "host1", NODE_LOCAL)._1.isEmpty) assert(manager.resourceOffer("exec1", "host1", RACK_LOCAL)._1.isEmpty) @@ -532,7 +532,7 @@ class TaskSetManagerSuite manager.handleFailedTask(offerResult.get.taskId, TaskState.FINISHED, TaskResultLost) assert(!sched.taskSetsFailed.contains(taskSet.id)) - // Ensure scheduling on exec1.1 fails after failure 2 due to blacklist + // Ensure scheduling on exec1.1 fails after failure 2 due to executor being excluded assert(manager.resourceOffer("exec1.1", "host1", NODE_LOCAL)._1.isEmpty) } @@ -548,12 +548,12 @@ class TaskSetManagerSuite manager.handleFailedTask(offerResult.get.taskId, TaskState.FINISHED, TaskResultLost) assert(!sched.taskSetsFailed.contains(taskSet.id)) - // Ensure scheduling on exec2 fails after failure 3 due to blacklist + // Ensure scheduling on exec2 fails after failure 3 due to executor being excluded assert(manager.resourceOffer("exec2", "host2", ANY)._1.isEmpty) } - // Despite advancing beyond the time for expiring executors from within the blacklist, - // we *never* expire from *within* the stage blacklist + // Despite advancing beyond the time for expiring executors from within the excludelist, + // we *never* expire from *within* the stage excludelist clock.advance(rescheduleDelay) { @@ -1358,20 +1358,20 @@ class TaskSetManagerSuite assert(manager3.name === "TaskSet_1.1") } - test("don't update blacklist for shuffle-fetch failures, preemption, denied commits, " + + test("don't update excludelist for shuffle-fetch failures, preemption, denied commits, " + "or killed tasks") { // Setup a taskset, and fail some tasks for a fetch failure, preemption, denied commit, // and killed task. val conf = new SparkConf(). - set(config.BLACKLIST_ENABLED, true) + set(config.EXCLUDE_ON_FAILURE_ENABLED, true) sc = new SparkContext("local", "test", conf) sched = new FakeTaskScheduler(sc, ("exec1", "host1"), ("exec2", "host2")) val taskSet = FakeTask.createTaskSet(4) val tsm = new TaskSetManager(sched, taskSet, 4) - // we need a spy so we can attach our mock blacklist + // we need a spy so we can attach our mock excludelist val tsmSpy = spy(tsm) - val blacklist = mock(classOf[TaskSetBlacklist]) - when(tsmSpy.taskSetBlacklistHelperOpt).thenReturn(Some(blacklist)) + val excludelist = mock(classOf[TaskSetExcludelist]) + when(tsmSpy.taskSetExcludelistHelperOpt).thenReturn(Some(excludelist)) // make some offers to our taskset, to get tasks we will fail val taskDescs = Seq( @@ -1392,23 +1392,23 @@ class TaskSetManagerSuite TaskCommitDenied(0, 2, 0)) tsmSpy.handleFailedTask(taskDescs(3).taskId, TaskState.KILLED, TaskKilled("test")) - // Make sure that the blacklist ignored all of the task failures above, since they aren't + // Make sure that the excludelist ignored all of the task failures above, since they aren't // the fault of the executor where the task was running. - verify(blacklist, never()) - .updateBlacklistForFailedTask(anyString(), anyString(), anyInt(), anyString()) + verify(excludelist, never()) + .updateExcludedForFailedTask(anyString(), anyString(), anyInt(), anyString()) } - test("update application blacklist for shuffle-fetch") { + test("update application healthTracker for shuffle-fetch") { // Setup a taskset, and fail some one task for fetch failure. val conf = new SparkConf() - .set(config.BLACKLIST_ENABLED, true) + .set(config.EXCLUDE_ON_FAILURE_ENABLED, true) .set(config.SHUFFLE_SERVICE_ENABLED, true) - .set(config.BLACKLIST_FETCH_FAILURE_ENABLED, true) + .set(config.EXCLUDE_ON_FAILURE_FETCH_FAILURE_ENABLED, true) sc = new SparkContext("local", "test", conf) sched = new FakeTaskScheduler(sc, ("exec1", "host1"), ("exec2", "host2")) val taskSet = FakeTask.createTaskSet(4) - val blacklistTracker = new BlacklistTracker(sc, None) - val tsm = new TaskSetManager(sched, taskSet, 4, Some(blacklistTracker)) + val healthTracker = new HealthTracker(sc, None) + val tsm = new TaskSetManager(sched, taskSet, 4, Some(healthTracker)) // make some offers to our taskset, to get tasks we will fail val taskDescs = Seq( @@ -1420,22 +1420,22 @@ class TaskSetManagerSuite } assert(taskDescs.size === 4) - assert(!blacklistTracker.isExecutorBlacklisted(taskDescs(0).executorId)) - assert(!blacklistTracker.isNodeBlacklisted("host1")) + assert(!healthTracker.isExecutorExcluded(taskDescs(0).executorId)) + assert(!healthTracker.isNodeExcluded("host1")) // Fail the task with fetch failure tsm.handleFailedTask(taskDescs(0).taskId, TaskState.FAILED, FetchFailed(BlockManagerId(taskDescs(0).executorId, "host1", 12345), 0, 0L, 0, 0, "ignored")) - assert(blacklistTracker.isNodeBlacklisted("host1")) + assert(healthTracker.isNodeExcluded("host1")) } - test("update blacklist before adding pending task to avoid race condition") { - // When a task fails, it should apply the blacklist policy prior to + test("update healthTracker before adding pending task to avoid race condition") { + // When a task fails, it should apply the excludeOnFailure policy prior to // retrying the task otherwise there's a race condition where run on // the same executor that it was intended to be black listed from. val conf = new SparkConf(). - set(config.BLACKLIST_ENABLED, true) + set(config.EXCLUDE_ON_FAILURE_ENABLED, true) // Create a task with two executors. sc = new SparkContext("local", "test", conf) @@ -1448,8 +1448,8 @@ class TaskSetManagerSuite val clock = new ManualClock val mockListenerBus = mock(classOf[LiveListenerBus]) - val blacklistTracker = new BlacklistTracker(mockListenerBus, conf, None, clock) - val taskSetManager = new TaskSetManager(sched, taskSet, 1, Some(blacklistTracker)) + val healthTracker = new HealthTracker(mockListenerBus, conf, None, clock) + val taskSetManager = new TaskSetManager(sched, taskSet, 1, Some(healthTracker)) val taskSetManagerSpy = spy(taskSetManager) val taskDesc = taskSetManagerSpy.resourceOffer(exec, host, TaskLocality.ANY)._1 @@ -1458,8 +1458,8 @@ class TaskSetManagerSuite when(taskSetManagerSpy.addPendingTask(anyInt(), anyBoolean(), anyBoolean())).thenAnswer( (invocationOnMock: InvocationOnMock) => { val task: Int = invocationOnMock.getArgument(0) - assert(taskSetManager.taskSetBlacklistHelperOpt.get. - isExecutorBlacklistedForTask(exec, task)) + assert(taskSetManager.taskSetExcludelistHelperOpt.get. + isExecutorExcludedForTask(exec, task)) } ) diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala index 397fdce8ae6e3..4acb4bbc779c3 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala @@ -31,7 +31,7 @@ class KryoSerializerDistributedSuite extends SparkFunSuite with LocalSparkContex .set(config.SERIALIZER, "org.apache.spark.serializer.KryoSerializer") .set(config.Kryo.KRYO_USER_REGISTRATORS, Seq(classOf[AppJarRegistrator].getName)) .set(config.TASK_MAX_FAILURES, 1) - .set(config.BLACKLIST_ENABLED, false) + .set(config.EXCLUDE_ON_FAILURE_ENABLED, false) val jar = TestUtils.createJarWithClasses(List(AppJarRegistrator.customClassName)) conf.setJars(List(jar.getPath)) diff --git a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala index d5829c352be9b..6ca1109791c35 100644 --- a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala @@ -256,9 +256,9 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { } } - // Blacklisting executor for stage + // Excluding executor for stage time += 1 - listener.onExecutorBlacklistedForStage(SparkListenerExecutorBlacklistedForStage( + listener.onExecutorExcludedForStage(SparkListenerExecutorExcludedForStage( time = time, executorId = execIds.head, taskFailures = 2, @@ -273,18 +273,21 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { assert(executorStageSummaryWrappers.nonEmpty) executorStageSummaryWrappers.foreach { exec => - // only the first executor is expected to be blacklisted - val expectedBlacklistedFlag = exec.executorId == execIds.head - assert(exec.info.isBlacklistedForStage === expectedBlacklistedFlag) + // only the first executor is expected to be excluded + val expectedExcludedFlag = exec.executorId == execIds.head + assert(exec.info.isBlacklistedForStage === expectedExcludedFlag) + assert(exec.info.isExcludedForStage === expectedExcludedFlag) } check[ExecutorSummaryWrapper](execIds.head) { exec => assert(exec.info.blacklistedInStages === Set(stages.head.stageId)) + assert(exec.info.excludedInStages === Set(stages.head.stageId)) + } - // Blacklisting node for stage + // Excluding node for stage time += 1 - listener.onNodeBlacklistedForStage(SparkListenerNodeBlacklistedForStage( + listener.onNodeExcludedForStage(SparkListenerNodeExcludedForStage( time = time, hostId = "2.example.com", // this is where the second executor is hosted executorFailures = 1, @@ -299,8 +302,10 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { assert(executorStageSummaryWrappersForNode.nonEmpty) executorStageSummaryWrappersForNode.foreach { exec => - // both executor is expected to be blacklisted + // both executor is expected to be excluded assert(exec.info.isBlacklistedForStage) + assert(exec.info.isExcludedForStage) + } // Fail one of the tasks, re-start it. @@ -450,6 +455,7 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { check[ExecutorSummaryWrapper](execIds.head) { exec => assert(exec.info.blacklistedInStages === Set()) + assert(exec.info.excludedInStages === Set()) } // Submit stage 2. @@ -466,9 +472,9 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { assert(stage.info.submissionTime === Some(new Date(stages.last.submissionTime.get))) } - // Blacklisting node for stage + // Excluding node for stage time += 1 - listener.onNodeBlacklistedForStage(SparkListenerNodeBlacklistedForStage( + listener.onNodeExcludedForStage(SparkListenerNodeExcludedForStage( time = time, hostId = "1.example.com", executorFailures = 1, @@ -477,6 +483,7 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { check[ExecutorSummaryWrapper](execIds.head) { exec => assert(exec.info.blacklistedInStages === Set(stages.last.stageId)) + assert(exec.info.excludedInStages === Set(stages.last.stageId)) } // Start and fail all tasks of stage 2. @@ -628,30 +635,34 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { assert(job.info.numSkippedTasks === s1Tasks.size) } - // Blacklist an executor. + // Exclude an executor. time += 1 - listener.onExecutorBlacklisted(SparkListenerExecutorBlacklisted(time, "1", 42)) + listener.onExecutorExcluded(SparkListenerExecutorExcluded(time, "1", 42)) check[ExecutorSummaryWrapper]("1") { exec => assert(exec.info.isBlacklisted) + assert(exec.info.isExcluded) } time += 1 - listener.onExecutorUnblacklisted(SparkListenerExecutorUnblacklisted(time, "1")) + listener.onExecutorUnexcluded(SparkListenerExecutorUnexcluded(time, "1")) check[ExecutorSummaryWrapper]("1") { exec => assert(!exec.info.isBlacklisted) + assert(!exec.info.isExcluded) } - // Blacklist a node. + // Exclude a node. time += 1 - listener.onNodeBlacklisted(SparkListenerNodeBlacklisted(time, "1.example.com", 2)) + listener.onNodeExcluded(SparkListenerNodeExcluded(time, "1.example.com", 2)) check[ExecutorSummaryWrapper]("1") { exec => assert(exec.info.isBlacklisted) + assert(exec.info.isExcluded) } time += 1 - listener.onNodeUnblacklisted(SparkListenerNodeUnblacklisted(time, "1.example.com")) + listener.onNodeUnexcluded(SparkListenerNodeUnexcluded(time, "1.example.com")) check[ExecutorSummaryWrapper]("1") { exec => assert(!exec.info.isBlacklisted) + assert(!exec.info.isExcluded) } // Stop executors. diff --git a/core/src/test/scala/org/apache/spark/status/api/v1/ExecutorSummarySuite.scala b/core/src/test/scala/org/apache/spark/status/api/v1/ExecutorSummarySuite.scala index 286911bdfc19a..541a7821a51fb 100644 --- a/core/src/test/scala/org/apache/spark/status/api/v1/ExecutorSummarySuite.scala +++ b/core/src/test/scala/org/apache/spark/status/api/v1/ExecutorSummarySuite.scala @@ -33,7 +33,8 @@ class ExecutorSummarySuite extends SparkFunSuite { 0, 0, 1, 100, 1, 100, 100, 10, false, 20, new Date(1600984336352L), - Option.empty, Option.empty, Map(), Option.empty, Set(), Option.empty, Map(), Map(), 1) + Option.empty, Option.empty, Map(), Option.empty, Set(), Option.empty, Map(), Map(), 1, + false, Set()) val expectedJson = "{\"id\":\"id\",\"hostPort\":\"host:port\",\"isActive\":true," + "\"rddBlocks\":1,\"memoryUsed\":10,\"diskUsed\":10,\"totalCores\":1,\"maxTasks\":1," + "\"activeTasks\":1,\"failedTasks\":0,\"completedTasks\":0,\"totalTasks\":1," + @@ -41,7 +42,8 @@ class ExecutorSummarySuite extends SparkFunSuite { "\"totalShuffleRead\":100,\"totalShuffleWrite\":10,\"isBlacklisted\":false," + "\"maxMemory\":20,\"addTime\":1600984336352,\"removeTime\":null,\"removeReason\":null," + "\"executorLogs\":{},\"memoryMetrics\":null,\"blacklistedInStages\":[]," + - "\"peakMemoryMetrics\":null,\"attributes\":{},\"resources\":{},\"resourceProfileId\":1}" + "\"peakMemoryMetrics\":null,\"attributes\":{},\"resources\":{},\"resourceProfileId\":1," + + "\"isExcluded\":false,\"excludedInStages\":[]}" val json = mapper.writeValueAsString(executorSummary) assert(expectedJson.equals(json)) val deserializeExecutorSummary = mapper.readValue(json, new TypeReference[ExecutorSummary] {}) diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 2ae51f425dcb5..4cd1fc19f1484 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -94,12 +94,18 @@ class JsonProtocolSuite extends SparkFunSuite { val executorAdded = SparkListenerExecutorAdded(executorAddedTime, "exec1", new ExecutorInfo("Hostee.awesome.com", 11, logUrlMap, attributes, resources.toMap, 4)) val executorRemoved = SparkListenerExecutorRemoved(executorRemovedTime, "exec2", "test reason") - val executorBlacklisted = SparkListenerExecutorBlacklisted(executorBlacklistedTime, "exec1", 22) + val executorBlacklisted = SparkListenerExecutorBlacklisted(executorExcludedTime, "exec1", 22) val executorUnblacklisted = - SparkListenerExecutorUnblacklisted(executorUnblacklistedTime, "exec1") - val nodeBlacklisted = SparkListenerNodeBlacklisted(nodeBlacklistedTime, "node1", 33) + SparkListenerExecutorUnblacklisted(executorUnexcludedTime, "exec1") + val nodeBlacklisted = SparkListenerNodeBlacklisted(nodeExcludedTime, "node1", 33) + val executorExcluded = SparkListenerExecutorExcluded(executorExcludedTime, "exec1", 22) + val executorUnexcluded = + SparkListenerExecutorUnexcluded(executorUnexcludedTime, "exec1") + val nodeExcluded = SparkListenerNodeExcluded(nodeExcludedTime, "node1", 33) val nodeUnblacklisted = - SparkListenerNodeUnblacklisted(nodeUnblacklistedTime, "node1") + SparkListenerNodeUnblacklisted(nodeUnexcludedTime, "node1") + val nodeUnexcluded = + SparkListenerNodeUnexcluded(nodeUnexcludedTime, "node1") val executorMetricsUpdate = { // Use custom accum ID for determinism val accumUpdates = @@ -147,8 +153,12 @@ class JsonProtocolSuite extends SparkFunSuite { testEvent(executorRemoved, executorRemovedJsonString) testEvent(executorBlacklisted, executorBlacklistedJsonString) testEvent(executorUnblacklisted, executorUnblacklistedJsonString) + testEvent(executorExcluded, executorExcludedJsonString) + testEvent(executorUnexcluded, executorUnexcludedJsonString) testEvent(nodeBlacklisted, nodeBlacklistedJsonString) testEvent(nodeUnblacklisted, nodeUnblacklistedJsonString) + testEvent(nodeExcluded, nodeExcludedJsonString) + testEvent(nodeUnexcluded, nodeUnexcludedJsonString) testEvent(executorMetricsUpdate, executorMetricsUpdateJsonString) testEvent(blockUpdated, blockUpdatedJsonString) testEvent(stageExecutorMetrics, stageExecutorMetricsJsonString) @@ -598,10 +608,10 @@ private[spark] object JsonProtocolSuite extends Assertions { private val jobCompletionTime = 1421191296660L private val executorAddedTime = 1421458410000L private val executorRemovedTime = 1421458922000L - private val executorBlacklistedTime = 1421458932000L - private val executorUnblacklistedTime = 1421458942000L - private val nodeBlacklistedTime = 1421458952000L - private val nodeUnblacklistedTime = 1421458962000L + private val executorExcludedTime = 1421458932000L + private val executorUnexcludedTime = 1421458942000L + private val nodeExcludedTime = 1421458952000L + private val nodeUnexcludedTime = 1421458962000L private def testEvent(event: SparkListenerEvent, jsonString: String): Unit = { val actualJsonString = compact(render(JsonProtocol.sparkEventToJson(event))) @@ -2415,36 +2425,70 @@ private[spark] object JsonProtocolSuite extends Assertions { s""" |{ | "Event" : "org.apache.spark.scheduler.SparkListenerExecutorBlacklisted", - | "time" : ${executorBlacklistedTime}, + | "time" : ${executorExcludedTime}, | "executorId" : "exec1", | "taskFailures" : 22 |} """.stripMargin + private val executorExcludedJsonString = + s""" + |{ + | "Event" : "org.apache.spark.scheduler.SparkListenerExecutorExcluded", + | "time" : ${executorExcludedTime}, + | "executorId" : "exec1", + | "taskFailures" : 22 + |} + """.stripMargin private val executorUnblacklistedJsonString = s""" |{ | "Event" : "org.apache.spark.scheduler.SparkListenerExecutorUnblacklisted", - | "time" : ${executorUnblacklistedTime}, + | "time" : ${executorUnexcludedTime}, | "executorId" : "exec1" |} """.stripMargin + private val executorUnexcludedJsonString = + s""" + |{ + | "Event" : "org.apache.spark.scheduler.SparkListenerExecutorUnexcluded", + | "time" : ${executorUnexcludedTime}, + | "executorId" : "exec1" + |} + """.stripMargin private val nodeBlacklistedJsonString = s""" |{ | "Event" : "org.apache.spark.scheduler.SparkListenerNodeBlacklisted", - | "time" : ${nodeBlacklistedTime}, + | "time" : ${nodeExcludedTime}, | "hostId" : "node1", | "executorFailures" : 33 |} """.stripMargin + private val nodeExcludedJsonString = + s""" + |{ + | "Event" : "org.apache.spark.scheduler.SparkListenerNodeExcluded", + | "time" : ${nodeExcludedTime}, + | "hostId" : "node1", + | "executorFailures" : 33 + |} + """.stripMargin private val nodeUnblacklistedJsonString = s""" |{ | "Event" : "org.apache.spark.scheduler.SparkListenerNodeUnblacklisted", - | "time" : ${nodeUnblacklistedTime}, + | "time" : ${nodeUnexcludedTime}, | "hostId" : "node1" |} """.stripMargin + private val nodeUnexcludedJsonString = + s""" + |{ + | "Event" : "org.apache.spark.scheduler.SparkListenerNodeUnexcluded", + | "time" : ${nodeUnexcludedTime}, + | "hostId" : "node1" + |} + """.stripMargin private val resourceProfileJsonString = """ |{ diff --git a/docs/configuration.md b/docs/configuration.md index d825a589dfd31..232ea4079d436 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -2146,113 +2146,113 @@ Apart from these, the following properties are also available, and may be useful 3.1.0 - spark.scheduler.blacklist.unschedulableTaskSetTimeout + spark.scheduler.excludeOnFailure.unschedulableTaskSetTimeout 120s The timeout in seconds to wait to acquire a new executor and schedule a task before aborting a - TaskSet which is unschedulable because of being completely blacklisted. + TaskSet which is unschedulable because all executors are exluded due to task failures. 2.4.1 - spark.blacklist.enabled + spark.excludeOnFailure.enabled false - If set to "true", prevent Spark from scheduling tasks on executors that have been blacklisted - due to too many task failures. The blacklisting algorithm can be further controlled by the - other "spark.blacklist" configuration options. + If set to "true", prevent Spark from scheduling tasks on executors that have been excluded + due to too many task failures. The algorithm used to exclude executors and nodes can be further + controlled by the other "spark.excludeOnFailure" configuration options. 2.1.0 - spark.blacklist.timeout + spark.excludeOnFailure.timeout 1h - (Experimental) How long a node or executor is blacklisted for the entire application, before it - is unconditionally removed from the blacklist to attempt running new tasks. + (Experimental) How long a node or executor is excluded for the entire application, before it + is unconditionally removed from the excludelist to attempt running new tasks. 2.1.0 - spark.blacklist.task.maxTaskAttemptsPerExecutor + spark.excludeOnFailure.task.maxTaskAttemptsPerExecutor 1 (Experimental) For a given task, how many times it can be retried on one executor before the - executor is blacklisted for that task. + executor is excluded for that task. 2.1.0 - spark.blacklist.task.maxTaskAttemptsPerNode + spark.excludeOnFailure.task.maxTaskAttemptsPerNode 2 (Experimental) For a given task, how many times it can be retried on one node, before the entire - node is blacklisted for that task. + node is excluded for that task. 2.1.0 - spark.blacklist.stage.maxFailedTasksPerExecutor + spark.excludeOnFailure.stage.maxFailedTasksPerExecutor 2 (Experimental) How many different tasks must fail on one executor, within one stage, before the - executor is blacklisted for that stage. + executor is excluded for that stage. 2.1.0 - spark.blacklist.stage.maxFailedExecutorsPerNode + spark.excludeOnFailure.stage.maxFailedExecutorsPerNode 2 - (Experimental) How many different executors are marked as blacklisted for a given stage, before + (Experimental) How many different executors are marked as excluded for a given stage, before the entire node is marked as failed for the stage. 2.1.0 - spark.blacklist.application.maxFailedTasksPerExecutor + spark.excludeOnFailure.application.maxFailedTasksPerExecutor 2 (Experimental) How many different tasks must fail on one executor, in successful task sets, - before the executor is blacklisted for the entire application. Blacklisted executors will + before the executor is excluded for the entire application. Excluded executors will be automatically added back to the pool of available resources after the timeout specified by - spark.blacklist.timeout. Note that with dynamic allocation, though, the executors + spark.excludeOnFailure.timeout. Note that with dynamic allocation, though, the executors may get marked as idle and be reclaimed by the cluster manager. 2.2.0 - spark.blacklist.application.maxFailedExecutorsPerNode + spark.excludeOnFailure.application.maxFailedExecutorsPerNode 2 - (Experimental) How many different executors must be blacklisted for the entire application, - before the node is blacklisted for the entire application. Blacklisted nodes will + (Experimental) How many different executors must be excluded for the entire application, + before the node is excluded for the entire application. Excluded nodes will be automatically added back to the pool of available resources after the timeout specified by - spark.blacklist.timeout. Note that with dynamic allocation, though, the executors - on the node may get marked as idle and be reclaimed by the cluster manager. + spark.excludeOnFailure.timeout. Note that with dynamic allocation, though, the + executors on the node may get marked as idle and be reclaimed by the cluster manager. 2.2.0 - spark.blacklist.killBlacklistedExecutors + spark.excludeOnFailure.killExcludedExecutors false (Experimental) If set to "true", allow Spark to automatically kill the executors - when they are blacklisted on fetch failure or blacklisted for the entire application, - as controlled by spark.blacklist.application.*. Note that, when an entire node is added - to the blacklist, all of the executors on that node will be killed. + when they are excluded on fetch failure or excluded for the entire application, + as controlled by spark.killExcludedExecutors.application.*. Note that, when an entire node is added + excluded, all of the executors on that node will be killed. 2.2.0 - spark.blacklist.application.fetchFailure.enabled + spark.excludeOnFailure.application.fetchFailure.enabled false - (Experimental) If set to "true", Spark will blacklist the executor immediately when a fetch + (Experimental) If set to "true", Spark will exclude the executor immediately when a fetch failure happens. If external shuffle service is enabled, then the whole node will be - blacklisted. + excluded. 2.3.0 diff --git a/docs/monitoring.md b/docs/monitoring.md index 97948f6fac4d9..3513fed7b3d78 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -1125,12 +1125,14 @@ This is the component with the largest amount of instrumented metrics - stages.failedStages.count - stages.skippedStages.count - stages.completedStages.count - - tasks.blackListedExecutors.count + - tasks.blackListedExecutors.count // deprecated use excludedExecutors instead + - tasks.excludedExecutors.count - tasks.completedTasks.count - tasks.failedTasks.count - tasks.killedTasks.count - tasks.skippedTasks.count - - tasks.unblackListedExecutors.count + - tasks.unblackListedExecutors.count // deprecated use unexcludedExecutors instead + - tasks.unexcludedExecutors.count - jobs.succeededJobs - jobs.failedJobs - jobDuration diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 6f7aaf2baeccd..5e8eb48093c8a 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -551,12 +551,12 @@ To use a custom metrics.properties for the application master and executors, upd 2.0.0 - spark.yarn.blacklist.executor.launch.blacklisting.enabled + spark.yarn.executor.launch.excludeOnFailure.enabled false - Flag to enable blacklisting of nodes having YARN resource allocation problems. - The error limit for blacklisting can be configured by - spark.blacklist.application.maxFailedExecutorsPerNode. + Flag to enable exclusion of nodes having YARN resource allocation problems. + The error limit for excluding can be configured by + spark.excludeOnFailure.application.maxFailedExecutorsPerNode. 2.4.0 diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala index 5655ef50d214f..4ea22ebd93eef 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala @@ -185,7 +185,7 @@ private[spark] class KubernetesClusterSchedulerBackend( Some(new HadoopDelegationTokenManager(conf, sc.hadoopConfiguration, driverEndpoint)) } - override protected def isBlacklisted(executorId: String, hostname: String): Boolean = { + override protected def isExecutorExcluded(executorId: String, hostname: String): Boolean = { podAllocator.isDeleted(executorId) } diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala index 32cd50298bc6c..bbe1ff495d8a6 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala @@ -63,7 +63,7 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( with MesosScheduler with MesosSchedulerUtils { - // Blacklist a agent after this many failures + // Exclude an agent after this many failures private val MAX_AGENT_FAILURES = 2 private val maxCoresOption = conf.get(config.CORES_MAX) @@ -667,12 +667,12 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( totalGpusAcquired -= gpus gpusByTaskId -= taskId } - // If it was a failure, mark the agent as failed for blacklisting purposes + // If it was a failure, mark the agent as failed for excluding purposes if (TaskState.isFailed(state)) { agent.taskFailures += 1 if (agent.taskFailures >= MAX_AGENT_FAILURES) { - logInfo(s"Blacklisting Mesos agent $agentId due to too many failures; " + + logInfo(s"Excluding Mesos agent $agentId due to too many failures; " + "is Spark installed on it?") } } diff --git a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala index 4d7f6441020b7..2b7272a490376 100644 --- a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala +++ b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala @@ -833,7 +833,7 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite when(driver.start()).thenReturn(Protos.Status.DRIVER_RUNNING) taskScheduler = mock[TaskSchedulerImpl] - when(taskScheduler.nodeBlacklist).thenReturn(Set[String]()) + when(taskScheduler.excludedNodes).thenReturn(Set[String]()) when(taskScheduler.sc).thenReturn(sc) externalShuffleClient = mock[MesosExternalBlockStoreClient] diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 9b99e8ff9265c..e23773229c560 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -567,10 +567,10 @@ private[spark] class ApplicationMaster( finish(FinalApplicationStatus.FAILED, ApplicationMaster.EXIT_MAX_EXECUTOR_FAILURES, s"Max number of executor failures ($maxNumExecutorFailures) reached") - } else if (allocator.isAllNodeBlacklisted) { + } else if (allocator.isAllNodeExcluded) { finish(FinalApplicationStatus.FAILED, ApplicationMaster.EXIT_MAX_EXECUTOR_FAILURES, - "Due to executor failures all available nodes are blacklisted") + "Due to executor failures all available nodes are excluded") } else { logDebug("Sending progress") allocator.allocateResources() @@ -792,7 +792,7 @@ private[spark] class ApplicationMaster( r.resourceProfileToTotalExecs, r.numLocalityAwareTasksPerResourceProfileId, r.hostToLocalTaskCount, - r.nodeBlacklist)) { + r.excludedNodes)) { resetAllocatorInterval() } context.reply(true) diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala index adbbbc01a0bd5..ef01a2ad95483 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -159,8 +159,8 @@ private[yarn] class YarnAllocator( private[spark] val failureTracker = new FailureTracker(sparkConf, clock) - private val allocatorBlacklistTracker = - new YarnAllocatorBlacklistTracker(sparkConf, amClient, failureTracker) + private val allocatorNodeHealthTracker = + new YarnAllocatorNodeHealthTracker(sparkConf, amClient, failureTracker) // Executor memory in MiB. protected val executorMemory = sparkConf.get(EXECUTOR_MEMORY).toInt @@ -238,7 +238,7 @@ private[yarn] class YarnAllocator( def getNumExecutorsFailed: Int = failureTracker.numFailedExecutors - def isAllNodeBlacklisted: Boolean = allocatorBlacklistTracker.isAllNodeBlacklisted + def isAllNodeExcluded: Boolean = allocatorNodeHealthTracker.isAllNodeExcluded /** * A sequence of pending container requests that have not yet been fulfilled. @@ -358,15 +358,15 @@ private[yarn] class YarnAllocator( * placement hint. * @param hostToLocalTaskCount a map of preferred hostname to possible task counts for each * ResourceProfile id to be used as container placement hint. - * @param nodeBlacklist blacklisted nodes, which is passed in to avoid allocating new containers - * on them. It will be used to update the application master's blacklist. + * @param excludedNodes excluded nodes, which is passed in to avoid allocating new containers + * on them. It will be used to update the applications excluded node list. * @return Whether the new requested total is different than the old value. */ def requestTotalExecutorsWithPreferredLocalities( resourceProfileToTotalExecs: Map[ResourceProfile, Int], numLocalityAwareTasksPerResourceProfileId: Map[Int, Int], hostToLocalTaskCountPerResourceProfileId: Map[Int, Map[String, Int]], - nodeBlacklist: Set[String]): Boolean = synchronized { + excludedNodes: Set[String]): Boolean = synchronized { this.numLocalityAwareTasksPerResourceProfileId = numLocalityAwareTasksPerResourceProfileId this.hostToLocalTaskCountPerResourceProfileId = hostToLocalTaskCountPerResourceProfileId @@ -377,7 +377,7 @@ private[yarn] class YarnAllocator( logInfo(s"Driver requested a total number of $numExecs executor(s) " + s"for resource profile id: ${rp.id}.") targetNumExecutorsPerResourceProfileId(rp.id) = numExecs - allocatorBlacklistTracker.setSchedulerBlacklistedNodes(nodeBlacklist) + allocatorNodeHealthTracker.setSchedulerExcludedNodes(excludedNodes) true } else { false @@ -416,7 +416,7 @@ private[yarn] class YarnAllocator( val allocateResponse = amClient.allocate(progressIndicator) val allocatedContainers = allocateResponse.getAllocatedContainers() - allocatorBlacklistTracker.setNumClusterNodes(allocateResponse.getNumClusterNodes) + allocatorNodeHealthTracker.setNumClusterNodes(allocateResponse.getNumClusterNodes) if (allocatedContainers.size > 0) { logDebug(("Allocated containers: %d. Current executor count: %d. " + @@ -827,7 +827,7 @@ private[yarn] class YarnAllocator( s"$diag Consider boosting ${EXECUTOR_MEMORY_OVERHEAD.key}." (true, message) case other_exit_status => - // SPARK-26269: follow YARN's blacklisting behaviour(see https://github + // SPARK-26269: follow YARN's behaviour(see https://github // .com/apache/hadoop/blob/228156cfd1b474988bc4fedfbf7edddc87db41e3/had // oop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/ap // ache/hadoop/yarn/util/Apps.java#L273 for details) @@ -837,7 +837,7 @@ private[yarn] class YarnAllocator( s". Diagnostics: ${completedContainer.getDiagnostics}.") } else { // completed container from a bad node - allocatorBlacklistTracker.handleResourceAllocationFailure(hostOpt) + allocatorNodeHealthTracker.handleResourceAllocationFailure(hostOpt) (true, s"Container from a bad node: $containerId$onHostStr" + s". Exit status: ${completedContainer.getExitStatus}" + s". Diagnostics: ${completedContainer.getDiagnostics}.") diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocatorBlacklistTracker.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocatorNodeHealthTracker.scala similarity index 63% rename from resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocatorBlacklistTracker.scala rename to resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocatorNodeHealthTracker.scala index 339d3715a7316..de9e190361428 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocatorBlacklistTracker.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocatorNodeHealthTracker.scala @@ -27,42 +27,43 @@ import org.apache.spark.SparkConf import org.apache.spark.deploy.yarn.config._ import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ -import org.apache.spark.scheduler.BlacklistTracker +import org.apache.spark.scheduler.HealthTracker import org.apache.spark.util.{Clock, SystemClock} /** - * YarnAllocatorBlacklistTracker is responsible for tracking the blacklisted nodes - * and synchronizing the node list to YARN. + * YarnAllocatorNodeHealthTracker is responsible for tracking the health of nodes + * and synchronizing the node list to YARN as to which nodes are excluded. * - * Blacklisted nodes are coming from two different sources: + * Excluding nodes are coming from two different sources: * * * * The reason to realize this logic here (and not in the driver) is to avoid possible delays - * between synchronizing the blacklisted nodes with YARN and resource allocations. + * between synchronizing the excluded nodes with YARN and resource allocations. */ -private[spark] class YarnAllocatorBlacklistTracker( +private[spark] class YarnAllocatorNodeHealthTracker( sparkConf: SparkConf, amClient: AMRMClient[ContainerRequest], failureTracker: FailureTracker) extends Logging { - private val blacklistTimeoutMillis = BlacklistTracker.getBlacklistTimeout(sparkConf) + private val excludeOnFailureTimeoutMillis = HealthTracker.getExludeOnFailureTimeout(sparkConf) - private val launchBlacklistEnabled = sparkConf.get(YARN_EXECUTOR_LAUNCH_BLACKLIST_ENABLED) + private val launchExcludeOnFailureEnabled = + sparkConf.get(YARN_EXECUTOR_LAUNCH_EXCLUDE_ON_FAILURE_ENABLED) private val maxFailuresPerHost = sparkConf.get(MAX_FAILED_EXEC_PER_NODE) private val excludeNodes = sparkConf.get(YARN_EXCLUDE_NODES).toSet - private val allocatorBlacklist = new HashMap[String, Long]() + private val allocatorExcludedNodeList = new HashMap[String, Long]() - private var currentBlacklistedYarnNodes = Set.empty[String] + private var currentExcludededYarnNodes = Set.empty[String] - private var schedulerBlacklist = Set.empty[String] + private var schedulerExcludedNodeList = Set.empty[String] private var numClusterNodes = Int.MaxValue @@ -72,72 +73,76 @@ private[spark] class YarnAllocatorBlacklistTracker( def handleResourceAllocationFailure(hostOpt: Option[String]): Unit = { hostOpt match { - case Some(hostname) if launchBlacklistEnabled => - // failures on an already blacklisted nodes are not even tracked. + case Some(hostname) if launchExcludeOnFailureEnabled => + // failures on an already excluded node are not even tracked. // otherwise, such failures could shutdown the application // as resource requests are asynchronous // and a late failure response could exceed MAX_EXECUTOR_FAILURES - if (!schedulerBlacklist.contains(hostname) && - !allocatorBlacklist.contains(hostname)) { + if (!schedulerExcludedNodeList.contains(hostname) && + !allocatorExcludedNodeList.contains(hostname)) { failureTracker.registerFailureOnHost(hostname) - updateAllocationBlacklistedNodes(hostname) + updateAllocationExcludedNodes(hostname) } case _ => failureTracker.registerExecutorFailure() } } - private def updateAllocationBlacklistedNodes(hostname: String): Unit = { + private def updateAllocationExcludedNodes(hostname: String): Unit = { val failuresOnHost = failureTracker.numFailuresOnHost(hostname) if (failuresOnHost > maxFailuresPerHost) { - logInfo(s"blacklisting $hostname as YARN allocation failed $failuresOnHost times") - allocatorBlacklist.put( + logInfo(s"excluding $hostname as YARN allocation failed $failuresOnHost times") + allocatorExcludedNodeList.put( hostname, - failureTracker.clock.getTimeMillis() + blacklistTimeoutMillis) - refreshBlacklistedNodes() + failureTracker.clock.getTimeMillis() + excludeOnFailureTimeoutMillis) + refreshExcludedNodes() } } - def setSchedulerBlacklistedNodes(schedulerBlacklistedNodesWithExpiry: Set[String]): Unit = { - this.schedulerBlacklist = schedulerBlacklistedNodesWithExpiry - refreshBlacklistedNodes() + def setSchedulerExcludedNodes(schedulerExcludedNodesWithExpiry: Set[String]): Unit = { + this.schedulerExcludedNodeList = schedulerExcludedNodesWithExpiry + refreshExcludedNodes() } - def isAllNodeBlacklisted: Boolean = { + def isAllNodeExcluded: Boolean = { if (numClusterNodes <= 0) { logWarning("No available nodes reported, please check Resource Manager.") false } else { - currentBlacklistedYarnNodes.size >= numClusterNodes + currentExcludededYarnNodes.size >= numClusterNodes } } - private def refreshBlacklistedNodes(): Unit = { - removeExpiredYarnBlacklistedNodes() - val allBlacklistedNodes = excludeNodes ++ schedulerBlacklist ++ allocatorBlacklist.keySet - synchronizeBlacklistedNodeWithYarn(allBlacklistedNodes) + private def refreshExcludedNodes(): Unit = { + removeExpiredYarnExcludedNodes() + val allExcludedNodes = + excludeNodes ++ schedulerExcludedNodeList ++ allocatorExcludedNodeList.keySet + synchronizeExcludedNodesWithYarn(allExcludedNodes) } - private def synchronizeBlacklistedNodeWithYarn(nodesToBlacklist: Set[String]): Unit = { - // Update blacklist information to YARN ResourceManager for this application, + private def synchronizeExcludedNodesWithYarn(nodesToExclude: Set[String]): Unit = { + // Update YARN with the nodes that are excluded for this application, // in order to avoid allocating new Containers on the problematic nodes. - val additions = (nodesToBlacklist -- currentBlacklistedYarnNodes).toList.sorted - val removals = (currentBlacklistedYarnNodes -- nodesToBlacklist).toList.sorted + val additions = (nodesToExclude -- currentExcludededYarnNodes).toList.sorted + val removals = (currentExcludededYarnNodes -- nodesToExclude).toList.sorted if (additions.nonEmpty) { - logInfo(s"adding nodes to YARN application master's blacklist: $additions") + logInfo(s"adding nodes to YARN application master's excluded node list: $additions") } if (removals.nonEmpty) { - logInfo(s"removing nodes from YARN application master's blacklist: $removals") + logInfo(s"removing nodes from YARN application master's excluded node list: $removals") } if (additions.nonEmpty || removals.nonEmpty) { + // Note YARNs api for excluding nodes is updateBlacklist. + // TODO - We need to update once Hadoop changes - + // https://issues.apache.org/jira/browse/HADOOP-17169 amClient.updateBlacklist(additions.asJava, removals.asJava) } - currentBlacklistedYarnNodes = nodesToBlacklist + currentExcludededYarnNodes = nodesToExclude } - private def removeExpiredYarnBlacklistedNodes(): Unit = { + private def removeExpiredYarnExcludedNodes(): Unit = { val now = failureTracker.clock.getTimeMillis() - allocatorBlacklist.retain { (_, expiryTime) => expiryTime > now } + allocatorExcludedNodeList.retain { (_, expiryTime) => expiryTime > now } } } diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala index 1b0bf295db499..f2e838f6270c9 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala @@ -379,14 +379,15 @@ package object config extends Logging { .stringConf .createOptional - /* YARN allocator-level blacklisting related config entries. */ - private[spark] val YARN_EXECUTOR_LAUNCH_BLACKLIST_ENABLED = - ConfigBuilder("spark.yarn.blacklist.executor.launch.blacklisting.enabled") - .version("2.4.0") + /* YARN allocator-level excludeOnFailure related config entries. */ + private[spark] val YARN_EXECUTOR_LAUNCH_EXCLUDE_ON_FAILURE_ENABLED = + ConfigBuilder("spark.yarn.executor.launch.excludeOnFailure.enabled") + .version("3.1.0") + .withAlternative("spark.yarn.blacklist.executor.launch.blacklisting.enabled") .booleanConf .createWithDefault(false) - /* Initially blacklisted YARN nodes. */ + /* Initially excluded YARN nodes. */ private[spark] val YARN_EXCLUDE_NODES = ConfigBuilder("spark.yarn.exclude.nodes") .version("3.0.0") .stringConf diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala index 3f2e8846e85b3..b42bdb9816600 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala @@ -132,13 +132,13 @@ private[spark] abstract class YarnSchedulerBackend( private[cluster] def prepareRequestExecutors( resourceProfileToTotalExecs: Map[ResourceProfile, Int]): RequestExecutors = { - val nodeBlacklist: Set[String] = scheduler.nodeBlacklist() - // For locality preferences, ignore preferences for nodes that are blacklisted + val excludedNodes: Set[String] = scheduler.excludedNodes() + // For locality preferences, ignore preferences for nodes that are excluded val filteredRPHostToLocalTaskCount = rpHostToLocalTaskCount.map { case (rpid, v) => - (rpid, v.filter { case (host, count) => !nodeBlacklist.contains(host) }) + (rpid, v.filter { case (host, count) => !excludedNodes.contains(host) }) } RequestExecutors(resourceProfileToTotalExecs, numLocalityAwareTasksPerResourceProfileId, - filteredRPHostToLocalTaskCount, nodeBlacklist) + filteredRPHostToLocalTaskCount, excludedNodes) } /** diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorBlacklistTrackerSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorHealthTrackerSuite.scala similarity index 54% rename from resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorBlacklistTrackerSuite.scala rename to resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorHealthTrackerSuite.scala index 97615f5c936b0..c2fd5ff316592 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorBlacklistTrackerSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorHealthTrackerSuite.scala @@ -26,14 +26,14 @@ import org.scalatest.BeforeAndAfterEach import org.scalatest.matchers.must.Matchers import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.yarn.config.{YARN_EXCLUDE_NODES, YARN_EXECUTOR_LAUNCH_BLACKLIST_ENABLED} -import org.apache.spark.internal.config.{BLACKLIST_TIMEOUT_CONF, MAX_FAILED_EXEC_PER_NODE} +import org.apache.spark.deploy.yarn.config.{YARN_EXCLUDE_NODES, YARN_EXECUTOR_LAUNCH_EXCLUDE_ON_FAILURE_ENABLED} +import org.apache.spark.internal.config.{EXCLUDE_ON_FAILURE_TIMEOUT_CONF, MAX_FAILED_EXEC_PER_NODE} import org.apache.spark.util.ManualClock -class YarnAllocatorBlacklistTrackerSuite extends SparkFunSuite with Matchers +class YarnAllocatorHealthTrackerSuite extends SparkFunSuite with Matchers with BeforeAndAfterEach { - val BLACKLIST_TIMEOUT = 100L + val EXCLUDE_TIMEOUT = 100L val MAX_FAILED_EXEC_PER_NODE_VALUE = 2 var sparkConf: SparkConf = _ @@ -42,117 +42,117 @@ class YarnAllocatorBlacklistTrackerSuite extends SparkFunSuite with Matchers override def beforeEach(): Unit = { sparkConf = new SparkConf() - sparkConf.set(BLACKLIST_TIMEOUT_CONF, BLACKLIST_TIMEOUT) - sparkConf.set(YARN_EXECUTOR_LAUNCH_BLACKLIST_ENABLED, true) + sparkConf.set(EXCLUDE_ON_FAILURE_TIMEOUT_CONF, EXCLUDE_TIMEOUT) + sparkConf.set(YARN_EXECUTOR_LAUNCH_EXCLUDE_ON_FAILURE_ENABLED, true) sparkConf.set(MAX_FAILED_EXEC_PER_NODE, MAX_FAILED_EXEC_PER_NODE_VALUE) clock = new ManualClock() amClientMock = mock(classOf[AMRMClient[ContainerRequest]]) super.beforeEach() } - private def createYarnAllocatorBlacklistTracker( - sparkConf: SparkConf = sparkConf): YarnAllocatorBlacklistTracker = { + private def createYarnAllocatorHealthTracker( + sparkConf: SparkConf = sparkConf): YarnAllocatorNodeHealthTracker = { val failureTracker = new FailureTracker(sparkConf, clock) - val yarnBlacklistTracker = - new YarnAllocatorBlacklistTracker(sparkConf, amClientMock, failureTracker) - yarnBlacklistTracker.setNumClusterNodes(4) - yarnBlacklistTracker + val yarnHealthTracker = + new YarnAllocatorNodeHealthTracker(sparkConf, amClientMock, failureTracker) + yarnHealthTracker.setNumClusterNodes(4) + yarnHealthTracker } - test("expiring its own blacklisted nodes") { - val yarnBlacklistTracker = createYarnAllocatorBlacklistTracker() + test("expiring its own excluded nodes") { + val yarnHealthTracker = createYarnAllocatorHealthTracker() (1 to MAX_FAILED_EXEC_PER_NODE_VALUE).foreach { _ => { - yarnBlacklistTracker.handleResourceAllocationFailure(Some("host")) - // host should not be blacklisted at these failures as MAX_FAILED_EXEC_PER_NODE is 2 + yarnHealthTracker.handleResourceAllocationFailure(Some("host")) + // host should not be excluded at these failures as MAX_FAILED_EXEC_PER_NODE is 2 verify(amClientMock, never()) .updateBlacklist(Arrays.asList("host"), Collections.emptyList()) } } - yarnBlacklistTracker.handleResourceAllocationFailure(Some("host")) - // the third failure on the host triggers the blacklisting + yarnHealthTracker.handleResourceAllocationFailure(Some("host")) + // the third failure on the host triggers the exclusion verify(amClientMock).updateBlacklist(Arrays.asList("host"), Collections.emptyList()) - clock.advance(BLACKLIST_TIMEOUT) + clock.advance(EXCLUDE_TIMEOUT) - // trigger synchronisation of blacklisted nodes with YARN - yarnBlacklistTracker.setSchedulerBlacklistedNodes(Set()) + // trigger synchronisation of excluded nodes with YARN + yarnHealthTracker.setSchedulerExcludedNodes(Set()) verify(amClientMock).updateBlacklist(Collections.emptyList(), Arrays.asList("host")) } - test("not handling the expiry of scheduler blacklisted nodes") { - val yarnBlacklistTracker = createYarnAllocatorBlacklistTracker() + test("not handling the expiry of scheduler excluded nodes") { + val yarnHealthTracker = createYarnAllocatorHealthTracker() - yarnBlacklistTracker.setSchedulerBlacklistedNodes(Set("host1", "host2")) + yarnHealthTracker.setSchedulerExcludedNodes(Set("host1", "host2")) verify(amClientMock) .updateBlacklist(Arrays.asList("host1", "host2"), Collections.emptyList()) // advance timer more then host1, host2 expiry time clock.advance(200L) - // expired blacklisted nodes (simulating a resource request) - yarnBlacklistTracker.setSchedulerBlacklistedNodes(Set("host1", "host2")) - // no change is communicated to YARN regarding the blacklisting + // expired excluded nodes (simulating a resource request) + yarnHealthTracker.setSchedulerExcludedNodes(Set("host1", "host2")) + // no change is communicated to YARN regarding the exclusion verify(amClientMock, times(0)).updateBlacklist(Collections.emptyList(), Collections.emptyList()) } - test("combining scheduler and allocation blacklist") { + test("combining scheduler and allocation excluded node list") { sparkConf.set(YARN_EXCLUDE_NODES, Seq("initial1", "initial2")) - val yarnBlacklistTracker = createYarnAllocatorBlacklistTracker(sparkConf) - yarnBlacklistTracker.setSchedulerBlacklistedNodes(Set()) + val yarnHealthTracker = createYarnAllocatorHealthTracker(sparkConf) + yarnHealthTracker.setSchedulerExcludedNodes(Set()) - // initial1 and initial2 is added as blacklisted nodes at the very first updateBlacklist call + // initial1 and initial2 is added as excluded nodes at the very first updateBlacklist call // and they are never removed verify(amClientMock) .updateBlacklist(Arrays.asList("initial1", "initial2"), Collections.emptyList()) (1 to MAX_FAILED_EXEC_PER_NODE_VALUE).foreach { _ => { - yarnBlacklistTracker.handleResourceAllocationFailure(Some("host1")) - // host1 should not be blacklisted at these failures as MAX_FAILED_EXEC_PER_NODE is 2 + yarnHealthTracker.handleResourceAllocationFailure(Some("host1")) + // host1 should not be excluded at these failures as MAX_FAILED_EXEC_PER_NODE is 2 verify(amClientMock, never()) .updateBlacklist(Arrays.asList("host1"), Collections.emptyList()) } } - // as this is the third failure on host1 the node will be blacklisted - yarnBlacklistTracker.handleResourceAllocationFailure(Some("host1")) + // as this is the third failure on host1 the node will be excluded + yarnHealthTracker.handleResourceAllocationFailure(Some("host1")) verify(amClientMock) .updateBlacklist(Arrays.asList("host1"), Collections.emptyList()) - yarnBlacklistTracker.setSchedulerBlacklistedNodes(Set("host2", "host3")) + yarnHealthTracker.setSchedulerExcludedNodes(Set("host2", "host3")) verify(amClientMock) .updateBlacklist(Arrays.asList("host2", "host3"), Collections.emptyList()) clock.advance(10L) - yarnBlacklistTracker.setSchedulerBlacklistedNodes(Set("host3", "host4")) + yarnHealthTracker.setSchedulerExcludedNodes(Set("host3", "host4")) verify(amClientMock) .updateBlacklist(Arrays.asList("host4"), Arrays.asList("host2")) } - test("blacklist all available nodes") { - val yarnBlacklistTracker = createYarnAllocatorBlacklistTracker() - yarnBlacklistTracker.setSchedulerBlacklistedNodes(Set("host1", "host2", "host3")) + test("exclude all available nodes") { + val yarnHealthTracker = createYarnAllocatorHealthTracker() + yarnHealthTracker.setSchedulerExcludedNodes(Set("host1", "host2", "host3")) verify(amClientMock) .updateBlacklist(Arrays.asList("host1", "host2", "host3"), Collections.emptyList()) clock.advance(60L) (1 to MAX_FAILED_EXEC_PER_NODE_VALUE).foreach { _ => { - yarnBlacklistTracker.handleResourceAllocationFailure(Some("host4")) - // host4 should not be blacklisted at these failures as MAX_FAILED_EXEC_PER_NODE is 2 + yarnHealthTracker.handleResourceAllocationFailure(Some("host4")) + // host4 should not be excluded at these failures as MAX_FAILED_EXEC_PER_NODE is 2 verify(amClientMock, never()) .updateBlacklist(Arrays.asList("host4"), Collections.emptyList()) } } - // the third failure on the host triggers the blacklisting - yarnBlacklistTracker.handleResourceAllocationFailure(Some("host4")) + // the third failure on the host triggers the exclusion + yarnHealthTracker.handleResourceAllocationFailure(Some("host4")) verify(amClientMock).updateBlacklist(Arrays.asList("host4"), Collections.emptyList()) - assert(yarnBlacklistTracker.isAllNodeBlacklisted) + assert(yarnHealthTracker.isAllNodeExcluded) } } diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala index 63e2b97e0ecab..6b5c72ad7f7aa 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala @@ -523,9 +523,10 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter handler.getNumUnexpectedContainerRelease should be (2) } - test("blacklisted nodes reflected in amClient requests") { - // Internally we track the set of blacklisted nodes, but yarn wants us to send *changes* - // to the blacklist. This makes sure we are sending the right updates. + test("excluded nodes reflected in amClient requests") { + // Internally we track the set of excluded nodes, but yarn wants us to send *changes* + // to it. Note the YARN api uses the term blacklist for excluded nodes. + // This makes sure we are sending the right updates. val mockAmClient = mock(classOf[AMRMClient[ContainerRequest]]) val (handler, _) = createAllocator(4, mockAmClient) val resourceProfileToTotalExecs = mutable.HashMap(defaultRP -> 1) @@ -534,14 +535,14 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter numLocalityAwareTasksPerResourceProfileId.toMap, Map(), Set("hostA")) verify(mockAmClient).updateBlacklist(Seq("hostA").asJava, Seq[String]().asJava) - val blacklistedNodes = Set( + val excludedNodes = Set( "hostA", "hostB" ) resourceProfileToTotalExecs(defaultRP) = 2 handler.requestTotalExecutorsWithPreferredLocalities(resourceProfileToTotalExecs.toMap, - numLocalityAwareTasksPerResourceProfileId.toMap, Map(), blacklistedNodes) + numLocalityAwareTasksPerResourceProfileId.toMap, Map(), excludedNodes) verify(mockAmClient).updateBlacklist(Seq("hostB").asJava, Seq[String]().asJava) resourceProfileToTotalExecs(defaultRP) = 3 handler.requestTotalExecutorsWithPreferredLocalities(resourceProfileToTotalExecs.toMap, @@ -592,7 +593,7 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter handler.getNumExecutorsFailed should be (0) } - test("SPARK-26269: YarnAllocator should have same blacklist behaviour with YARN") { + test("SPARK-26269: YarnAllocator should have same excludeOnFailure behaviour with YARN") { val rmClientSpy = spy(rmClient) val maxExecutors = 11 @@ -600,7 +601,7 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter maxExecutors, rmClientSpy, Map( - YARN_EXECUTOR_LAUNCH_BLACKLIST_ENABLED.key -> "true", + YARN_EXECUTOR_LAUNCH_EXCLUDE_ON_FAILURE_ENABLED.key -> "true", MAX_FAILED_EXEC_PER_NODE.key -> "0")) handler.updateResourceRequests() @@ -608,7 +609,7 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter val ids = 0 to maxExecutors val containers = createContainers(hosts, ids) - val nonBlacklistedStatuses = Seq( + val nonExcludedStatuses = Seq( ContainerExitStatus.SUCCESS, ContainerExitStatus.PREEMPTED, ContainerExitStatus.KILLED_EXCEEDED_VMEM, @@ -619,24 +620,24 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter ContainerExitStatus.ABORTED, ContainerExitStatus.DISKS_FAILED) - val nonBlacklistedContainerStatuses = nonBlacklistedStatuses.zipWithIndex.map { + val nonExcludedContainerStatuses = nonExcludedStatuses.zipWithIndex.map { case (exitStatus, idx) => createContainerStatus(containers(idx).getId, exitStatus) } - val BLACKLISTED_EXIT_CODE = 1 - val blacklistedStatuses = Seq(ContainerExitStatus.INVALID, BLACKLISTED_EXIT_CODE) + val EXCLUDED_EXIT_CODE = 1 + val excludedStatuses = Seq(ContainerExitStatus.INVALID, EXCLUDED_EXIT_CODE) - val blacklistedContainerStatuses = blacklistedStatuses.zip(9 until maxExecutors).map { + val excludedContainerStatuses = excludedStatuses.zip(9 until maxExecutors).map { case (exitStatus, idx) => createContainerStatus(containers(idx).getId, exitStatus) } handler.handleAllocatedContainers(containers.slice(0, 9)) - handler.processCompletedContainers(nonBlacklistedContainerStatuses) + handler.processCompletedContainers(nonExcludedContainerStatuses) verify(rmClientSpy, never()) .updateBlacklist(hosts.slice(0, 9).asJava, Collections.emptyList()) handler.handleAllocatedContainers(containers.slice(9, 11)) - handler.processCompletedContainers(blacklistedContainerStatuses) + handler.processCompletedContainers(excludedContainerStatuses) verify(rmClientSpy) .updateBlacklist(hosts.slice(9, 10).asJava, Collections.emptyList()) verify(rmClientSpy) diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackendSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackendSuite.scala index 9003c2f630975..7959bb55d7ffc 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackendSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackendSuite.scala @@ -44,9 +44,10 @@ class YarnSchedulerBackendSuite extends SparkFunSuite with MockitoSugar with Loc } private class TestTaskSchedulerImpl(sc: SparkContext) extends TaskSchedulerImpl(sc) { - val blacklistedNodes = new AtomicReference[Set[String]]() - def setNodeBlacklist(nodeBlacklist: Set[String]): Unit = blacklistedNodes.set(nodeBlacklist) - override def nodeBlacklist(): Set[String] = blacklistedNodes.get() + val excludedNodesList = new AtomicReference[Set[String]]() + def setNodeExcludeList(nodeExcludeList: Set[String]): Unit = + excludedNodesList.set(nodeExcludeList) + override def excludedNodes(): Set[String] = excludedNodesList.get() } private class TestYarnSchedulerBackend(scheduler: TaskSchedulerImpl, sc: SparkContext) @@ -56,7 +57,7 @@ class YarnSchedulerBackendSuite extends SparkFunSuite with MockitoSugar with Loc } } - test("RequestExecutors reflects node blacklist and is serializable") { + test("RequestExecutors reflects node excludelist and is serializable") { sc = new SparkContext("local", "YarnSchedulerBackendSuite") // Subclassing the TaskSchedulerImpl here instead of using Mockito. For details see SPARK-26891. val sched = new TestTaskSchedulerImpl(sc) @@ -65,7 +66,7 @@ class YarnSchedulerBackendSuite extends SparkFunSuite with MockitoSugar with Loc val ser = new JavaSerializer(sc.conf).newInstance() val defaultResourceProf = ResourceProfile.getOrCreateDefaultProfile(sc.getConf) for { - blacklist <- IndexedSeq(Set[String](), Set("a", "b", "c")) + excludelist <- IndexedSeq(Set[String](), Set("a", "b", "c")) numRequested <- 0 until 10 hostToLocalCount <- IndexedSeq( Map(defaultResourceProf.id -> Map.empty[String, Int]), @@ -73,14 +74,14 @@ class YarnSchedulerBackendSuite extends SparkFunSuite with MockitoSugar with Loc ) } { yarnSchedulerBackendExtended.setHostToLocalTaskCount(hostToLocalCount) - sched.setNodeBlacklist(blacklist) + sched.setNodeExcludeList(excludelist) val request = Map(defaultResourceProf -> numRequested) val req = yarnSchedulerBackendExtended.prepareRequestExecutors(request) assert(req.resourceProfileToTotalExecs(defaultResourceProf) === numRequested) - assert(req.nodeBlacklist === blacklist) + assert(req.excludedNodes === excludelist) val hosts = req.hostToLocalTaskCount(ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID).keySet - assert(hosts.intersect(blacklist).isEmpty) + assert(hosts.intersect(excludelist).isEmpty) // Serialize to make sure serialization doesn't throw an error ser.serialize(req) }