From 9a6aaede723b206a95a616ae740dcc5ed6b74cbc Mon Sep 17 00:00:00 2001 From: mwws Date: Tue, 29 Dec 2015 14:01:17 +0800 Subject: [PATCH 01/57] enhance blacklist mechanism 1. create new BlacklistTracker and BlacklistStrategy interface to support complex use case for blacklist mechanism. 2. make Yarn allocator aware of node blacklist information 3. three strategies implemented for convenience, also user can define his own strategy SingleTaskStrategy: remain default behavior before this change. AdvanceSingleTaskStrategy: enhance SingleTaskStrategy by supporting stage level node blacklist ExecutorAndNodeStrategy: different taskSet can share blacklist information. --- .../scala/org/apache/spark/SparkContext.scala | 15 + .../spark/scheduler/BlacklistStrategy.scala | 149 +++++++++ .../spark/scheduler/BlacklistTracker.scala | 286 ++++++++++++++++++ .../spark/scheduler/TaskSetManager.scala | 46 ++- .../cluster/CoarseGrainedClusterMessage.scala | 3 +- .../CoarseGrainedSchedulerBackend.scala | 3 + .../apache/spark/HeartbeatReceiverSuite.scala | 4 +- .../scheduler/BlacklistTrackerSuite.scala | 177 +++++++++++ .../spark/scheduler/TaskSetManagerSuite.scala | 5 + docs/configuration.md | 28 ++ .../spark/deploy/yarn/ApplicationMaster.scala | 5 +- .../spark/deploy/yarn/YarnAllocator.scala | 14 +- .../cluster/YarnSchedulerBackend.scala | 6 +- .../deploy/yarn/YarnAllocatorSuite.scala | 12 +- 14 files changed, 712 insertions(+), 41 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/scheduler/BlacklistStrategy.scala create mode 100644 core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala create mode 100644 core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index fe15052b62478..6b98c25a11bbf 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -217,6 +217,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli private var _jars: Seq[String] = _ private var _files: Seq[String] = _ private var _shutdownHookRef: AnyRef = _ + private var _blacklistTracker: Option[BlacklistTracker] = _ /* ------------------------------------------------------------------------------------- * | Accessors and public fields. These provide access to the internal state of the | @@ -326,6 +327,8 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli private[spark] def cleaner: Option[ContextCleaner] = _cleaner + private[spark] def blacklistTracker: Option[BlacklistTracker] = _blacklistTracker + private[spark] var checkpointDir: Option[String] = None // Thread Local variable that can be used by users to pass information down the stack @@ -533,6 +536,14 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli } _executorAllocationManager.foreach(_.start()) + // By default blacklistTracker is enabled. + _blacklistTracker = if (_conf.getBoolean("spark.scheduler.blacklist.enabled", true)){ + Some(new BlacklistTracker(_conf)) + } else { + None + } + _blacklistTracker.foreach(_.start()) + _cleaner = if (_conf.getBoolean("spark.cleaner.referenceTracking", true)) { Some(new ContextCleaner(this)) @@ -1765,6 +1776,10 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli Utils.tryLogNonFatalError { _executorAllocationManager.foreach(_.stop()) } + Utils.tryLogNonFatalError { + _blacklistTracker.foreach(_.stop()) + } + if (_listenerBusStarted) { Utils.tryLogNonFatalError { listenerBus.stop() diff --git a/core/src/main/scala/org/apache/spark/scheduler/BlacklistStrategy.scala b/core/src/main/scala/org/apache/spark/scheduler/BlacklistStrategy.scala new file mode 100644 index 0000000000000..a3e0772871b9f --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/BlacklistStrategy.scala @@ -0,0 +1,149 @@ +/* + * 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 scala.collection.mutable + +import org.apache.spark.SparkConf +import org.apache.spark.util.Clock + +/** + * The interface to determine executor blacklist and node blacklist. + */ +private[scheduler] trait BlacklistStrategy { + /** Define a time interval to expire failure information of executors */ + val expireTimeInMilliseconds: Long + + /** Return executors in blacklist which are related to given stage and partition */ + def getExecutorBlacklist( + executorIdToFailureStatus: mutable.HashMap[String, FailureStatus], + atomTask: StageAndPartition, + clock: Clock): Set[String] + + /** Return all nodes in blacklist */ + def getNodeBlacklist( + executorIdToFailureStatus: mutable.HashMap[String, FailureStatus], + clock: Clock): Set[String] + + /** + * Return all nodes in blacklist for specified stage. By default it returns the same result as + * getNodeBlacklist. It could be override in strategy implementation. + */ + def getNodeBlacklistForStage( + executorIdToFailureStatus: mutable.HashMap[String, FailureStatus], + stageId: Int, + clock: Clock): Set[String] = getNodeBlacklist(executorIdToFailureStatus, clock) + + /** + * Choose which executors should be removed from blacklist. Return true if any executors are + * removed from the blacklist, false otherwise. The default implementation removes executors from + * the blacklist after [[expireTimeInMilliseconds]] + */ + def expireExecutorsInBlackList( + executorIdToFailureStatus: mutable.HashMap[String, FailureStatus], + clock: Clock): Boolean = { + + val now = clock.getTimeMillis() + val expiredKey = executorIdToFailureStatus.filter { + case (executorid, failureStatus) => { + (now - failureStatus.updatedTime) >= expireTimeInMilliseconds + } + }.keySet + + if (expiredKey.isEmpty) { + false + } else { + executorIdToFailureStatus --= expiredKey + true + } + } +} + +/** + * This strategy is applied to keep the same semantics as standard behavior before spark 1.6. + * + * If an executor failed running "task A", then we think this executor is blacked for "task A", + * but at the same time. it is still healthy for other task. Node blacklist is always empty. + */ +private[scheduler] class SingleTaskStrategy( + val expireTimeInMilliseconds: Long) extends BlacklistStrategy { + def getExecutorBlacklist( + executorIdToFailureStatus: mutable.HashMap[String, FailureStatus], + atomTask: StageAndPartition, + clock: Clock): Set[String] = { + executorIdToFailureStatus.filter{ + case (_, failureStatus) => failureStatus.numFailuresPerTask.keySet.contains(atomTask) && + clock.getTimeMillis() - failureStatus.updatedTime < expireTimeInMilliseconds + }.keys.toSet + } + + def getNodeBlacklist( + executorIdToFailureStatus: mutable.HashMap[String, FailureStatus], + clock: Clock): Set[String] = Set.empty[String] +} + +/** + * Comparing to SingleTaskStrategy, it supports node blacklist. With this strategy, once more than + * one executor failed running for specific stage, we put all executors on the same node into + * blacklist. So all tasks from the same stage will not be allocated to that node. + */ +private[scheduler] class AdvancedSingleTaskStrategy( + expireTimeInMilliseconds: Long) extends SingleTaskStrategy(expireTimeInMilliseconds) { + + override def getNodeBlacklistForStage( + executorIdToFailureStatus: mutable.HashMap[String, FailureStatus], + stageId: Int, + clock: Clock): Set[String] = { + val nodes = executorIdToFailureStatus.filter{ + case (_, failureStatus) => + failureStatus.numFailuresPerTask.keySet.map(_.stageId).contains(stageId) && + clock.getTimeMillis() - failureStatus.updatedTime < expireTimeInMilliseconds + }.values.map(_.host) + getDuplicateElem(nodes, 1) + } + + override def getNodeBlacklist( + executorIdToFailureStatus: mutable.HashMap[String, FailureStatus], + clock: Clock): Set[String] = { + // resolve a nodes sequence from failure status. + val nodes = executorIdToFailureStatus.values.map(_.host) + getDuplicateElem(nodes, 1) + } + + // A help function to find hosts which have more than "depTimes" executors on it in blacklist + private def getDuplicateElem(ndoes: Iterable[String], dupTimes: Int): Set[String] = { + ndoes.groupBy(identity).mapValues(_.size) // resolve map (nodeName => occurred times) + .filter(ele => ele._2 > dupTimes) // return nodes which occurred more than dupTimes. + .keys.toSet + } +} + +/** + * Create BlacklistStrategy instance according to SparkConf + */ +private[scheduler] object BlacklistStrategy { + def apply(sparkConf: SparkConf): BlacklistStrategy = { + val timeout = sparkConf.getTimeAsMs("spark.scheduler.blacklist.timeout", + sparkConf.getLong("spark.scheduler.executorTaskBlacklistTime", 0L).toString() + "ms") + + sparkConf.getBoolean("spark.scheduler.blacklist.advancedStrategy", false) match { + case false => new SingleTaskStrategy(timeout) + case true => new AdvancedSingleTaskStrategy(timeout) + } + } +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala new file mode 100644 index 0000000000000..57c16927a48fd --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala @@ -0,0 +1,286 @@ +/* + * 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.TimeUnit + +import scala.collection.mutable + +import org.apache.spark.SparkConf +import org.apache.spark.Success +import org.apache.spark.TaskEndReason +import org.apache.spark.util.Clock +import org.apache.spark.util.SystemClock +import org.apache.spark.util.ThreadUtils +import org.apache.spark.util.Utils + +/** + * BlacklistTracker is designed to track problematic executors and nodes. It belongs to + * SparkContext as an centralized and unified collection for all tasks with same SparkContext. + * So that a new TaskSet could be benefit from previous experiences of other TaskSets. + * + * Once task finished, the callback method in TaskSetManager should update + * executorIdToFailureStatus Map. + */ +private[spark] class BlacklistTracker( + sparkConf: SparkConf, + clock: Clock = new SystemClock()) extends BlacklistCache{ + + // maintain a ExecutorId --> FailureStatus HashMap + private val executorIdToFailureStatus: mutable.HashMap[String, FailureStatus] = mutable.HashMap() + // Apply Strategy pattern here to change different blacklist detection logic + private val strategy = BlacklistStrategy(sparkConf) + + + // A daemon thread to expire blacklist executor periodically + private val scheduler = ThreadUtils.newDaemonSingleThreadScheduledExecutor( + "spark-scheduler-blacklist-expire-timer") + + private val recoverPeriod = sparkConf.getTimeAsSeconds( + "spark.scheduler.blacklist.recoverPeriod", "60s") + + def start(): Unit = { + val scheduleTask = new Runnable() { + override def run(): Unit = { + Utils.logUncaughtExceptions(expireExecutorsInBlackList()) + } + } + scheduler.scheduleAtFixedRate(scheduleTask, 0L, recoverPeriod, TimeUnit.SECONDS) + } + + def stop(): Unit = { + scheduler.shutdown() + scheduler.awaitTermination(10, TimeUnit.SECONDS) + } + + // The actual implementation is delegated to strategy + private[scheduler] def expireExecutorsInBlackList(): Unit = synchronized { + val updated = strategy.expireExecutorsInBlackList(executorIdToFailureStatus, clock) + if (updated) { + invalidateCache() + } + } + + // The actual implementation is delegated to strategy + def executorBlacklist( + sched: TaskSchedulerImpl, stageId: Int, partition: Int): Set[String] = synchronized { + val atomTask = StageAndPartition(stageId, partition) + if (!isBlacklistExecutorCacheValid) { + reEvaluateExecutorBlacklistAndUpdateCache(sched, atomTask, clock) + } else { + getExecutorBlacklistFromCache(atomTask).getOrElse { + reEvaluateExecutorBlacklistAndUpdateCache(sched, atomTask, clock) + } + } + } + + // The actual implementation is delegated to strategy + def nodeBlacklist(): Set[String] = synchronized { + if (isBlacklistNodeCacheValid) { + getNodeBlacklistFromCache + } else { + val nodes = strategy.getNodeBlacklist(executorIdToFailureStatus, clock) + updateBlacklistNodeCache(nodes) + nodes + } + } + + // The actual implementation is delegated to strategy + def nodeBlacklistForStage(stageId: Int): Set[String] = synchronized { + if (isBlacklistNodeForStageCacheValid) { + getNodeBlacklistForStageFromCache(stageId).getOrElse( + reEvaluateNodeBlacklistForStageAndUpdateCache(stageId)) + } else { + reEvaluateNodeBlacklistForStageAndUpdateCache(stageId) + } + } + + def updateFailedExecutors( + stageId: Int, partition: Int, + info: TaskInfo, + reason: TaskEndReason) : Unit = synchronized { + + val atomTask = StageAndPartition(stageId, partition) + reason match { + // If the task succeeded, remove related record from executorIdToFailureStatus + case Success => + removeFailedExecutorsForTaskId(info.executorId, stageId, partition) + + // If the task failed, update latest failure time and failedTaskIds + case _ => + val executorId = info.executorId + executorIdToFailureStatus.get(executorId) match { + case Some(failureStatus) => + failureStatus.updatedTime = clock.getTimeMillis() + val failedTimes = failureStatus.numFailuresPerTask.getOrElse(atomTask, 0) + 1 + failureStatus.numFailuresPerTask(atomTask) = failedTimes + case None => + val failedTasks = mutable.HashMap(atomTask -> 1) + val failureStatus = new FailureStatus( + clock.getTimeMillis(), info.host, failedTasks) + executorIdToFailureStatus(executorId) = failureStatus + } + invalidateCache() + } + } + + /** remove the executorId from executorIdToFailureStatus */ + def removeFailedExecutors(executorId: String) : Unit = synchronized { + executorIdToFailureStatus.remove(executorId) + invalidateCache() + } + + /** + * remove the failure record related to given taskId from executorIdToFailureStatus. If the + * number of records of given executorId becomes 0, remove the completed executorId. + */ + def removeFailedExecutorsForTaskId( + executorId: String, + stageId: Int, + partition: Int) : Unit = synchronized { + val atomTask = StageAndPartition(stageId, partition) + executorIdToFailureStatus.get(executorId).map{ fs => + fs.numFailuresPerTask.remove(atomTask) + if (fs.numFailuresPerTask.isEmpty) { + executorIdToFailureStatus.remove(executorId) + } + invalidateCache() + } + } + + def isExecutorBlacklisted( + executorId: String, + sched: TaskSchedulerImpl, + stageId: Int, + partition: Int) : Boolean = { + + executorBlacklist(sched, stageId, partition).contains(executorId) + } + + // If the node is in blacklist, all executors allocated on that node will + // also be put into executor blacklist. + private def executorsOnBlacklistedNode( + sched: TaskSchedulerImpl, + atomTask: StageAndPartition): Set[String] = { + nodeBlacklistForStage(atomTask.stageId).flatMap(sched.getExecutorsAliveOnHost(_) + .getOrElse(Set.empty[String])).toSet + } + + private def reEvaluateExecutorBlacklistAndUpdateCache( + sched: TaskSchedulerImpl, + atomTask: StageAndPartition, + clock: Clock): Set[String] = { + val executors = executorsOnBlacklistedNode(sched, atomTask) ++ + strategy.getExecutorBlacklist(executorIdToFailureStatus, atomTask, clock) + updateBlacklistExecutorCache(atomTask, executors) + executors + } + + private def reEvaluateNodeBlacklistForStageAndUpdateCache(stageId: Int): Set[String] = { + val nodes = strategy.getNodeBlacklistForStage(executorIdToFailureStatus, stageId, clock) + updateBlacklistNodeCache(nodes) + nodes + } +} + +/** + * Hide cache details in this trait to make code clean and avoid operation mistake + */ +private[scheduler] trait BlacklistCache { + + // local cache to minimize the the work when query blacklisted executor and node + private val blacklistExecutorCache = mutable.HashMap.empty[StageAndPartition, Set[String]] + private val blacklistNodeCache = mutable.Set.empty[String] + private val blacklistNodeForStageCache = mutable.HashMap.empty[Int, Set[String]] + + // The flag to mark if cache is valid, it will be set to false when executorIdToFailureStatus be + // updated and it will be set to true, when called executorBlacklist and nodeBlacklist. + private var _isBlacklistExecutorCacheValid : Boolean = false + private var _isBlacklistNodeCacheValid: Boolean = false + private var _isBlacklistNodeForStageCacheValid: Boolean = false + + private val cacheLock = new Object() + + protected def isBlacklistExecutorCacheValid : Boolean = _isBlacklistExecutorCacheValid + protected def isBlacklistNodeCacheValid: Boolean = _isBlacklistNodeCacheValid + protected def isBlacklistNodeForStageCacheValid: Boolean = _isBlacklistNodeForStageCacheValid + + protected def updateBlacklistExecutorCache( + atomTask: StageAndPartition, + blacklistExecutor: Set[String]): Unit = cacheLock.synchronized { + if (!_isBlacklistExecutorCacheValid) { + blacklistExecutorCache.clear() + } + blacklistExecutorCache.update(atomTask, blacklistExecutor) + _isBlacklistExecutorCacheValid = true + } + + protected def updateBlacklistNodeCache( + blacklistNode: Set[String]): Unit = cacheLock.synchronized { + if (!_isBlacklistNodeCacheValid) { + blacklistNodeCache.clear() + } + blacklistNodeCache ++= blacklistNode + _isBlacklistNodeCacheValid = true + } + + protected def updateBlacklistNodeForStageCache( + stageId: Int, + blacklistNode: Set[String]): Unit = cacheLock.synchronized { + if (!_isBlacklistNodeForStageCacheValid) { + blacklistNodeForStageCache.clear() + } + blacklistNodeForStageCache.update(stageId, blacklistNode) + _isBlacklistNodeForStageCacheValid = true + } + + protected def invalidateCache(): Unit = cacheLock.synchronized { + _isBlacklistExecutorCacheValid = false + _isBlacklistNodeCacheValid = false + _isBlacklistNodeForStageCacheValid = false + } + + protected def getExecutorBlacklistFromCache( + atomTask: StageAndPartition): Option[Set[String]] = { + blacklistExecutorCache.get(atomTask) + } + + protected def getNodeBlacklistFromCache: Set[String] = blacklistNodeCache.toSet + + protected def getNodeBlacklistForStageFromCache(stageId: Int): Option[Set[String]] = + blacklistNodeForStageCache.get(stageId) +} + +/** + * A class to record details of failure. + * + * @param initialTime the time when failure status be created + * @param host the node name which running executor on + * @param numFailuresPerTask all tasks failed on the executor (key is StageAndPartition, value + * is the number of failures of this task) + */ +private[scheduler] final class FailureStatus( + initialTime: Long, + val host: String, + val numFailuresPerTask: mutable.HashMap[StageAndPartition, Int]) { + + var updatedTime = initialTime + def totalNumFailures : Int = numFailuresPerTask.values.sum +} + +private[scheduler] case class StageAndPartition(val stageId: Int, val partition: Int) 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 2fef447b0a3c1..a906856d8e29c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -58,15 +58,6 @@ private[spark] class TaskSetManager( val conf = sched.sc.conf - /* - * Sometimes if an executor is dead or in an otherwise invalid state, the driver - * does not realize right away leading to repeated task failures. If enabled, - * this temporarily prevents a task from re-launching on an executor where - * it just failed. - */ - private val EXECUTOR_TASK_BLACKLIST_TIMEOUT = - conf.getLong("spark.scheduler.executorTaskBlacklistTime", 0L) - // Quantile of tasks at which to start speculation val SPECULATION_QUANTILE = conf.getDouble("spark.speculation.quantile", 0.75) val SPECULATION_MULTIPLIER = conf.getDouble("spark.speculation.multiplier", 1.5) @@ -83,8 +74,6 @@ private[spark] class TaskSetManager( val copiesRunning = new Array[Int](numTasks) val successful = new Array[Boolean](numTasks) private val numFailures = new Array[Int](numTasks) - // key is taskId (aka TaskInfo.index), value is a Map of executor id to when it failed - private val failedExecutors = new HashMap[Int, HashMap[String, Long]]() val taskAttempts = Array.fill[List[TaskInfo]](numTasks)(Nil) var tasksSuccessful = 0 @@ -250,7 +239,8 @@ private[spark] class TaskSetManager( while (indexOffset > 0) { indexOffset -= 1 val index = list(indexOffset) - if (!executorIsBlacklisted(execId, index)) { + if (!blacklistTracker.map(_.isExecutorBlacklisted(execId, sched, stageId, index)) + .getOrElse(false)) { // This should almost always be list.trimEnd(1) to remove tail list.remove(indexOffset) if (copiesRunning(index) == 0 && !successful(index)) { @@ -266,19 +256,11 @@ private[spark] class TaskSetManager( taskAttempts(taskIndex).exists(_.host == host) } - /** - * Is this re-execution of a failed task on an executor it already failed in before - * EXECUTOR_TASK_BLACKLIST_TIMEOUT has elapsed ? - */ - private[scheduler] def executorIsBlacklisted(execId: String, taskId: Int): Boolean = { - if (failedExecutors.contains(taskId)) { - val failed = failedExecutors.get(taskId).get - - return failed.contains(execId) && - clock.getTimeMillis() - failed.get(execId).get < EXECUTOR_TASK_BLACKLIST_TIMEOUT - } + var blacklistTracker = sched.sc.blacklistTracker - false + /** VisibleForTesting */ + private[scheduler] def setBlacklistTracker (tracker: BlacklistTracker) = { + blacklistTracker = Some(tracker) } /** @@ -293,7 +275,9 @@ private[spark] class TaskSetManager( speculatableTasks.retain(index => !successful(index)) // Remove finished tasks from set def canRunOnHost(index: Int): Boolean = - !hasAttemptOnHost(index, host) && !executorIsBlacklisted(execId, index) + !hasAttemptOnHost(index, host) && + !blacklistTracker.map(_.isExecutorBlacklisted(execId, sched, stageId, index)) + .getOrElse(false) if (!speculatableTasks.isEmpty) { // Check for process-local tasks; note that tasks can be process-local @@ -688,7 +672,10 @@ private[spark] class TaskSetManager( logInfo("Ignoring task-finished event for " + info.id + " in stage " + taskSet.id + " because task " + index + " has already completed successfully") } - failedExecutors.remove(index) + + blacklistTracker.foreach{ + _.updateFailedExecutors(stageId, tasks(index).partitionId, info, Success) + } maybeFinishTaskSet() } @@ -770,9 +757,12 @@ private[spark] class TaskSetManager( logError("Unknown TaskEndReason: " + e) None } + // always add to failed executors - failedExecutors.getOrElseUpdate(index, new HashMap[String, Long]()). - put(info.executorId, clock.getTimeMillis()) + blacklistTracker.foreach { + _.updateFailedExecutors(stageId, tasks(index).partitionId, info, reason) + } + sched.dagScheduler.taskEnded(tasks(index), reason, null, accumUpdates, info) if (successful(index)) { 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 edc8aac5d1515..684bf75c03d80 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 @@ -94,7 +94,8 @@ private[spark] object CoarseGrainedClusterMessages { case class RequestExecutors( requestedTotal: Int, localityAwareTasks: Int, - hostToLocalTaskCount: Map[String, Int]) + hostToLocalTaskCount: Map[String, Int], + nodeBlacklist: 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 8259923ce31c3..eb2c45c1fdbf3 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 @@ -296,6 +296,9 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp scheduler.sc.env.blockManager.master.removeExecutorAsync(executorId) logInfo(s"Asked to remove non-existent executor $executorId") } + + // Remove disconnected executor from blacklistTracker to keep consistency + scheduler.sc.blacklistTracker.foreach(_.removeFailedExecutors(executorId)) } /** diff --git a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala index 5e2ba311ee773..fd8819a8daba2 100644 --- a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala +++ b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala @@ -272,7 +272,7 @@ private class FakeSchedulerBackend( protected override def doRequestTotalExecutors(requestedTotal: Int): Boolean = { clusterManagerEndpoint.askWithRetry[Boolean]( - RequestExecutors(requestedTotal, localityAwareTasks, hostToLocalTaskCount)) + RequestExecutors(requestedTotal, localityAwareTasks, hostToLocalTaskCount, Set.empty[String])) } protected override def doKillExecutors(executorIds: Seq[String]): Boolean = { @@ -291,7 +291,7 @@ private class FakeClusterManager(override val rpcEnv: RpcEnv) extends RpcEndpoin def getExecutorIdsToKill: Set[String] = executorIdsToKill.toSet override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { - case RequestExecutors(requestedTotal, _, _) => + case RequestExecutors(requestedTotal, _, _, _) => targetNumExecutors = requestedTotal context.reply(true) case KillExecutors(executorIds) => diff --git a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala new file mode 100644 index 0000000000000..5d090d8f2b512 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala @@ -0,0 +1,177 @@ +/* + * 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.Mockito.when +import org.scalatest.BeforeAndAfter +import org.scalatest.mock.MockitoSugar + +import org.apache.spark.ExceptionFailure +import org.apache.spark.SparkConf +import org.apache.spark.SparkFunSuite +import org.apache.spark.Success +import org.apache.spark.TaskEndReason +import org.apache.spark.util.ManualClock + +class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfter with MockitoSugar { + + val FAILURE: TaskEndReason = new ExceptionFailure( + "Fake", + "fake failure", + Array.empty[StackTraceElement], + "fake stack trace", + None) + + val stage1 = 1 + val stage2 = 2 + + val partition1 = 1 + val partition2 = 2 + val partition3 = 3 + + // Variable name can indicate basic information of taskInfo + // hostA: executor 1, 2, 4 + // hostB: executor 3 + // The format is "taskInfo_executorId_hostName" + val taskInfo_1_hostA = new TaskInfo(1L, 1, 1, 0L, "1", "hostA", TaskLocality.ANY, false) + val taskInfo_2_hostA = new TaskInfo(2L, 1, 1, 0L, "2", "hostA", TaskLocality.ANY, false) + val taskInfo_3_hostB = new TaskInfo(3L, 3, 1, 0L, "3", "hostB", TaskLocality.ANY, false) + + val clock = new ManualClock(0) + + test ("expireExecutorsInBlacklist works") { + // expire time is set to 6s + val conf = new SparkConf().setAppName("test").setMaster("local") + .set("spark.ui.enabled", "false") + .set("spark.scheduler.executorTaskBlacklistTime", "6000") + + val scheduler = mock[TaskSchedulerImpl] + + val tracker = new BlacklistTracker(conf, clock) + // Executor 1 into blacklist at Time 00:00:00 + tracker.updateFailedExecutors(stage1, partition1, taskInfo_1_hostA, FAILURE) + assert(tracker.executorBlacklist(scheduler, stage1, 1) === Set("1")) + + clock.setTime(2000) + tracker.expireExecutorsInBlackList() + assert(tracker.executorBlacklist(scheduler, stage1, 1) === Set("1")) + // Executor 1 failed again at Time 00::00:02 + tracker.updateFailedExecutors(stage1, partition1, taskInfo_1_hostA, FAILURE) + + clock.setTime(3000) + // Executor 2 failed at Time 00:00:03 + tracker.updateFailedExecutors(stage1, partition1, taskInfo_2_hostA, FAILURE) + assert(tracker.executorBlacklist(scheduler, stage1, 1) === Set("1", "2")) + + clock.setTime(6000) + tracker.expireExecutorsInBlackList() + assert(tracker.executorBlacklist(scheduler, stage1, 1) === Set("1", "2")) + + clock.setTime(8000) + tracker.expireExecutorsInBlackList() + assert(tracker.executorBlacklist(scheduler, stage1, 1) === Set("2")) + + clock.setTime(10000) + tracker.expireExecutorsInBlackList() + assert(tracker.executorBlacklist(scheduler, stage1, 1) === Set()) + } + + test("blacklist feature is off by default") { + val conf = new SparkConf().setAppName("test").setMaster("local") + .set("spark.ui.enabled", "false") + val scheduler = mock[TaskSchedulerImpl] + + val tracker = new BlacklistTracker(conf, clock) + tracker.updateFailedExecutors(stage1, partition1, taskInfo_1_hostA, FAILURE) + tracker.updateFailedExecutors(stage1, partition1, taskInfo_2_hostA, FAILURE) + assert(tracker.executorBlacklist(scheduler, stage1, 1) === Set()) + assert(tracker.executorBlacklist(scheduler, stage1, 1) === Set()) + + tracker.updateFailedExecutors(stage1, partition3, taskInfo_3_hostB, FAILURE) + assert(tracker.executorBlacklist(scheduler, stage1, 3) === Set()) + assert(tracker.nodeBlacklist() === Set()) + } + + test("SingleTask strategy works") { + val conf = new SparkConf().setAppName("test").setMaster("local") + .set("spark.ui.enabled", "false") + .set("spark.scheduler.executorTaskBlacklistTime", "1000") + val scheduler = mock[TaskSchedulerImpl] + + // Task 1 failed on both executor 1 and executor 2 + val tracker = new BlacklistTracker(conf, clock) + tracker.updateFailedExecutors(stage1, partition1, taskInfo_1_hostA, FAILURE) + tracker.updateFailedExecutors(stage1, partition1, taskInfo_2_hostA, FAILURE) + assert(tracker.executorBlacklist(scheduler, stage1, 1) === Set("1", "2")) + assert(tracker.executorBlacklist(scheduler, stage1, 2) === Set()) + + // Task 1 succeeded on executor 1, so we remove executor 1 from blacklist + tracker.updateFailedExecutors(stage1, partition1, taskInfo_1_hostA, Success) + assert(tracker.executorBlacklist(scheduler, stage1, 1) === Set("2")) + assert(tracker.nodeBlacklist() === Set()) + + // Task 2 succeed on executor 3, no effect on blacklist for Task 1 + tracker.updateFailedExecutors(stage1, partition3, taskInfo_3_hostB, Success) + assert(tracker.executorBlacklist(scheduler, stage1, 1) === Set("2")) + + tracker.updateFailedExecutors(stage1, partition3, taskInfo_3_hostB, FAILURE) + assert(tracker.executorBlacklist(scheduler, stage1, 3) === Set("3")) + assert(tracker.nodeBlacklist() === Set()) + + tracker.updateFailedExecutors(stage1, partition1, taskInfo_2_hostA, Success) + assert(tracker.executorBlacklist(scheduler, stage1, 1) === Set()) + + // Task 2 on Stage 2 failed on Executor 2 + tracker.updateFailedExecutors(stage2, partition2, taskInfo_2_hostA, FAILURE) + assert(tracker.executorBlacklist(scheduler, stage1, 1) === Set()) + assert(tracker.executorBlacklist(scheduler, stage2, 1) === Set()) + assert(tracker.executorBlacklist(scheduler, stage1, 2) === Set()) + assert(tracker.executorBlacklist(scheduler, stage2, 2) === Set("2")) + } + + test("AdvencedSingleTask strategy works") { + val conf = new SparkConf().setAppName("test").setMaster("local") + .set("spark.ui.enabled", "false") + .set("spark.scheduler.blacklist.advancedStrategy", "true") + .set("spark.scheduler.executorTaskBlacklistTime", "1000") + val scheduler = mock[TaskSchedulerImpl] + when(scheduler.getExecutorsAliveOnHost("hostA")).thenReturn(Some(Set("1", "2", "4"))) + + // Task 1 failed on both executor 1 + val tracker = new BlacklistTracker(conf, clock) + tracker.updateFailedExecutors(stage1, partition1, taskInfo_1_hostA, FAILURE) + assert(tracker.executorBlacklist(scheduler, stage1, 1) === Set("1")) + assert(tracker.executorBlacklist(scheduler, stage1, 2) === Set()) + assert(tracker.nodeBlacklist() === Set()) + + // Task 1 failed on both executor 2 + tracker.updateFailedExecutors(stage1, partition1, taskInfo_2_hostA, FAILURE) + assert(tracker.executorBlacklist(scheduler, stage1, 1) === Set("1", "2", "4")) + assert(tracker.executorBlacklist(scheduler, stage1, 2) === Set("1", "2", "4")) + assert(tracker.executorBlacklist(scheduler, stage2, 1) === Set()) + assert(tracker.nodeBlacklistForStage(stage1) === Set("hostA")) + assert(tracker.nodeBlacklistForStage(stage2) === Set()) + assert(tracker.nodeBlacklist() === Set("hostA")) + + // Task 1 succeeded on executor 1, so we remove executor 1 from blacklist + tracker.updateFailedExecutors(stage1, partition1, taskInfo_1_hostA, Success) + assert(tracker.executorBlacklist(scheduler, stage1, 1) === Set("2")) + assert(tracker.nodeBlacklistForStage(stage1) === Set()) + assert(tracker.nodeBlacklist() === Set()) + } +} 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 36d1c5690f3c6..5b58241788d68 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -422,7 +422,11 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg // affinity to exec1 on host1 - which we will fail. val taskSet = FakeTask.createTaskSet(1, Seq(TaskLocation("host1", "exec1"))) val clock = new ManualClock + + // spy taskSetManager to set Manual clock for BlacklistTracker val manager = new TaskSetManager(sched, taskSet, 4, clock) + val tracker = new BlacklistTracker(conf, clock) + manager.setBlacklistTracker(tracker) { val offerResult = manager.resourceOffer("exec1", "host1", PROCESS_LOCAL) @@ -477,6 +481,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg // After reschedule delay, scheduling on exec1 should be possible. clock.advance(rescheduleDelay) + tracker.expireExecutorsInBlackList() { val offerResult = manager.resourceOffer("exec1", "host1", PROCESS_LOCAL) diff --git a/docs/configuration.md b/docs/configuration.md index 1e95b862441f5..1e293331a54e3 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1148,6 +1148,34 @@ Apart from these, the following properties are also available, and may be useful The interval length for the scheduler to revive the worker resource offers to run tasks. + + spark.scheduler.blacklist.enabled + true + + 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.scheduler.blacklist" configuration options. + + + + spark.scheduler.blacklist.timeout + 0s + + If executor blacklisting is enabled, this controls how long an executor remains in the blacklist before it is returned to the pool of available executors. + + + + spark.scheduler.blacklist.recoverPeriod + 60s + + If executor blacklisting is enabled, this controls how often to check if executors can be returned to the pool of active executors. + + + + spark.scheduler.blacklist.advancedStrategy + false + + set to tree to enable experimental advanced blacklist strategy. Comparing with the standard behavior before spark 1.6, it enables blacklist on node level. + + spark.speculation false diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index b6f45dd63473b..f2fa5b44cd638 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -682,11 +682,12 @@ private[spark] class ApplicationMaster( } override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { - case RequestExecutors(requestedTotal, localityAwareTasks, hostToLocalTaskCount) => + case RequestExecutors( + requestedTotal, localityAwareTasks, hostToLocalTaskCount, nodeBlacklist) => Option(allocator) match { case Some(a) => if (a.requestTotalExecutorsWithPreferredLocalities(requestedTotal, - localityAwareTasks, hostToLocalTaskCount)) { + localityAwareTasks, hostToLocalTaskCount, nodeBlacklist)) { resetAllocatorInterval() } context.reply(true) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala index dbdac3369b905..5f4a9e6403d5e 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -114,6 +114,8 @@ private[yarn] class YarnAllocator( @volatile private var targetNumExecutors = YarnSparkHadoopUtil.getInitialTargetExecutorNumber(sparkConf) + private var currentNodeBlacklist = Set.empty[String] + // Executor loss reason requests that are pending - maps from executor ID for inquiry to a // list of requesters that should be responded to once we find out why the given executor // was lost. @@ -217,18 +219,28 @@ private[yarn] class YarnAllocator( * @param localityAwareTasks number of locality aware tasks to be used as container placement hint * @param hostToLocalTaskCount a map of preferred hostname to possible task counts to be used as * container placement hint. + * @param nodeBlacklist a set of blacklisted node to avoid allocating new container on them. It + * will be used to update AM blacklist. * @return Whether the new requested total is different than the old value. */ def requestTotalExecutorsWithPreferredLocalities( requestedTotal: Int, localityAwareTasks: Int, - hostToLocalTaskCount: Map[String, Int]): Boolean = synchronized { + hostToLocalTaskCount: Map[String, Int], + nodeBlacklist: Set[String]): Boolean = synchronized { this.numLocalityAwareTasks = localityAwareTasks this.hostToLocalTaskCounts = hostToLocalTaskCount if (requestedTotal != targetNumExecutors) { logInfo(s"Driver requested a total number of $requestedTotal executor(s).") targetNumExecutors = requestedTotal + + // Update blacklist infomation to YARN ResouceManager for this application, + // in order to avoid allocating new Container on the problematic nodes. + val blacklistAdditions = nodeBlacklist -- currentNodeBlacklist + val blacklistRemovals = currentNodeBlacklist -- nodeBlacklist + amClient.updateBlacklist(blacklistAdditions.toList.asJava, blacklistRemovals.toList.asJava) + currentNodeBlacklist = nodeBlacklist true } else { false diff --git a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala index 6b3c831e60472..60fdb22f6a000 100644 --- a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala +++ b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala @@ -125,8 +125,12 @@ private[spark] abstract class YarnSchedulerBackend( * This includes executors already pending or running. */ override def doRequestTotalExecutors(requestedTotal: Int): Boolean = { + + val nodeBlacklist: Set[String] = scheduler.sc.blacklistTracker + .map(_.nodeBlacklist()).getOrElse(Set.empty[String]) + yarnSchedulerEndpointRef.askWithRetry[Boolean]( - RequestExecutors(requestedTotal, localityAwareTasks, hostToLocalTaskCount)) + RequestExecutors(requestedTotal, localityAwareTasks, hostToLocalTaskCount, nodeBlacklist)) } /** diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala index 207dbf56d3606..acd4d31014577 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala @@ -183,7 +183,7 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter handler.getNumExecutorsRunning should be (0) handler.getPendingAllocate.size should be (4) - handler.requestTotalExecutorsWithPreferredLocalities(3, 0, Map.empty) + handler.requestTotalExecutorsWithPreferredLocalities(3, 0, Map.empty, Set.empty) handler.updateResourceRequests() handler.getPendingAllocate.size should be (3) @@ -194,7 +194,7 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter handler.allocatedContainerToHostMap.get(container.getId).get should be ("host1") handler.allocatedHostToContainersMap.get("host1").get should contain (container.getId) - handler.requestTotalExecutorsWithPreferredLocalities(2, 0, Map.empty) + handler.requestTotalExecutorsWithPreferredLocalities(2, 0, Map.empty, Set.empty) handler.updateResourceRequests() handler.getPendingAllocate.size should be (1) } @@ -205,7 +205,7 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter handler.getNumExecutorsRunning should be (0) handler.getPendingAllocate.size should be (4) - handler.requestTotalExecutorsWithPreferredLocalities(3, 0, Map.empty) + handler.requestTotalExecutorsWithPreferredLocalities(3, 0, Map.empty, Set.empty) handler.updateResourceRequests() handler.getPendingAllocate.size should be (3) @@ -215,7 +215,7 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter handler.getNumExecutorsRunning should be (2) - handler.requestTotalExecutorsWithPreferredLocalities(1, 0, Map.empty) + handler.requestTotalExecutorsWithPreferredLocalities(1, 0, Map.empty, Set.empty) handler.updateResourceRequests() handler.getPendingAllocate.size should be (0) handler.getNumExecutorsRunning should be (2) @@ -231,7 +231,7 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter val container2 = createContainer("host2") handler.handleAllocatedContainers(Array(container1, container2)) - handler.requestTotalExecutorsWithPreferredLocalities(1, 0, Map.empty) + handler.requestTotalExecutorsWithPreferredLocalities(1, 0, Map.empty, Set.empty) handler.executorIdToContainer.keys.foreach { id => handler.killExecutor(id ) } val statuses = Seq(container1, container2).map { c => @@ -253,7 +253,7 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter val container2 = createContainer("host2") handler.handleAllocatedContainers(Array(container1, container2)) - handler.requestTotalExecutorsWithPreferredLocalities(2, 0, Map()) + handler.requestTotalExecutorsWithPreferredLocalities(2, 0, Map(), Set.empty) val statuses = Seq(container1, container2).map { c => ContainerStatus.newInstance(c.getId(), ContainerState.COMPLETE, "Failed", -1) From 5bfe94106f8c49f2c3d60d13bd2bb6389be94e4f Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 10 May 2016 12:49:05 -0500 Subject: [PATCH 02/57] Update for new design --- .../scala/org/apache/spark/SparkContext.scala | 15 - .../spark/scheduler/BlacklistStrategy.scala | 149 ------ .../spark/scheduler/BlacklistTracker.scala | 506 +++++++++++------- .../spark/scheduler/TaskSchedulerImpl.scala | 108 +++- .../spark/scheduler/TaskSetManager.scala | 87 +-- .../CoarseGrainedSchedulerBackend.scala | 3 - .../org/apache/spark/DistributedSuite.scala | 8 +- .../scheduler/BlacklistIntegrationSuite.scala | 47 +- .../scheduler/BlacklistTrackerSuite.scala | 318 +++++++---- .../org/apache/spark/scheduler/FakeTask.scala | 10 +- .../scheduler/SchedulerIntegrationSuite.scala | 28 +- .../scheduler/TaskSchedulerImplSuite.scala | 220 +++++++- .../spark/scheduler/TaskSetManagerSuite.scala | 31 +- docs/configuration.md | 20 +- .../cluster/YarnSchedulerBackend.scala | 3 +- 15 files changed, 952 insertions(+), 601 deletions(-) delete mode 100644 core/src/main/scala/org/apache/spark/scheduler/BlacklistStrategy.scala diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 6b98c25a11bbf..fe15052b62478 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -217,7 +217,6 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli private var _jars: Seq[String] = _ private var _files: Seq[String] = _ private var _shutdownHookRef: AnyRef = _ - private var _blacklistTracker: Option[BlacklistTracker] = _ /* ------------------------------------------------------------------------------------- * | Accessors and public fields. These provide access to the internal state of the | @@ -327,8 +326,6 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli private[spark] def cleaner: Option[ContextCleaner] = _cleaner - private[spark] def blacklistTracker: Option[BlacklistTracker] = _blacklistTracker - private[spark] var checkpointDir: Option[String] = None // Thread Local variable that can be used by users to pass information down the stack @@ -536,14 +533,6 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli } _executorAllocationManager.foreach(_.start()) - // By default blacklistTracker is enabled. - _blacklistTracker = if (_conf.getBoolean("spark.scheduler.blacklist.enabled", true)){ - Some(new BlacklistTracker(_conf)) - } else { - None - } - _blacklistTracker.foreach(_.start()) - _cleaner = if (_conf.getBoolean("spark.cleaner.referenceTracking", true)) { Some(new ContextCleaner(this)) @@ -1776,10 +1765,6 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli Utils.tryLogNonFatalError { _executorAllocationManager.foreach(_.stop()) } - Utils.tryLogNonFatalError { - _blacklistTracker.foreach(_.stop()) - } - if (_listenerBusStarted) { Utils.tryLogNonFatalError { listenerBus.stop() diff --git a/core/src/main/scala/org/apache/spark/scheduler/BlacklistStrategy.scala b/core/src/main/scala/org/apache/spark/scheduler/BlacklistStrategy.scala deleted file mode 100644 index a3e0772871b9f..0000000000000 --- a/core/src/main/scala/org/apache/spark/scheduler/BlacklistStrategy.scala +++ /dev/null @@ -1,149 +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 scala.collection.mutable - -import org.apache.spark.SparkConf -import org.apache.spark.util.Clock - -/** - * The interface to determine executor blacklist and node blacklist. - */ -private[scheduler] trait BlacklistStrategy { - /** Define a time interval to expire failure information of executors */ - val expireTimeInMilliseconds: Long - - /** Return executors in blacklist which are related to given stage and partition */ - def getExecutorBlacklist( - executorIdToFailureStatus: mutable.HashMap[String, FailureStatus], - atomTask: StageAndPartition, - clock: Clock): Set[String] - - /** Return all nodes in blacklist */ - def getNodeBlacklist( - executorIdToFailureStatus: mutable.HashMap[String, FailureStatus], - clock: Clock): Set[String] - - /** - * Return all nodes in blacklist for specified stage. By default it returns the same result as - * getNodeBlacklist. It could be override in strategy implementation. - */ - def getNodeBlacklistForStage( - executorIdToFailureStatus: mutable.HashMap[String, FailureStatus], - stageId: Int, - clock: Clock): Set[String] = getNodeBlacklist(executorIdToFailureStatus, clock) - - /** - * Choose which executors should be removed from blacklist. Return true if any executors are - * removed from the blacklist, false otherwise. The default implementation removes executors from - * the blacklist after [[expireTimeInMilliseconds]] - */ - def expireExecutorsInBlackList( - executorIdToFailureStatus: mutable.HashMap[String, FailureStatus], - clock: Clock): Boolean = { - - val now = clock.getTimeMillis() - val expiredKey = executorIdToFailureStatus.filter { - case (executorid, failureStatus) => { - (now - failureStatus.updatedTime) >= expireTimeInMilliseconds - } - }.keySet - - if (expiredKey.isEmpty) { - false - } else { - executorIdToFailureStatus --= expiredKey - true - } - } -} - -/** - * This strategy is applied to keep the same semantics as standard behavior before spark 1.6. - * - * If an executor failed running "task A", then we think this executor is blacked for "task A", - * but at the same time. it is still healthy for other task. Node blacklist is always empty. - */ -private[scheduler] class SingleTaskStrategy( - val expireTimeInMilliseconds: Long) extends BlacklistStrategy { - def getExecutorBlacklist( - executorIdToFailureStatus: mutable.HashMap[String, FailureStatus], - atomTask: StageAndPartition, - clock: Clock): Set[String] = { - executorIdToFailureStatus.filter{ - case (_, failureStatus) => failureStatus.numFailuresPerTask.keySet.contains(atomTask) && - clock.getTimeMillis() - failureStatus.updatedTime < expireTimeInMilliseconds - }.keys.toSet - } - - def getNodeBlacklist( - executorIdToFailureStatus: mutable.HashMap[String, FailureStatus], - clock: Clock): Set[String] = Set.empty[String] -} - -/** - * Comparing to SingleTaskStrategy, it supports node blacklist. With this strategy, once more than - * one executor failed running for specific stage, we put all executors on the same node into - * blacklist. So all tasks from the same stage will not be allocated to that node. - */ -private[scheduler] class AdvancedSingleTaskStrategy( - expireTimeInMilliseconds: Long) extends SingleTaskStrategy(expireTimeInMilliseconds) { - - override def getNodeBlacklistForStage( - executorIdToFailureStatus: mutable.HashMap[String, FailureStatus], - stageId: Int, - clock: Clock): Set[String] = { - val nodes = executorIdToFailureStatus.filter{ - case (_, failureStatus) => - failureStatus.numFailuresPerTask.keySet.map(_.stageId).contains(stageId) && - clock.getTimeMillis() - failureStatus.updatedTime < expireTimeInMilliseconds - }.values.map(_.host) - getDuplicateElem(nodes, 1) - } - - override def getNodeBlacklist( - executorIdToFailureStatus: mutable.HashMap[String, FailureStatus], - clock: Clock): Set[String] = { - // resolve a nodes sequence from failure status. - val nodes = executorIdToFailureStatus.values.map(_.host) - getDuplicateElem(nodes, 1) - } - - // A help function to find hosts which have more than "depTimes" executors on it in blacklist - private def getDuplicateElem(ndoes: Iterable[String], dupTimes: Int): Set[String] = { - ndoes.groupBy(identity).mapValues(_.size) // resolve map (nodeName => occurred times) - .filter(ele => ele._2 > dupTimes) // return nodes which occurred more than dupTimes. - .keys.toSet - } -} - -/** - * Create BlacklistStrategy instance according to SparkConf - */ -private[scheduler] object BlacklistStrategy { - def apply(sparkConf: SparkConf): BlacklistStrategy = { - val timeout = sparkConf.getTimeAsMs("spark.scheduler.blacklist.timeout", - sparkConf.getLong("spark.scheduler.executorTaskBlacklistTime", 0L).toString() + "ms") - - sparkConf.getBoolean("spark.scheduler.blacklist.advancedStrategy", false) match { - case false => new SingleTaskStrategy(timeout) - case true => new AdvancedSingleTaskStrategy(timeout) - } - } -} diff --git a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala index 57c16927a48fd..677cea3741b9e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala @@ -17,270 +17,362 @@ package org.apache.spark.scheduler -import java.util.concurrent.TimeUnit +import java.util.concurrent.atomic.AtomicReference -import scala.collection.mutable +import scala.collection.mutable.{HashMap, HashSet} import org.apache.spark.SparkConf -import org.apache.spark.Success -import org.apache.spark.TaskEndReason +import org.apache.spark.internal.Logging import org.apache.spark.util.Clock import org.apache.spark.util.SystemClock -import org.apache.spark.util.ThreadUtils import org.apache.spark.util.Utils + /** - * BlacklistTracker is designed to track problematic executors and nodes. It belongs to - * SparkContext as an centralized and unified collection for all tasks with same SparkContext. - * So that a new TaskSet could be benefit from previous experiences of other TaskSets. - * - * Once task finished, the callback method in TaskSetManager should update - * executorIdToFailureStatus Map. + * Abstract interface for tracking all info related to executor and node blacklist. Though we only + * have one real implmentation currently, [[BlacklistTrackerImpl]], usage becomes much simpler + * by adding [[NoopBlacklistTracker]] than by making all uses switch to Option[BlacklistTracker]. + * See [[BlacklistTrackerImpl]] for all meaningful documentation. */ -private[spark] class BlacklistTracker( - sparkConf: SparkConf, - clock: Clock = new SystemClock()) extends BlacklistCache{ +private[scheduler] trait BlacklistTracker { + def start(): Unit + + def stop(): Unit + + def expireExecutorsInBlacklist(): Unit + + /** Get the full blacklist. This *is* thread-safe, unlike all other methods. */ + def nodeBlacklist(): Set[String] + + def isNodeBlacklisted(node: String): Boolean + + def isNodeBlacklistedForStage(node: String, stageId: Int): Boolean + + def isExecutorBlacklisted(executorId: String): Boolean + + def isExecutorBlacklistedForStage(stageId: Int, executorId: String): Boolean + + def isExecutorBlacklisted( + executorId: String, + stageId: Int, + partition: Int) : Boolean - // maintain a ExecutorId --> FailureStatus HashMap - private val executorIdToFailureStatus: mutable.HashMap[String, FailureStatus] = mutable.HashMap() - // Apply Strategy pattern here to change different blacklist detection logic - private val strategy = BlacklistStrategy(sparkConf) + def taskSucceeded( + stageId: Int, + partition: Int, + info: TaskInfo): Unit + def taskFailed( + stageId: Int, + partition: Int, + info: TaskInfo): Unit - // A daemon thread to expire blacklist executor periodically - private val scheduler = ThreadUtils.newDaemonSingleThreadScheduledExecutor( - "spark-scheduler-blacklist-expire-timer") + def taskSetSucceeded(stageId: Int, scheduler: TaskSchedulerImpl): Unit - private val recoverPeriod = sparkConf.getTimeAsSeconds( - "spark.scheduler.blacklist.recoverPeriod", "60s") + def taskSetFailed(stageId: Int): Unit - def start(): Unit = { - val scheduleTask = new Runnable() { - override def run(): Unit = { - Utils.logUncaughtExceptions(expireExecutorsInBlackList()) + def removeExecutor(executorId: String): Unit +} + +/** + * BlacklistTracker is designed to track problematic executors and nodes. It supports blacklisting + * specific (executor, task) pairs within a stage, blacklisting entire executors and nodes for a + * stage, and blacklisting executors and nodes across an entire application (with a periodic + * expiry). + * + * The tracker needs to deal with a variety of workloads, eg.: bad user code, which may lead to many + * task failures, but that should not count against individual executors; many small stages, which + * may prevent a bad executor for having many failures within one stage, but still many failures + * over the entire application; "flaky" executors, that don't fail every task, but are still + * faulty; etc. + * + * 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 BlacklistTrackerImpl( + conf: SparkConf, + clock: Clock = new SystemClock()) extends BlacklistTracker with Logging { + + private val MAX_FAILURES_PER_EXEC = + conf.getInt("spark.blacklist.maxFailedTasksPerExecutor", 2) + private val MAX_FAILURES_PER_EXEC_STAGE = + conf.getInt("spark.blacklist.maxFailedTasksPerExecutorStage", 2) + private val MAX_FAILED_EXEC_PER_NODE = + conf.getInt("spark.blacklist.maxFailedExecutorsPerNode", 2) + private val MAX_FAILED_EXEC_PER_NODE_STAGE = + conf.getInt("spark.blacklist.maxFailedExecutorsPerNodeStage", 2) + val EXECUTOR_RECOVERY_MILLIS = BlacklistTracker.getBlacklistExpiryTime(conf) + + // a count of failed tasks for each executor. Only counts failures after tasksets complete + // successfully + private val executorIdToFailureCount: HashMap[String, Int] = HashMap() + // failures for each executor by stage. Only tracked while the stage is running. + val stageIdToExecToFailures: HashMap[Int, HashMap[String, FailureStatus]] = + new HashMap() + val stageIdToBlacklistedNodes: HashMap[Int, HashSet[String]] = new HashMap() + private val executorIdToBlacklistExpiryTime: HashMap[String, Long] = new HashMap() + private val nodeIdToBlacklistExpiryTime: HashMap[String, Long] = new HashMap() + private val _nodeBlacklist: AtomicReference[Set[String]] = new AtomicReference(Set()) + private var nextExpiryTime: Long = Long.MaxValue + + override def start(): Unit = {} + + override def stop(): Unit = {} + + override def expireExecutorsInBlacklist(): 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) { + val execsToClear = executorIdToBlacklistExpiryTime.filter(_._2 < now).keys + if (execsToClear.nonEmpty) { + logInfo(s"Removing executors $execsToClear from blacklist during periodic recovery") + execsToClear.foreach { exec => executorIdToBlacklistExpiryTime.remove(exec) } + } + if (executorIdToBlacklistExpiryTime.nonEmpty) { + nextExpiryTime = executorIdToBlacklistExpiryTime.map{_._2}.min + } else { + nextExpiryTime = Long.MaxValue + } + val nodesToClear = nodeIdToBlacklistExpiryTime.filter(_._2 < now).keys + if (nodesToClear.nonEmpty) { + logInfo(s"Removing nodes $nodesToClear from blacklist during periodic recovery") + nodesToClear.foreach { node => nodeIdToBlacklistExpiryTime.remove(node) } + // make a copy of the blacklisted nodes so nodeBlacklist() is threadsafe + _nodeBlacklist.set(nodeIdToBlacklistExpiryTime.keySet.toSet) } } - scheduler.scheduleAtFixedRate(scheduleTask, 0L, recoverPeriod, TimeUnit.SECONDS) + } + + override def taskSetSucceeded(stageId: Int, scheduler: TaskSchedulerImpl): Unit = { + // if any tasks failed, we count them towards the overall failure count for the executor at + // this point. Also clean out all data about the stage to avoid increasing memory use. + stageIdToExecToFailures.remove(stageId).map { failuresForStage => + failuresForStage.foreach { case (exec, newFailures) => + val prevFailures = executorIdToFailureCount.getOrElse(exec, 0) + val newTotal = prevFailures + newFailures.totalFailures + + if (newTotal >= MAX_FAILURES_PER_EXEC) { + logInfo(s"Blacklisting executor $exec because it has $newTotal" + + s" task failures in successful task sets") + val now = clock.getTimeMillis() + val expiryTime = now + EXECUTOR_RECOVERY_MILLIS + executorIdToBlacklistExpiryTime.put(exec, expiryTime) + executorIdToFailureCount.remove(exec) + if (expiryTime < nextExpiryTime) { + nextExpiryTime = expiryTime + } + + val node = scheduler.getHostForExecutor(exec) + val execs = scheduler.getExecutorsAliveOnHost(node).getOrElse(Set()) + val blacklistedExecs = execs.filter(executorIdToBlacklistExpiryTime.contains(_)) + if (blacklistedExecs.size >= MAX_FAILED_EXEC_PER_NODE) { + logInfo(s"Blacklisting node $node because it has ${blacklistedExecs.size} executors " + + s"blacklisted: ${blacklistedExecs}") + nodeIdToBlacklistExpiryTime.put(node, expiryTime) + // make a copy of the blacklisted nodes so nodeBlacklist() is threadsafe + _nodeBlacklist.set(nodeIdToBlacklistExpiryTime.keySet.toSet) + } + } else { + executorIdToFailureCount.put(exec, newTotal) + } + } + } + // when we blacklist a node within a stage, we don't directly promote that node to being + // blacklisted for the app. Instead, we use the mechanism above to decide whether or not to + // blacklist any executors for the app, and when doing so we'll check whether or not to also + // blacklist the node. That is why we just remove this entry without doing any promotion to + // the full app blacklist. + stageIdToBlacklistedNodes.remove(stageId) } - def stop(): Unit = { - scheduler.shutdown() - scheduler.awaitTermination(10, TimeUnit.SECONDS) + override def taskSetFailed(stageId: Int): Unit = { + // just throw away all the info for the failures in this taskSet -- assume the executors were + // fine, the failures were just b/c the taskSet itself was bad (eg., bad user code) + stageIdToExecToFailures.remove(stageId) + stageIdToBlacklistedNodes.remove(stageId) } - // The actual implementation is delegated to strategy - private[scheduler] def expireExecutorsInBlackList(): Unit = synchronized { - val updated = strategy.expireExecutorsInBlackList(executorIdToFailureStatus, clock) - if (updated) { - invalidateCache() - } + /** + * Return true if this executor is blacklisted for the given stage. Completely ignores whether + * the executor is blacklisted overall (or anything to do with the node the executor is on). + */ + override def isExecutorBlacklistedForStage( + stageId: Int, + executorId: String): Boolean = { + stageIdToExecToFailures.get(stageId).flatMap(_.get(executorId)) + .map(_.totalFailures >= MAX_FAILURES_PER_EXEC_STAGE).getOrElse(false) } - // The actual implementation is delegated to strategy - def executorBlacklist( - sched: TaskSchedulerImpl, stageId: Int, partition: Int): Set[String] = synchronized { - val atomTask = StageAndPartition(stageId, partition) - if (!isBlacklistExecutorCacheValid) { - reEvaluateExecutorBlacklistAndUpdateCache(sched, atomTask, clock) - } else { - getExecutorBlacklistFromCache(atomTask).getOrElse { - reEvaluateExecutorBlacklistAndUpdateCache(sched, atomTask, clock) - } - } + override def isExecutorBlacklisted(executorId: String): Boolean = { + executorIdToBlacklistExpiryTime.contains(executorId) } - // The actual implementation is delegated to strategy - def nodeBlacklist(): Set[String] = synchronized { - if (isBlacklistNodeCacheValid) { - getNodeBlacklistFromCache - } else { - val nodes = strategy.getNodeBlacklist(executorIdToFailureStatus, clock) - updateBlacklistNodeCache(nodes) - nodes - } + def isNodeBlacklistedForStage(node: String, stageId: Int): Boolean = { + stageIdToBlacklistedNodes.get(stageId).map(_.contains(node)).getOrElse(false) } - // The actual implementation is delegated to strategy - def nodeBlacklistForStage(stageId: Int): Set[String] = synchronized { - if (isBlacklistNodeForStageCacheValid) { - getNodeBlacklistForStageFromCache(stageId).getOrElse( - reEvaluateNodeBlacklistForStageAndUpdateCache(stageId)) - } else { - reEvaluateNodeBlacklistForStageAndUpdateCache(stageId) - } + override def nodeBlacklist(): Set[String] = { + _nodeBlacklist.get() } - def updateFailedExecutors( - stageId: Int, partition: Int, - info: TaskInfo, - reason: TaskEndReason) : Unit = synchronized { - - val atomTask = StageAndPartition(stageId, partition) - reason match { - // If the task succeeded, remove related record from executorIdToFailureStatus - case Success => - removeFailedExecutorsForTaskId(info.executorId, stageId, partition) - - // If the task failed, update latest failure time and failedTaskIds - case _ => - val executorId = info.executorId - executorIdToFailureStatus.get(executorId) match { - case Some(failureStatus) => - failureStatus.updatedTime = clock.getTimeMillis() - val failedTimes = failureStatus.numFailuresPerTask.getOrElse(atomTask, 0) + 1 - failureStatus.numFailuresPerTask(atomTask) = failedTimes - case None => - val failedTasks = mutable.HashMap(atomTask -> 1) - val failureStatus = new FailureStatus( - clock.getTimeMillis(), info.host, failedTasks) - executorIdToFailureStatus(executorId) = failureStatus - } - invalidateCache() - } + override def isNodeBlacklisted(node: String): Boolean = { + nodeIdToBlacklistExpiryTime.contains(node) } - /** remove the executorId from executorIdToFailureStatus */ - def removeFailedExecutors(executorId: String) : Unit = synchronized { - executorIdToFailureStatus.remove(executorId) - invalidateCache() + override def taskSucceeded( + stageId: Int, + partition: Int, + info: TaskInfo): Unit = { + // no-op intentionally, included just for symmetry. success to failure ratio is irrelevant, we + // just blacklist based on failures. Furthermore, one success does not override previous + // failures, since the bad node / executor may not fail *every* time } - /** - * remove the failure record related to given taskId from executorIdToFailureStatus. If the - * number of records of given executorId becomes 0, remove the completed executorId. - */ - def removeFailedExecutorsForTaskId( - executorId: String, + override def taskFailed( stageId: Int, - partition: Int) : Unit = synchronized { - val atomTask = StageAndPartition(stageId, partition) - executorIdToFailureStatus.get(executorId).map{ fs => - fs.numFailuresPerTask.remove(atomTask) - if (fs.numFailuresPerTask.isEmpty) { - executorIdToFailureStatus.remove(executorId) + partition: Int, + info: TaskInfo): Unit = { + val stageFailures = stageIdToExecToFailures.getOrElseUpdate(stageId, new HashMap()) + val failureStatus = stageFailures.getOrElseUpdate(info.executorId, new FailureStatus()) + failureStatus.totalFailures += 1 + failureStatus.failuresByPart += partition + if (failureStatus.totalFailures >= MAX_FAILURES_PER_EXEC_STAGE) { + // this executor has been pushed into the blacklist for this stage. Lets check if it pushes + // the whole node into the blacklist + val blacklistedExecutors = + stageFailures.filter{_._2.totalFailures >= MAX_FAILURES_PER_EXEC_STAGE} + if (blacklistedExecutors.size >= MAX_FAILED_EXEC_PER_NODE_STAGE) { + logInfo(s"Blacklisting ${info.host} for stage $stageId") + stageIdToBlacklistedNodes.getOrElseUpdate(stageId, new HashSet()) += info.host } - invalidateCache() } } - def isExecutorBlacklisted( + /** + * 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 + * altogether. + */ + override def isExecutorBlacklisted( executorId: String, - sched: TaskSchedulerImpl, stageId: Int, partition: Int) : Boolean = { - - executorBlacklist(sched, stageId, partition).contains(executorId) + // intentionally avoiding .getOrElse(..., new HashMap()) to avoid lots of object + // creation, since this method gets called a *lot* + stageIdToExecToFailures.get(stageId) match { + case Some(stageFailures) => + stageFailures.get(executorId) match { + case Some(failures) => + failures.failuresByPart.contains(partition) + case None => + false + } + case None => + false + } } - // If the node is in blacklist, all executors allocated on that node will - // also be put into executor blacklist. - private def executorsOnBlacklistedNode( - sched: TaskSchedulerImpl, - atomTask: StageAndPartition): Set[String] = { - nodeBlacklistForStage(atomTask.stageId).flatMap(sched.getExecutorsAliveOnHost(_) - .getOrElse(Set.empty[String])).toSet + override def removeExecutor(executorId: String): Unit = { + executorIdToBlacklistExpiryTime -= executorId + executorIdToFailureCount -= executorId + stageIdToExecToFailures.values.foreach { execFailureOneStage => + execFailureOneStage -= executorId + } } +} - private def reEvaluateExecutorBlacklistAndUpdateCache( - sched: TaskSchedulerImpl, - atomTask: StageAndPartition, - clock: Clock): Set[String] = { - val executors = executorsOnBlacklistedNode(sched, atomTask) ++ - strategy.getExecutorBlacklist(executorIdToFailureStatus, atomTask, clock) - updateBlacklistExecutorCache(atomTask, executors) - executors - } - private def reEvaluateNodeBlacklistForStageAndUpdateCache(stageId: Int): Set[String] = { - val nodes = strategy.getNodeBlacklistForStage(executorIdToFailureStatus, stageId, clock) - updateBlacklistNodeCache(nodes) - nodes - } -} +private[scheduler] object BlacklistTracker extends Logging { + val LEGACY_TIMEOUT_CONF = "spark.scheduler.executorTaskBlacklistTime" + val EXPIRY_TIMEOUT_CONF = "spark.scheduler.blacklist.recoverPeriod" + val ENABLED_CONF = "spark.scheduler.blacklist.enabled" -/** - * Hide cache details in this trait to make code clean and avoid operation mistake - */ -private[scheduler] trait BlacklistCache { - - // local cache to minimize the the work when query blacklisted executor and node - private val blacklistExecutorCache = mutable.HashMap.empty[StageAndPartition, Set[String]] - private val blacklistNodeCache = mutable.Set.empty[String] - private val blacklistNodeForStageCache = mutable.HashMap.empty[Int, Set[String]] - - // The flag to mark if cache is valid, it will be set to false when executorIdToFailureStatus be - // updated and it will be set to true, when called executorBlacklist and nodeBlacklist. - private var _isBlacklistExecutorCacheValid : Boolean = false - private var _isBlacklistNodeCacheValid: Boolean = false - private var _isBlacklistNodeForStageCacheValid: Boolean = false - - private val cacheLock = new Object() - - protected def isBlacklistExecutorCacheValid : Boolean = _isBlacklistExecutorCacheValid - protected def isBlacklistNodeCacheValid: Boolean = _isBlacklistNodeCacheValid - protected def isBlacklistNodeForStageCacheValid: Boolean = _isBlacklistNodeForStageCacheValid - - protected def updateBlacklistExecutorCache( - atomTask: StageAndPartition, - blacklistExecutor: Set[String]): Unit = cacheLock.synchronized { - if (!_isBlacklistExecutorCacheValid) { - blacklistExecutorCache.clear() + /** + * Return true if the blacklist is enabled, based on the following order of preferences: + * 1. Is it specifically enabled or disabled? + * 2. Is it enabled via the legacy timeout conf? + * 3. Use the default for the spark-master: + * - off for local mode + * - on for distributed modes (including local-cluster) + */ + def isBlacklistEnabled(conf: SparkConf): Boolean = { + val isEnabled = conf.get(ENABLED_CONF, null) + if (isEnabled == null) { + // if they've got a non-zero setting for the legacy conf, always enable the blacklist, + // otherwise, use the default based on the cluster-mode (off for local-mode, on otherwise). + val legacyTimeout = conf.getLong(LEGACY_TIMEOUT_CONF, 0L) + if (legacyTimeout > 0) { + // mostly this is necessary just for tests, since real users that want the blacklist will + // get it anyway by default + logWarning(s"Turning on blacklisting due to legacy configuration: $LEGACY_TIMEOUT_CONF > 0") + true + } else { + // local-cluster is *not* considered local for these purposes, we still want the blacklist + // enabled by default + !Utils.isLocalMaster(conf) + } + } else { + // always take whatever value is explicitly set by the user + isEnabled.toBoolean } - blacklistExecutorCache.update(atomTask, blacklistExecutor) - _isBlacklistExecutorCacheValid = true } - protected def updateBlacklistNodeCache( - blacklistNode: Set[String]): Unit = cacheLock.synchronized { - if (!_isBlacklistNodeCacheValid) { - blacklistNodeCache.clear() - } - blacklistNodeCache ++= blacklistNode - _isBlacklistNodeCacheValid = true + def getBlacklistExpiryTime(conf: SparkConf): Long = { + conf.getTimeAsMs(BlacklistTracker.EXPIRY_TIMEOUT_CONF, + conf.get(BlacklistTracker.LEGACY_TIMEOUT_CONF, (60 * 60 * 1000).toString)) } +} - protected def updateBlacklistNodeForStageCache( - stageId: Int, - blacklistNode: Set[String]): Unit = cacheLock.synchronized { - if (!_isBlacklistNodeForStageCacheValid) { - blacklistNodeForStageCache.clear() - } - blacklistNodeForStageCache.update(stageId, blacklistNode) - _isBlacklistNodeForStageCacheValid = true - } +/** Failures for one executor, within one stage */ +private[scheduler] final class FailureStatus { + val failuresByPart = HashSet[Int]() + var totalFailures = 0 - protected def invalidateCache(): Unit = cacheLock.synchronized { - _isBlacklistExecutorCacheValid = false - _isBlacklistNodeCacheValid = false - _isBlacklistNodeForStageCacheValid = false + override def toString(): String = { + s"totalFailures = $totalFailures; partitionsFailed = $failuresByPart" } +} + +/** Used to turn off blacklisting completely */ +private[scheduler] object NoopBlacklistTracker extends BlacklistTracker { + + override def start: Unit = {} - protected def getExecutorBlacklistFromCache( - atomTask: StageAndPartition): Option[Set[String]] = { - blacklistExecutorCache.get(atomTask) + override def stop: Unit = {} + + override def expireExecutorsInBlacklist(): Unit = {} + + override def taskSetSucceeded(stageId: Int, scheduler: TaskSchedulerImpl): Unit = {} + + override def taskSetFailed(stageId: Int): Unit = {} + + override def isExecutorBlacklistedForStage(stageId: Int, executorId: String): Boolean = { + false } - protected def getNodeBlacklistFromCache: Set[String] = blacklistNodeCache.toSet + override def isExecutorBlacklisted(executorId: String): Boolean = false - protected def getNodeBlacklistForStageFromCache(stageId: Int): Option[Set[String]] = - blacklistNodeForStageCache.get(stageId) -} + override def nodeBlacklist(): Set[String] = Set() -/** - * A class to record details of failure. - * - * @param initialTime the time when failure status be created - * @param host the node name which running executor on - * @param numFailuresPerTask all tasks failed on the executor (key is StageAndPartition, value - * is the number of failures of this task) - */ -private[scheduler] final class FailureStatus( - initialTime: Long, - val host: String, - val numFailuresPerTask: mutable.HashMap[StageAndPartition, Int]) { + override def isNodeBlacklisted(node: String): Boolean = false - var updatedTime = initialTime - def totalNumFailures : Int = numFailuresPerTask.values.sum -} + override def isNodeBlacklistedForStage(node: String, stageId: Int): Boolean = false + + override def isExecutorBlacklisted( + executorId: String, + stageId: Int, + partition: Int) : Boolean = false + + override def taskSucceeded( + stageId: Int, + partition: Int, + info: TaskInfo): Unit = {} -private[scheduler] case class StageAndPartition(val stageId: Int, val partition: Int) + override def taskFailed( + stageId: Int, + partition: Int, + info: TaskInfo): Unit = {} + + override def removeExecutor(executorId: String): Unit = {} +} 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 2ce49ca1345f2..ec1292fe31d9f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -22,9 +22,8 @@ import java.util.{Timer, TimerTask} import java.util.concurrent.TimeUnit import java.util.concurrent.atomic.AtomicLong -import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.HashMap -import scala.collection.mutable.HashSet +import scala.collection.Set +import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import scala.language.postfixOps import scala.util.Random @@ -35,7 +34,7 @@ import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.scheduler.TaskLocality.TaskLocality import org.apache.spark.scheduler.local.LocalSchedulerBackend import org.apache.spark.storage.BlockManagerId -import org.apache.spark.util.{AccumulatorV2, ThreadUtils, Utils} +import org.apache.spark.util.{AccumulatorV2, Clock, SystemClock, ThreadUtils, Utils} /** * Schedules tasks for multiple types of clusters by acting through a SchedulerBackend. @@ -52,13 +51,23 @@ import org.apache.spark.util.{AccumulatorV2, ThreadUtils, Utils} * acquire a lock on us, so we need to make sure that we don't try to lock the backend while * we are holding a lock on ourselves. */ -private[spark] class TaskSchedulerImpl( +private[spark] class TaskSchedulerImpl private[scheduler]( val sc: SparkContext, val maxTaskFailures: Int, + private[scheduler] val blacklistTracker: BlacklistTracker, + private val clock: Clock = new SystemClock, isLocal: Boolean = false) extends TaskScheduler with Logging { - def this(sc: SparkContext) = this(sc, sc.conf.getInt("spark.task.maxFailures", 4)) + def this(sc: SparkContext) = { + this(sc, sc.conf.getInt("spark.task.maxFailures", 4), + TaskSchedulerImpl.createBlacklistTracker(sc.conf)) + } + + def this(sc: SparkContext, maxTaskFailures: Int, isLocal: Boolean) = { + this(sc, maxTaskFailures, TaskSchedulerImpl.createBlacklistTracker(sc.conf), + clock = new SystemClock, isLocal) + } val conf = sc.conf @@ -153,6 +162,7 @@ private[spark] class TaskSchedulerImpl( override def start() { backend.start() + blacklistTracker.start() if (!isLocal && conf.getBoolean("spark.speculation", false)) { logInfo("Starting speculative execution thread") @@ -208,7 +218,7 @@ private[spark] class TaskSchedulerImpl( private[scheduler] def createTaskSetManager( taskSet: TaskSet, maxTaskFailures: Int): TaskSetManager = { - new TaskSetManager(this, taskSet, maxTaskFailures) + new TaskSetManager(this, taskSet, maxTaskFailures, clock) } override def cancelTasks(stageId: Int, interruptThread: Boolean): Unit = synchronized { @@ -236,7 +246,7 @@ private[spark] class TaskSchedulerImpl( * given TaskSetManager have completed, so state associated with the TaskSetManager should be * cleaned up. */ - def taskSetFinished(manager: TaskSetManager): Unit = synchronized { + def taskSetFinished(manager: TaskSetManager, success: Boolean): Unit = synchronized { taskSetsByStageIdAndAttempt.get(manager.taskSet.stageId).foreach { taskSetsForStage => taskSetsForStage -= manager.taskSet.stageAttemptId if (taskSetsForStage.isEmpty) { @@ -244,21 +254,33 @@ private[spark] class TaskSchedulerImpl( } } manager.parent.removeSchedulable(manager) - logInfo("Removed TaskSet %s, whose tasks have all completed, from pool %s" - .format(manager.taskSet.id, manager.parent.name)) + if (success) { + blacklistTracker.taskSetSucceeded(manager.taskSet.stageId, this) + logInfo(s"Removed TaskSet ${manager.taskSet.id}, whose tasks have all completed, from pool" + + s" ${manager.parent.name}") + } else { + blacklistTracker.taskSetFailed(manager.taskSet.stageId) + logInfo(s"Removed TaskSet ${manager.taskSet.id}, since it failed, from pool" + + s" ${manager.parent.name}") + } } private def resourceOfferSingleTaskSet( taskSet: TaskSetManager, maxLocality: TaskLocality, - shuffledOffers: Seq[WorkerOffer], + shuffledOffers: IndexedSeq[WorkerOffer], availableCpus: Array[Int], tasks: Seq[ArrayBuffer[TaskDescription]]) : Boolean = { var launchedTask = false + // nodes and executors that are blacklisted for the entire application have already been + // filtered out by this point for (i <- 0 until shuffledOffers.size) { - val execId = shuffledOffers(i).executorId - val host = shuffledOffers(i).host - if (availableCpus(i) >= CPUS_PER_TASK) { + val offer = shuffledOffers(i) + val host = offer.host + val nodeBlacklisted = blacklistTracker.isNodeBlacklistedForStage(host, taskSet.stageId) + val execId = offer.executorId + val execBlacklisted = blacklistTracker.isExecutorBlacklistedForStage(taskSet.stageId, execId) + if (!nodeBlacklisted && !execBlacklisted && availableCpus(i) >= CPUS_PER_TASK) { try { for (task <- taskSet.resourceOffer(execId, host, maxLocality)) { tasks(i) += task @@ -280,11 +302,21 @@ private[spark] class TaskSchedulerImpl( } } if (!launchedTask) { - taskSet.abortIfCompletelyBlacklisted(executorIdToHost.keys) + taskSet.abortIfCompletelyBlacklisted(executorsByHost, blacklistTracker) } return launchedTask } + private[scheduler] def areAllExecutorsBlacklisted(): Boolean = { + executorsByHost.foreach { case (host, execs) => + if (!blacklistTracker.isNodeBlacklisted(host) && + execs.exists(!blacklistTracker.isExecutorBlacklisted(_))) { + return false + } + } + true + } + /** * Called by cluster manager to offer resources on slaves. We respond by asking our active task * sets for tasks in order of priority. We fill each node with tasks in a round-robin manner so @@ -310,12 +342,38 @@ private[spark] class TaskSchedulerImpl( } } - // Randomly shuffle offers to avoid always placing tasks on the same set of workers. - val shuffledOffers = Random.shuffle(offers) + // ensure that we periodically check if executors can be removed from the blacklist, without + // requiring a separate thread and added synchronization overhead + blacklistTracker.expireExecutorsInBlacklist() + + val sortedTaskSets = rootPool.getSortedTaskSetQueue + val filteredOffers: IndexedSeq[WorkerOffer] = offers.filter { offer => + !blacklistTracker.isNodeBlacklisted(offer.host) && + !blacklistTracker.isExecutorBlacklisted(offer.executorId) + } match { + // toIndexedSeq always makes an *immutable* IndexedSeq, though we don't care if its mutable + // or immutable. So we do this to avoid making a pointless copy + case is: IndexedSeq[WorkerOffer] => is + case other: Seq[WorkerOffer] => other.toIndexedSeq + } + if (offers.nonEmpty && filteredOffers.isEmpty) { + // Its possible that all the executors are now blacklisted, though we haven't aborted stages + // during the check in resourceOfferSingleTaskSet. If so, fail all existing task sets to + // avoid unschedulability. + if (areAllExecutorsBlacklisted()) { + sortedTaskSets.foreach { tsm => + tsm.abort(s"All executors are blacklisted, so aborting ${tsm.taskSet}") + } + } + return Seq() + } + + // Randomly shuffle offers to avoid always placing tasks on the same set of workers. We will + // index into this list by position later, so we want an IndexedSeq so its efficient. + val shuffledOffers: IndexedSeq[WorkerOffer] = Random.shuffle(filteredOffers).toIndexedSeq // Build a list of tasks to assign to each worker. val tasks = shuffledOffers.map(o => new ArrayBuffer[TaskDescription](o.cores)) val availableCpus = shuffledOffers.map(o => o.cores).toArray - val sortedTaskSets = rootPool.getSortedTaskSetQueue for (taskSet <- sortedTaskSets) { logDebug("parentName: %s, name: %s, runningTasks: %s".format( taskSet.parent.name, taskSet.name, taskSet.runningTasks)) @@ -459,6 +517,7 @@ private[spark] class TaskSchedulerImpl( override def stop() { speculationScheduler.shutdown() + blacklistTracker.stop() if (backend != null) { backend.stop() } @@ -552,12 +611,17 @@ private[spark] class TaskSchedulerImpl( executorIdToHost -= executorId rootPool.executorLost(executorId, host, reason) } + blacklistTracker.removeExecutor(executorId) } def executorAdded(execId: String, host: String) { dagScheduler.executorAdded(execId, host) } + def getHostForExecutor(execId: String): String = synchronized { + executorIdToHost(execId) + } + def getExecutorsAliveOnHost(host: String): Option[Set[String]] = synchronized { executorsByHost.get(host).map(_.toSet) } @@ -657,4 +721,12 @@ private[spark] object TaskSchedulerImpl { retval.toList } + private def createBlacklistTracker(conf: SparkConf): BlacklistTracker = { + if (BlacklistTracker.isBlacklistEnabled(conf)) { + new BlacklistTrackerImpl(conf) + } else { + NoopBlacklistTracker + } + } + } 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 a906856d8e29c..1c1c1836f16b1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -22,9 +22,7 @@ import java.nio.ByteBuffer import java.util.Arrays import java.util.concurrent.ConcurrentLinkedQueue -import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.HashMap -import scala.collection.mutable.HashSet +import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import scala.math.{max, min} import scala.util.control.NonFatal @@ -50,12 +48,21 @@ import org.apache.spark.util.{AccumulatorV2, Clock, SystemClock, Utils} * task set will be aborted */ private[spark] class TaskSetManager( - sched: TaskSchedulerImpl, + val sched: TaskSchedulerImpl, + val blacklistTracker: BlacklistTracker, val taskSet: TaskSet, val maxTaskFailures: Int, - clock: Clock = new SystemClock()) + val clock: Clock) extends Schedulable with Logging { + def this( + sched: TaskSchedulerImpl, + taskSet: TaskSet, + maxTaskFailures: Int, + clock: Clock = new SystemClock()) { + this(sched, sched.blacklistTracker, taskSet, maxTaskFailures, clock) + } + val conf = sched.sc.conf // Quantile of tasks at which to start speculation @@ -239,8 +246,7 @@ private[spark] class TaskSetManager( while (indexOffset > 0) { indexOffset -= 1 val index = list(indexOffset) - if (!blacklistTracker.map(_.isExecutorBlacklisted(execId, sched, stageId, index)) - .getOrElse(false)) { + if (!blacklistTracker.isExecutorBlacklisted(execId, stageId, index)) { // This should almost always be list.trimEnd(1) to remove tail list.remove(indexOffset) if (copiesRunning(index) == 0 && !successful(index)) { @@ -256,13 +262,6 @@ private[spark] class TaskSetManager( taskAttempts(taskIndex).exists(_.host == host) } - var blacklistTracker = sched.sc.blacklistTracker - - /** VisibleForTesting */ - private[scheduler] def setBlacklistTracker (tracker: BlacklistTracker) = { - blacklistTracker = Some(tracker) - } - /** * Return a speculative task for a given executor if any are available. The task should not have * an attempt running on this host, in case the host is slow. In addition, the task should meet @@ -276,8 +275,7 @@ private[spark] class TaskSetManager( def canRunOnHost(index: Int): Boolean = !hasAttemptOnHost(index, host) && - !blacklistTracker.map(_.isExecutorBlacklisted(execId, sched, stageId, index)) - .getOrElse(false) + !blacklistTracker.isExecutorBlacklisted(execId, stageId, index) if (!speculatableTasks.isEmpty) { // Check for process-local tasks; note that tasks can be process-local @@ -462,8 +460,8 @@ private[spark] class TaskSetManager( // a good proxy to task serialization time. // val timeTaken = clock.getTime() - startTime val taskName = s"task ${info.id} in stage ${taskSet.id}" - logInfo(s"Starting $taskName (TID $taskId, $host, partition ${task.partitionId}," + - s" $taskLocality, ${serializedTask.limit} bytes)") + logInfo(s"Starting $taskName (TID $taskId, $host, exec ${info.executorId}, " + + s"partition ${task.partitionId},$taskLocality, ${serializedTask.limit} bytes)") sched.dagScheduler.taskStarted(task, info) return Some(new TaskDescription(taskId = taskId, attemptNumber = attemptNum, execId, @@ -476,7 +474,8 @@ private[spark] class TaskSetManager( private def maybeFinishTaskSet() { if (isZombie && runningTasks == 0) { - sched.taskSetFinished(this) + val success = tasksSuccessful == numTasks + sched.taskSetFinished(this, success) } } @@ -576,7 +575,9 @@ private[spark] class TaskSetManager( * failures (this is because the method picks on unscheduled task, and then iterates through each * executor until it finds one that the task hasn't failed on already). */ - private[scheduler] def abortIfCompletelyBlacklisted(executors: Iterable[String]): Unit = { + private[scheduler] def abortIfCompletelyBlacklisted( + executorsByHost: HashMap[String, HashSet[String]], + blacklist: BlacklistTracker): Unit = { val pendingTask: Option[Int] = { // usually this will just take the last pending task, but because of the lazy removal @@ -595,16 +596,32 @@ private[spark] class TaskSetManager( // If no executors have registered yet, don't abort the stage, just wait. We probably // got here because a task set was added before the executors registered. - if (executors.nonEmpty) { + if (executorsByHost.nonEmpty) { // take any task that needs to be scheduled, and see if we can find some executor it *could* // run on pendingTask.foreach { taskId => - if (executors.forall(executorIsBlacklisted(_, taskId))) { - val execs = executors.toIndexedSeq.sorted.mkString("(", ",", ")") - val partition = tasks(taskId).partitionId - abort(s"Aborting ${taskSet} because task $taskId (partition $partition)" + - s" has already failed on executors $execs, and no other executors are available.") + val stage = taskSet.stageId + val part = tasks(taskId).partitionId + executorsByHost.foreach { case (host, execs) => + if (!blacklistTracker.isNodeBlacklisted(host) && + !blacklistTracker.isNodeBlacklistedForStage(host, stage)) { + execs.foreach { exec => + if ( + !blacklistTracker.isExecutorBlacklisted(exec) && + !blacklistTracker.isExecutorBlacklistedForStage(stage, exec) && + !blacklistTracker.isExecutorBlacklisted(exec, stageId = stage, partition = part) + ) { + // we've found some executor this task can run on. Its possible that some *other* + // task isn't schedulable anywhere, but we will discover that in some later call, + // when that unschedulable task is the last task remaining. + return + } + } + } } + val partition = tasks(taskId).partitionId + abort(s"Aborting ${taskSet} because task $taskId (partition $partition) cannot run " + + s"anywhere due to node and executor blacklist.") } } } @@ -661,8 +678,9 @@ private[spark] class TaskSetManager( } if (!successful(index)) { tasksSuccessful += 1 - logInfo("Finished task %s in stage %s (TID %d) in %d ms on %s (%d/%d)".format( - info.id, taskSet.id, info.taskId, info.duration, info.host, tasksSuccessful, numTasks)) + logInfo("Finished task %s in stage %s (TID %d) in %d ms on %s / exec %s (%d/%d)".format( + info.id, taskSet.id, info.taskId, info.duration, info.host, info.executorId, + tasksSuccessful, numTasks)) // Mark successful and stop if all the tasks have succeeded. successful(index) = true if (tasksSuccessful == numTasks) { @@ -673,9 +691,7 @@ private[spark] class TaskSetManager( " because task " + index + " has already completed successfully") } - blacklistTracker.foreach{ - _.updateFailedExecutors(stageId, tasks(index).partitionId, info, Success) - } + blacklistTracker.taskSucceeded(stageId, tasks(index).partitionId, info) maybeFinishTaskSet() } @@ -693,8 +709,8 @@ private[spark] class TaskSetManager( val index = info.index copiesRunning(index) -= 1 var accumUpdates: Seq[AccumulatorV2[_, _]] = Seq.empty - val failureReason = s"Lost task ${info.id} in stage ${taskSet.id} (TID $tid, ${info.host}): " + - reason.asInstanceOf[TaskFailedReason].toErrorString + val failureReason = s"Lost task ${info.id} in stage ${taskSet.id} (TID $tid, ${info.host}," + + s" exec ${info.executorId}): ${reason.asInstanceOf[TaskFailedReason].toErrorString}" val failureException: Option[Throwable] = reason match { case fetchFailed: FetchFailed => logWarning(failureReason) @@ -759,9 +775,8 @@ private[spark] class TaskSetManager( } // always add to failed executors - blacklistTracker.foreach { - _.updateFailedExecutors(stageId, tasks(index).partitionId, info, reason) - } + // TODO if there is a fetch failure, does it really make sense to add this? + blacklistTracker.taskFailed(stageId, tasks(index).partitionId, info) sched.dagScheduler.taskEnded(tasks(index), reason, null, accumUpdates, info) 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 eb2c45c1fdbf3..8259923ce31c3 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 @@ -296,9 +296,6 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp scheduler.sc.env.blockManager.master.removeExecutorAsync(executorId) logInfo(s"Asked to remove non-existent executor $executorId") } - - // Remove disconnected executor from blacklistTracker to keep consistency - scheduler.sc.blacklistTracker.foreach(_.removeFailedExecutors(executorId)) } /** diff --git a/core/src/test/scala/org/apache/spark/DistributedSuite.scala b/core/src/test/scala/org/apache/spark/DistributedSuite.scala index 0515e6e3a6319..6193019dd9c2d 100644 --- a/core/src/test/scala/org/apache/spark/DistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/DistributedSuite.scala @@ -108,7 +108,9 @@ class DistributedSuite extends SparkFunSuite with Matchers with LocalSparkContex } test("repeatedly failing task") { - sc = new SparkContext(clusterUrl, "test") + val conf = new SparkConf().setAppName("test").setMaster(clusterUrl) + .set("spark.scheduler.blacklist.enabled", "false") + sc = new SparkContext(conf) val thrown = intercept[SparkException] { // scalastyle:off println sc.parallelize(1 to 10, 10).foreach(x => println(x / 0)) @@ -269,7 +271,9 @@ class DistributedSuite extends SparkFunSuite with Matchers with LocalSparkContex test("recover from repeated node failures during shuffle-reduce") { import DistributedSuite.{markNodeIfIdentity, failOnMarkedIdentity} DistributedSuite.amMaster = true - sc = new SparkContext(clusterUrl, "test") + val conf = new SparkConf().setAppName("test").setMaster(clusterUrl) + .set("spark.scheduler.blacklist.enabled", "false") + sc = new SparkContext(conf) for (i <- 1 to 3) { val data = sc.parallelize(Seq(true, true), 2) assert(data.count === 2) diff --git a/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala index 14c8b664d4d8b..09fcd2b3560ce 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala @@ -42,7 +42,10 @@ 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 blacklist job will fail", + extraConfs = Seq( + "spark.scheduler.blacklist.enabled" -> "false" + )) { val rdd = new MockRDDWithLocalityPrefs(sc, 10, Nil, badHost) withBackend(badHostBackend _) { val jobFuture = submit(rdd, (0 until 10).toArray) @@ -51,37 +54,48 @@ class BlacklistIntegrationSuite extends SchedulerIntegrationSuite[MultiExecutorM assertDataStructuresEmpty(noFailure = false) } - // even with the blacklist turned on, if maxTaskFailures is not more than the number - // of executors on the bad node, then locality preferences will lead to us cycling through - // the executors on the bad node, and still failing the job + // even with the blacklist turned on, bad configs can lead to job failure. To survive one + // bad node, you need to make sure that + // maxTaskFailures > spark.blacklist.maxFailedTasksPerExecutorStage * + // spark.blacklist.maxFailedExecutorsPerNodeStage testScheduler( "With blacklist on, job will still fail if there are too many bad executors on bad host", extraConfs = Seq( + "spark.scheduler.blacklist.enabled" -> "true", // set this to something much longer than the test duration so that executors don't get // removed from the blacklist during the test - ("spark.scheduler.executorTaskBlacklistTime", "10000000") + "spark.scheduler.executorTaskBlacklistTime" -> "10000000", + "spark.testing.nHosts" -> "2", + "spark.testing.nExecutorsPerHost" -> "5", + "spark.testing.nCoresPerExecutor" -> "10", + "spark.task.maxFailures" -> "4", + "spark.blacklist.maxFailedTasksPerExecutorStage" -> "1", + "spark.blacklist.maxFailedExecutorsPerNodeStage" -> "5" ) ) { - val rdd = new MockRDDWithLocalityPrefs(sc, 10, Nil, badHost) + // to reliably reproduce the failure, 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 -- we never try scheduling tasks on the good host-1, since 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 10).toArray) + val jobFuture = submit(rdd, (0 until 1).toArray) awaitJobTermination(jobFuture, duration) } assertDataStructuresEmpty(noFailure = false) } - // Here we run with the blacklist on, and maxTaskFailures high enough that we'll eventually - // schedule on a good node and succeed the job + // Here we run with the blacklist 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( - // set this to something much longer than the test duration so that executors don't get - // removed from the blacklist during the test - ("spark.scheduler.executorTaskBlacklistTime", "10000000"), - // this has to be higher than the number of executors on the bad host - ("spark.task.maxFailures", "5"), + "spark.scheduler.blacklist.enabled" -> "true", + // just set this to something much longer than the test duration + "spark.scheduler.executorTaskBlacklistTime" -> "10000000", // just to avoid this test taking too long - ("spark.locality.wait", "10ms") + "spark.locality.wait" -> "10ms" ) ) { val rdd = new MockRDDWithLocalityPrefs(sc, 10, Nil, badHost) @@ -98,6 +112,7 @@ class BlacklistIntegrationSuite extends SchedulerIntegrationSuite[MultiExecutorM testScheduler( "SPARK-15865 Progress with fewer executors than maxTaskFailures", extraConfs = Seq( + "spark.scheduler.blacklist.enabled" -> "true", // set this to something much longer than the test duration so that executors don't get // removed from the blacklist during the test "spark.scheduler.executorTaskBlacklistTime" -> "10000000", @@ -114,7 +129,7 @@ class BlacklistIntegrationSuite extends SchedulerIntegrationSuite[MultiExecutorM val jobFuture = submit(new MockRDD(sc, 10, Nil), (0 until 10).toArray) Await.ready(jobFuture, duration) val pattern = ("Aborting TaskSet 0.0 because task .* " + - "already failed on executors \\(.*\\), and no other executors are available").r + "cannot run anywhere due to node and executor blacklist").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/BlacklistTrackerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala index 5d090d8f2b512..42ec9d4fcc9dc 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala @@ -18,24 +18,14 @@ package org.apache.spark.scheduler import org.mockito.Mockito.when -import org.scalatest.BeforeAndAfter +import org.scalatest.BeforeAndAfterEach import org.scalatest.mock.MockitoSugar -import org.apache.spark.ExceptionFailure import org.apache.spark.SparkConf import org.apache.spark.SparkFunSuite -import org.apache.spark.Success -import org.apache.spark.TaskEndReason import org.apache.spark.util.ManualClock -class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfter with MockitoSugar { - - val FAILURE: TaskEndReason = new ExceptionFailure( - "Fake", - "fake failure", - Array.empty[StackTraceElement], - "fake stack trace", - None) +class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with MockitoSugar { val stage1 = 1 val stage2 = 2 @@ -54,124 +44,244 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfter with Mocki val clock = new ManualClock(0) - test ("expireExecutorsInBlacklist works") { - // expire time is set to 6s - val conf = new SparkConf().setAppName("test").setMaster("local") - .set("spark.ui.enabled", "false") - .set("spark.scheduler.executorTaskBlacklistTime", "6000") - - val scheduler = mock[TaskSchedulerImpl] + var blacklistTracker: BlacklistTrackerImpl = _ - val tracker = new BlacklistTracker(conf, clock) - // Executor 1 into blacklist at Time 00:00:00 - tracker.updateFailedExecutors(stage1, partition1, taskInfo_1_hostA, FAILURE) - assert(tracker.executorBlacklist(scheduler, stage1, 1) === Set("1")) - - clock.setTime(2000) - tracker.expireExecutorsInBlackList() - assert(tracker.executorBlacklist(scheduler, stage1, 1) === Set("1")) - // Executor 1 failed again at Time 00::00:02 - tracker.updateFailedExecutors(stage1, partition1, taskInfo_1_hostA, FAILURE) - - clock.setTime(3000) - // Executor 2 failed at Time 00:00:03 - tracker.updateFailedExecutors(stage1, partition1, taskInfo_2_hostA, FAILURE) - assert(tracker.executorBlacklist(scheduler, stage1, 1) === Set("1", "2")) - - clock.setTime(6000) - tracker.expireExecutorsInBlackList() - assert(tracker.executorBlacklist(scheduler, stage1, 1) === Set("1", "2")) - - clock.setTime(8000) - tracker.expireExecutorsInBlackList() - assert(tracker.executorBlacklist(scheduler, stage1, 1) === Set("2")) - - clock.setTime(10000) - tracker.expireExecutorsInBlackList() - assert(tracker.executorBlacklist(scheduler, stage1, 1) === Set()) + override def afterEach(): Unit = { + if (blacklistTracker != null) { + blacklistTracker.stop() + blacklistTracker = null + } + super.afterEach() } - test("blacklist feature is off by default") { - val conf = new SparkConf().setAppName("test").setMaster("local") - .set("spark.ui.enabled", "false") - val scheduler = mock[TaskSchedulerImpl] - - val tracker = new BlacklistTracker(conf, clock) - tracker.updateFailedExecutors(stage1, partition1, taskInfo_1_hostA, FAILURE) - tracker.updateFailedExecutors(stage1, partition1, taskInfo_2_hostA, FAILURE) - assert(tracker.executorBlacklist(scheduler, stage1, 1) === Set()) - assert(tracker.executorBlacklist(scheduler, stage1, 1) === Set()) + val allOptions = (('A' to 'Z').map("host" + _) ++ (1 to 100).map{_.toString}).toSet - tracker.updateFailedExecutors(stage1, partition3, taskInfo_3_hostB, FAILURE) - assert(tracker.executorBlacklist(scheduler, stage1, 3) === Set()) - assert(tracker.nodeBlacklist() === Set()) + /** + * 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 (for thread-safety), 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 = { + allOptions.foreach { opt => + val actual = f(opt) + val exp = expected.contains(opt) + assert(actual === exp, raw"""for string "$opt" """) + } } - test("SingleTask strategy works") { + test("Blacklisting individual tasks") { val conf = new SparkConf().setAppName("test").setMaster("local") .set("spark.ui.enabled", "false") + .set("spark.scheduler.blacklist.advancedStrategy", "true") .set("spark.scheduler.executorTaskBlacklistTime", "1000") val scheduler = mock[TaskSchedulerImpl] + when(scheduler.getExecutorsAliveOnHost("hostA")).thenReturn(Some(Set("1", "2", "4"))) + Set("1", "2", "4").foreach { execId => + when(scheduler.getHostForExecutor(execId)).thenReturn("hostA") + } - // Task 1 failed on both executor 1 and executor 2 - val tracker = new BlacklistTracker(conf, clock) - tracker.updateFailedExecutors(stage1, partition1, taskInfo_1_hostA, FAILURE) - tracker.updateFailedExecutors(stage1, partition1, taskInfo_2_hostA, FAILURE) - assert(tracker.executorBlacklist(scheduler, stage1, 1) === Set("1", "2")) - assert(tracker.executorBlacklist(scheduler, stage1, 2) === Set()) - - // Task 1 succeeded on executor 1, so we remove executor 1 from blacklist - tracker.updateFailedExecutors(stage1, partition1, taskInfo_1_hostA, Success) - assert(tracker.executorBlacklist(scheduler, stage1, 1) === Set("2")) - assert(tracker.nodeBlacklist() === Set()) - - // Task 2 succeed on executor 3, no effect on blacklist for Task 1 - tracker.updateFailedExecutors(stage1, partition3, taskInfo_3_hostB, Success) - assert(tracker.executorBlacklist(scheduler, stage1, 1) === Set("2")) + // Task 1 failed on executor 1 + blacklistTracker = new BlacklistTrackerImpl(conf, clock) + blacklistTracker.taskFailed(stage1, partition1, taskInfo_1_hostA) + for { + executor <- (1 to 4).map(_.toString) + partition <- 0 until 10 + stage <- (1 to 2) + } { + val exp = (executor == "1" && stage == stage1 && partition == 1) + assert(blacklistTracker.isExecutorBlacklisted(executor, stage, partition) === exp) + } + assert(blacklistTracker.nodeBlacklist() === Set()) + assertEquivalentToSet(blacklistTracker.isNodeBlacklisted(_), Set()) + assertEquivalentToSet(blacklistTracker.isNodeBlacklistedForStage(_, stage1), Set()) + assertEquivalentToSet(blacklistTracker.isNodeBlacklistedForStage(_, stage2), Set()) - tracker.updateFailedExecutors(stage1, partition3, taskInfo_3_hostB, FAILURE) - assert(tracker.executorBlacklist(scheduler, stage1, 3) === Set("3")) - assert(tracker.nodeBlacklist() === Set()) + // Task 1 & 2 failed on both executor 1 & 2, so we blacklist all executors on that host, + // for all tasks for the stage. Note the api expects multiple checks for each type of + // blacklist -- this actually fits naturally with its use in the scheduler + blacklistTracker.taskFailed(stage1, partition1, + new TaskInfo(2L, 1, 1, 0L, "2", "hostA", TaskLocality.ANY, false)) + blacklistTracker.taskFailed(stage1, partition2, + new TaskInfo(3L, 2, 1, 0L, "2", "hostA", TaskLocality.ANY, false)) + blacklistTracker.taskFailed(stage1, partition2, + new TaskInfo(4L, 2, 1, 0L, "1", "hostA", TaskLocality.ANY, false)) + // we don't explicitly return the executors in hostA here, but that is OK + for { + executor <- (1 to 4).map(_.toString) + stage <- (1 to 2) + partition <- 0 until 10 + } { + withClue(s"exec = $executor; stage = $stage; part = $partition") { + val badExec = (executor == "1" || executor == "2") + val badPart = (partition == 1 || partition == 2) + val taskExp = (badExec && stage == stage1 && badPart) + assert(blacklistTracker.isExecutorBlacklisted(executor, stage, partition) === taskExp) + val executorExp = badExec && stage == stage1 + assert(blacklistTracker.isExecutorBlacklistedForStage(stage, executor) === executorExp) + } + } + assertEquivalentToSet(blacklistTracker.isNodeBlacklistedForStage(_, stage1), Set("hostA")) + assertEquivalentToSet(blacklistTracker.isNodeBlacklistedForStage(_, stage2), Set()) + // we dont' blacklist the nodes or executors till the stages complete + assert(blacklistTracker.nodeBlacklist() === Set()) + assertEquivalentToSet(blacklistTracker.isNodeBlacklisted(_), Set()) + assertEquivalentToSet(blacklistTracker.isExecutorBlacklisted(_), Set()) - tracker.updateFailedExecutors(stage1, partition1, taskInfo_2_hostA, Success) - assert(tracker.executorBlacklist(scheduler, stage1, 1) === Set()) + // when the stage completes successfully, now there is sufficient evidence we've got + // bad executors and node + blacklistTracker.taskSetSucceeded(stage1, scheduler) + assert(blacklistTracker.nodeBlacklist() === Set("hostA")) + assertEquivalentToSet(blacklistTracker.isNodeBlacklisted(_), Set("hostA")) + assertEquivalentToSet(blacklistTracker.isExecutorBlacklisted(_), Set("1", "2")) - // Task 2 on Stage 2 failed on Executor 2 - tracker.updateFailedExecutors(stage2, partition2, taskInfo_2_hostA, FAILURE) - assert(tracker.executorBlacklist(scheduler, stage1, 1) === Set()) - assert(tracker.executorBlacklist(scheduler, stage2, 1) === Set()) - assert(tracker.executorBlacklist(scheduler, stage1, 2) === Set()) - assert(tracker.executorBlacklist(scheduler, stage2, 2) === Set("2")) + clock.advance(blacklistTracker.EXECUTOR_RECOVERY_MILLIS + 1) + blacklistTracker.expireExecutorsInBlacklist() + assert(blacklistTracker.nodeBlacklist() === Set()) + assertEquivalentToSet(blacklistTracker.isNodeBlacklisted(_), Set()) + assertEquivalentToSet(blacklistTracker.isExecutorBlacklisted(_), Set()) } - test("AdvencedSingleTask strategy works") { - val conf = new SparkConf().setAppName("test").setMaster("local") + def trackerFixture: (BlacklistTrackerImpl, TaskSchedulerImpl) = { + val conf = new SparkConf().setAppName("test").setMaster("local") .set("spark.ui.enabled", "false") .set("spark.scheduler.blacklist.advancedStrategy", "true") .set("spark.scheduler.executorTaskBlacklistTime", "1000") val scheduler = mock[TaskSchedulerImpl] when(scheduler.getExecutorsAliveOnHost("hostA")).thenReturn(Some(Set("1", "2", "4"))) + Set("1", "2", "4").foreach { execId => + when(scheduler.getHostForExecutor(execId)).thenReturn("hostA") + } - // Task 1 failed on both executor 1 - val tracker = new BlacklistTracker(conf, clock) - tracker.updateFailedExecutors(stage1, partition1, taskInfo_1_hostA, FAILURE) - assert(tracker.executorBlacklist(scheduler, stage1, 1) === Set("1")) - assert(tracker.executorBlacklist(scheduler, stage1, 2) === Set()) + clock.setTime(0) + blacklistTracker = new BlacklistTrackerImpl(conf, clock) + (blacklistTracker, scheduler) + } + + test("executors can be blacklisted with only a few failures per stage") { + val (tracker, scheduler) = trackerFixture + // for 4 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 + (0 until 4).foreach { stage => + tracker.taskFailed(stage, 0, + new TaskInfo(stage, 0, 0, 0, "1", "hostA", TaskLocality.ANY, false)) + tracker.taskSucceeded(stage, 0, + new TaskInfo(stage, 0, 1, 0, "2", "hostA", TaskLocality.ANY, false)) + tracker.taskSetSucceeded(stage, scheduler) + } + assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set("1")) + } + + // if an executor has many task failures, but the task set ends up failing, don't count it + // against the executor + test("executors aren't blacklisted if task sets fail") { + val (tracker, scheduler) = trackerFixture + // for 4 different stages, executor 1 fails a task, and then the taskSet fails. + (0 until 4).foreach { stage => + tracker.taskFailed(stage, 0, + new TaskInfo(stage, 0, 0, 0, "1", "hostA", TaskLocality.ANY, false)) + tracker.taskSetFailed(stage) + } + assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set()) + } + + Seq(true, false).foreach { succeedTaskSet => + test(s"stage blacklist updates correctly on stage completion ($succeedTaskSet)") { + // within one taskset, an executor fails a few times, so its blacklisted for the taskset. + // but if the taskset fails, we don't blacklist the executor after the stage. + val (tracker, scheduler) = trackerFixture + val stageId = 1 + (if (succeedTaskSet) 1 else 0) + (0 until 4).foreach { partition => + tracker.taskFailed(stageId, partition, new TaskInfo(stageId * 4 + partition, partition, 0, + clock.getTimeMillis(), "1", "hostA", TaskLocality.ANY, false)) + } + assert(tracker.isExecutorBlacklistedForStage(stageId, "1")) + assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set()) + if (succeedTaskSet) { + // the task set succeeded elsewhere, so we count those failures against our executor, + // and blacklist it across stages + tracker.taskSetSucceeded(stageId, scheduler) + assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set("1")) + } else { + // the task set failed, so we don't count these failures against the executor for other + // stages + tracker.taskSetFailed(stageId) + assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set()) + } + } + } + + test("blacklisted executors and nodes get recovered with time") { + val (tracker, scheduler) = trackerFixture + (0 until 4).foreach { partition => + tracker.taskFailed(0, partition, new TaskInfo(partition, partition, 0, clock.getTimeMillis(), + "1", "hostA", TaskLocality.ANY, false)) + } + tracker.taskSetSucceeded(0, scheduler) assert(tracker.nodeBlacklist() === Set()) + assertEquivalentToSet(tracker.isNodeBlacklisted(_), Set()) + assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set("1")) - // Task 1 failed on both executor 2 - tracker.updateFailedExecutors(stage1, partition1, taskInfo_2_hostA, FAILURE) - assert(tracker.executorBlacklist(scheduler, stage1, 1) === Set("1", "2", "4")) - assert(tracker.executorBlacklist(scheduler, stage1, 2) === Set("1", "2", "4")) - assert(tracker.executorBlacklist(scheduler, stage2, 1) === Set()) - assert(tracker.nodeBlacklistForStage(stage1) === Set("hostA")) - assert(tracker.nodeBlacklistForStage(stage2) === Set()) + (0 until 4).foreach { partition => + tracker.taskFailed(1, partition, new TaskInfo(partition + 4, partition, 0, + clock.getTimeMillis(), "2", "hostA", TaskLocality.ANY, false)) + } + tracker.taskSetSucceeded(1, scheduler) assert(tracker.nodeBlacklist() === Set("hostA")) + assertEquivalentToSet(tracker.isNodeBlacklisted(_), Set("hostA")) + assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set("1", "2")) - // Task 1 succeeded on executor 1, so we remove executor 1 from blacklist - tracker.updateFailedExecutors(stage1, partition1, taskInfo_1_hostA, Success) - assert(tracker.executorBlacklist(scheduler, stage1, 1) === Set("2")) - assert(tracker.nodeBlacklistForStage(stage1) === Set()) + clock.advance(tracker.EXECUTOR_RECOVERY_MILLIS + 1) + // TODO might want to change this to avoid the need for expiry thread, if that eliminates the + // need for the locks. In which case, expiry would happen automatically. + tracker.expireExecutorsInBlacklist() assert(tracker.nodeBlacklist() === Set()) + assertEquivalentToSet(tracker.isNodeBlacklisted(_), Set()) + assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set()) + + // fail one more task, but executor isn't put back into blacklist since count reset to 0 + tracker.taskFailed(1, 0, new TaskInfo(5, 0, 0, clock.getTimeMillis(), + "1", "hostA", TaskLocality.ANY, false)) + tracker.taskSetSucceeded(1, scheduler) + assert(tracker.nodeBlacklist() === Set()) + assertEquivalentToSet(tracker.isNodeBlacklisted(_), Set()) + assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set()) + } + + test("memory cleaned up as tasksets complete") { + // We want to make sure that memory used by the blacklist tracker is not O(nTotalTaskSetsRun), + // that would be really bad for long-lived applications. This test just requires some knowledge + // of the internals on what to check (without the overhead of trying to trigger an OOM or + // something). + val (tracker, scheduler) = trackerFixture + // fail a couple of tasks in two stages + for { + stage <- 0 until 2 + partition <- 0 until 4 + } { + val tid = stage * 4 + partition + // we want to fail on multiple executors, to trigger node blacklist + val exec = (partition % 2).toString + tracker.taskFailed(stage, partition, new TaskInfo(tid, partition, 0, clock.getTimeMillis(), + exec, "hostA", TaskLocality.ANY, false)) + } + + when(scheduler.getHostForExecutor("0")).thenReturn("hostA") + when(scheduler.getHostForExecutor("1")).thenReturn("hostA") + when(scheduler.getExecutorsAliveOnHost("hostA")).thenReturn(Some(Set("0", "1"))) + + // just make sure our test is even checking something useful -- we expect these data structures + // to grow for running task sets with failed tasks + assert(tracker.stageIdToExecToFailures.nonEmpty) + assert(tracker.stageIdToBlacklistedNodes.nonEmpty) + + // now say stage 0 fails, and stage 1 completes + tracker.taskSetFailed(0) + tracker.taskSetSucceeded(1, scheduler) + + // datastructures should be empty again + assert(tracker.stageIdToExecToFailures.isEmpty) + assert(tracker.stageIdToBlacklistedNodes.isEmpty) } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala b/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala index 87600fe504b98..37ca6f3d53793 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala @@ -33,16 +33,20 @@ object FakeTask { * locations for each task (given as varargs) if this sequence is not empty. */ def createTaskSet(numTasks: Int, prefLocs: Seq[TaskLocation]*): TaskSet = { - createTaskSet(numTasks, 0, prefLocs: _*) + createTaskSet(numTasks, 0, 0, prefLocs: _*) } - def createTaskSet(numTasks: Int, stageAttemptId: Int, prefLocs: Seq[TaskLocation]*): TaskSet = { + def createTaskSet( + numTasks: Int, + stageId: Int, + stageAttemptId: Int, + prefLocs: Seq[TaskLocation]*): TaskSet = { if (prefLocs.size != 0 && prefLocs.size != numTasks) { throw new IllegalArgumentException("Wrong number of task locations") } val tasks = Array.tabulate[Task[_]](numTasks) { i => new FakeTask(0, i, if (prefLocs.size != 0) prefLocs(i) else Nil) } - new TaskSet(tasks, 0, stageAttemptId, 0, null) + new TaskSet(tasks, stageId, stageAttemptId, 0, null) } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala index 14f52a6be9d1f..85e09a0da361f 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala @@ -325,7 +325,11 @@ private[spark] abstract class MockBackend( executorIdToExecutor(task.executorId).freeCores += taskScheduler.CPUS_PER_TASK freeCores += taskScheduler.CPUS_PER_TASK } - reviveOffers() + // optimization (which is used by the actual backends too) -- don't revive offers on *all* + // executors when a task completes, just on the one which completed + val exec = executorIdToExecutor(task.executorId) + reviveWithOffers(Seq(WorkerOffer(executorId = exec.executorId, host = exec.host, + cores = exec.freeCores))) } } @@ -381,7 +385,10 @@ private[spark] abstract class MockBackend( * scheduling. */ override def reviveOffers(): Unit = { - val offers: Seq[WorkerOffer] = generateOffers() + reviveWithOffers(generateOffers()) + } + + def reviveWithOffers(offers: Seq[WorkerOffer]): Unit = { val newTaskDescriptions = taskScheduler.resourceOffers(offers).flatten // get the task now, since that requires a lock on TaskSchedulerImpl, to prevent individual // tests from introducing a race if they need it @@ -504,9 +511,9 @@ class TestTaskScheduler(sc: SparkContext) extends TaskSchedulerImpl(sc) { super.submitTasks(taskSet) } - override def taskSetFinished(manager: TaskSetManager): Unit = { + override def taskSetFinished(manager: TaskSetManager, success: Boolean): Unit = { runningTaskSets -= manager.taskSet - super.taskSetFinished(manager) + super.taskSetFinished(manager, success) } } @@ -590,7 +597,7 @@ class BasicSchedulerIntegrationSuite extends SchedulerIntegrationSuite[SingleCor * (a) map output is available whenever we run stage 1 * (b) we get a second attempt for stage 0 & stage 1 */ - testScheduler("job with fetch failure") { + testNoBlacklist("job with fetch failure") { val input = new MockRDD(sc, 2, Nil) val shuffledRdd = shuffle(10, input) val shuffleId = shuffledRdd.shuffleDeps.head.shuffleId @@ -621,12 +628,12 @@ class BasicSchedulerIntegrationSuite extends SchedulerIntegrationSuite[SingleCor val duration = Duration(1, SECONDS) awaitJobTermination(jobFuture, duration) } + assertDataStructuresEmpty() assert(results === (0 until 10).map { idx => idx -> (42 + idx) }.toMap) assert(stageToAttempts === Map(0 -> Set(0, 1), 1 -> Set(0, 1))) - assertDataStructuresEmpty() } - testScheduler("job failure after 4 attempts") { + testNoBlacklist("job failure after 4 attempts") { def runBackend(): Unit = { val (taskDescription, _) = backend.beginTask() backend.taskFailed(taskDescription, new RuntimeException("test task failure")) @@ -639,4 +646,11 @@ class BasicSchedulerIntegrationSuite extends SchedulerIntegrationSuite[SingleCor } assertDataStructuresEmpty(noFailure = false) } + + + def testNoBlacklist(name: String)(body: => Unit): Unit = { + // in these simple tests, we only have one executor, so it doens't make sense to turn on the + // blacklist. Just an artifact of this simple test-framework still kinda acting like local-mode + testScheduler(name, extraConfs = Seq("spark.scheduler.blacklist.enabled" -> "false"))(body) + } } 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 100b15740ca92..29ed433730a6c 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -17,7 +17,10 @@ package org.apache.spark.scheduler +import org.mockito.Matchers._ +import org.mockito.Mockito.{atLeast, never, times, verify, when} import org.scalatest.BeforeAndAfterEach +import org.scalatest.mock.MockitoSugar import org.apache.spark._ import org.apache.spark.internal.Logging @@ -30,7 +33,7 @@ class FakeSchedulerBackend extends SchedulerBackend { } class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with BeforeAndAfterEach - with Logging { + with Logging with MockitoSugar { var failedTaskSetException: Option[Throwable] = None @@ -60,11 +63,26 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B } def setupScheduler(confs: (String, String)*): TaskSchedulerImpl = { + val conf = new SparkConf().setMaster("local").setAppName("TaskSchedulerImplSuite") + confs.foreach { case (k, v) => + conf.set(k, v) + } + sc = new SparkContext(conf) + taskScheduler = new TaskSchedulerImpl(sc) + setupHelper() + } + + def setupScheduler(blacklist: BlacklistTracker, confs: (String, String)*): TaskSchedulerImpl = { sc = new SparkContext("local", "TaskSchedulerImplSuite") confs.foreach { case (k, v) => sc.conf.set(k, v) } - taskScheduler = new TaskSchedulerImpl(sc) + taskScheduler = + new TaskSchedulerImpl(sc, sc.conf.getInt("spark.task.maxFailures", 4), blacklist) + setupHelper() + } + + def setupHelper(): TaskSchedulerImpl = { taskScheduler.initialize(new FakeSchedulerBackend) // Need to initialize a DAGScheduler for the taskScheduler to use for callbacks. dagScheduler = new DAGScheduler(sc, taskScheduler) { @@ -163,8 +181,8 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B test("refuse to schedule concurrent attempts for the same stage (SPARK-8103)") { val taskScheduler = setupScheduler() - val attempt1 = FakeTask.createTaskSet(1, 0) - val attempt2 = FakeTask.createTaskSet(1, 1) + val attempt1 = FakeTask.createTaskSet(1, 0, 0) + val attempt2 = FakeTask.createTaskSet(1, 0, 1) taskScheduler.submitTasks(attempt1) intercept[IllegalStateException] { taskScheduler.submitTasks(attempt2) } @@ -172,7 +190,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B taskScheduler.taskSetManagerForAttempt(attempt1.stageId, attempt1.stageAttemptId) .get.isZombie = true taskScheduler.submitTasks(attempt2) - val attempt3 = FakeTask.createTaskSet(1, 2) + val attempt3 = FakeTask.createTaskSet(1, 0, 2) intercept[IllegalStateException] { taskScheduler.submitTasks(attempt3) } taskScheduler.taskSetManagerForAttempt(attempt2.stageId, attempt2.stageAttemptId) .get.isZombie = true @@ -201,7 +219,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B assert(0 === taskDescriptions2.length) // if we schedule another attempt for the same stage, it should get scheduled - val attempt2 = FakeTask.createTaskSet(10, 1) + val attempt2 = FakeTask.createTaskSet(10, 0, 1) // submit attempt 2, offer some resources, some tasks get scheduled taskScheduler.submitTasks(attempt2) @@ -233,12 +251,12 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B assert(0 === taskDescriptions2.length) // submit attempt 2 - val attempt2 = FakeTask.createTaskSet(10, 1) + val attempt2 = FakeTask.createTaskSet(10, 0, 1) taskScheduler.submitTasks(attempt2) // attempt 1 finished (this can happen even if it was marked zombie earlier -- all tasks were // already submitted, and then they finish) - taskScheduler.taskSetFinished(mgr1) + taskScheduler.taskSetFinished(mgr1, true) // now with another resource offer, we should still schedule all the tasks in attempt2 val taskDescriptions3 = taskScheduler.resourceOffers(workerOffers).flatten @@ -281,6 +299,174 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B assert(!failedTaskSet) } + test("scheduled tasks obey task and stage blacklists") { + val blacklist = mock[BlacklistTracker] + taskScheduler = setupScheduler(blacklist) + val stage0 = FakeTask.createTaskSet(numTasks = 2, stageId = 0, stageAttemptId = 0) + val stage1 = FakeTask.createTaskSet(numTasks = 2, stageId = 1, stageAttemptId = 0) + val stage2 = FakeTask.createTaskSet(numTasks = 2, stageId = 2, stageAttemptId = 0) + taskScheduler.submitTasks(stage0) + taskScheduler.submitTasks(stage1) + taskScheduler.submitTasks(stage2) + + val offers = Seq( + new WorkerOffer("executor0", "host0", 1), + new WorkerOffer("executor1", "host1", 1), + new WorkerOffer("executor2", "host1", 1), + new WorkerOffer("executor3", "host2", 10) + ) + + // setup our mock blacklist: + // stage 0 is blacklisted on node "host1" + // stage 1 is blacklist on executor "executor3" + // stage 0, part 0 is blacklisted on executor 0 + // (later stubs take precedence over earlier ones) + when(blacklist.isNodeBlacklisted(anyString())).thenReturn(false) + when(blacklist.isExecutorBlacklisted(anyString())).thenReturn(false) + when(blacklist.isNodeBlacklistedForStage(anyString(), anyInt())).thenReturn(false) + when(blacklist.isNodeBlacklistedForStage("host1", 0)).thenReturn(true) + when(blacklist.isExecutorBlacklistedForStage(anyInt(), anyString())).thenReturn(false) + when(blacklist.isExecutorBlacklistedForStage(1, "executor3")).thenReturn(true) + when(blacklist.isExecutorBlacklisted(anyString(), anyInt(), anyInt())).thenReturn(false) + when(blacklist.isExecutorBlacklisted("executor0", 0, 0)).thenReturn(true) + + val firstTaskAttempts = taskScheduler.resourceOffers(offers).flatten + // these verifications are tricky b/c we reference them multiple times -- also invoked when we + // check if we need to abort any stages from unschedulability. + ('0' until '2').foreach { hostNum => + val host = s"host$hostNum" + verify(blacklist, atLeast(1)).isNodeBlacklisted(host) + verify(blacklist, atLeast(1)).isNodeBlacklistedForStage(host, 0) + verify(blacklist, atLeast(1)).isNodeBlacklistedForStage(host, 1) + verify(blacklist, atLeast(1)).isNodeBlacklistedForStage(host, 2) + } + for { + exec <- Seq("executor1", "executor2") + part <- 0 to 1 + } { + // the node blacklist should ensure we never check the task blacklist. This is important + // for performance, otherwise we end up changing an O(1) operation into a + // O(numPendingTasks) one + verify(blacklist, never).isExecutorBlacklisted(exec, 0, part) + } + + // similarly, the executor blacklist for an entire stage should prevent us from ever checking + // the blacklist for specific parts in a stage. + (0 to 1).foreach { part => + verify(blacklist, never).isExecutorBlacklisted("executor3", 1, part) + } + + // we should schedule all tasks. + assert(firstTaskAttempts.size == 6) + def tasksForStage(stageId: Int): Seq[TaskDescription] = { + firstTaskAttempts.filter{_.name.contains(s"stage $stageId")} + } + tasksForStage(0).foreach { task => + // exec 1 & 2 blacklisted for node + // exec 0 blacklisted just for part 0 + if (task.index == 0) { + assert(task.executorId == "executor3") + } else { + assert(Set("executor0", "executor3").contains(task.executorId)) + } + } + tasksForStage(1).foreach { task => + // exec 3 blacklisted + assert("executor3" != task.executorId) + } + // no restrictions on stage 2 + + // have all tasksets finish (stages 0 & 1 successfully, 2 unsuccessfully) + (0 to 2).foreach { stageId => + val tasks = tasksForStage(stageId) + val tsm = taskScheduler.taskSetManagerForAttempt(stageId, 0).get + val valueSer = SparkEnv.get.serializer.newInstance() + if (stageId == 2) { + // just need to make one task fail 4 times + var task = tasks(0) + val taskIndex = task.index + (0 until 4).foreach { attempt => + assert(task.attemptNumber == attempt) + tsm.handleFailedTask(task.taskId, TaskState.FAILED, TaskResultLost) + val nextAttempts = + taskScheduler.resourceOffers(Seq(WorkerOffer("executor4", "host4", 1))).flatten + if (attempt < 3) { + assert(nextAttempts.size == 1) + task = nextAttempts(0) + assert(task.index == taskIndex) + } else { + assert(nextAttempts.size == 0) + } + } + // end the other task of the taskset, doesn't matter whether it succeeds or fails + val otherTask = tasks(1) + val result = new DirectTaskResult[Int](valueSer.serialize(otherTask.taskId), Seq()) + tsm.handleSuccessfulTask(otherTask.taskId, result) + } else { + tasks.foreach { task => + val result = new DirectTaskResult[Int](valueSer.serialize(task.taskId), Seq()) + tsm.handleSuccessfulTask(task.taskId, result) + } + } + } + + // the tasksSets complete, so the tracker should be notified + verify(blacklist, times(1)).taskSetSucceeded(0, taskScheduler) + verify(blacklist, times(1)).taskSetSucceeded(1, taskScheduler) + verify(blacklist, times(1)).taskSetFailed(2) + } + + test("scheduled tasks obey node and executor blacklists") { + // another case with full node & executor blacklist + val blacklist = mock[BlacklistTracker] + taskScheduler = setupScheduler(blacklist) + val stage0 = FakeTask.createTaskSet(numTasks = 2, stageId = 0, stageAttemptId = 0) + val stage1 = FakeTask.createTaskSet(numTasks = 2, stageId = 1, stageAttemptId = 0) + val stage2 = FakeTask.createTaskSet(numTasks = 2, stageId = 2, stageAttemptId = 0) + taskScheduler.submitTasks(stage0) + taskScheduler.submitTasks(stage1) + taskScheduler.submitTasks(stage2) + + val offers = Seq( + new WorkerOffer("executor0", "host0", 1), + new WorkerOffer("executor1", "host1", 1), + new WorkerOffer("executor2", "host1", 1), + new WorkerOffer("executor3", "host2", 10) + ) + + // setup our mock blacklist: + // host1, executor0 & executor3 are completely blacklisted + when(blacklist.isNodeBlacklisted(anyString())).thenReturn(false) + when(blacklist.isNodeBlacklisted("host1")).thenReturn(true) + when(blacklist.isExecutorBlacklisted(anyString())).thenReturn(false) + when(blacklist.isExecutorBlacklisted("executor0")).thenReturn(true) + when(blacklist.isExecutorBlacklisted("executor3")).thenReturn(true) + when(blacklist.isNodeBlacklistedForStage(anyString(), anyInt())).thenReturn(false) + when(blacklist.isExecutorBlacklistedForStage(anyInt(), anyString())).thenReturn(false) + when(blacklist.isExecutorBlacklisted(anyString(), anyInt(), anyInt())).thenReturn(false) + + val stageToTsm = (0 to 2).map { stageId => + val tsm = taskScheduler.taskSetManagerForAttempt(stageId, 0).get + stageId -> tsm + }.toMap + + val firstTaskAttempts = taskScheduler.resourceOffers(offers).flatten + firstTaskAttempts.foreach { task => logInfo(s"scheduled $task on ${task.executorId}") } + assert(firstTaskAttempts.isEmpty) + ('0' until '2').foreach { hostNum => + verify(blacklist, atLeast(1)).isNodeBlacklisted("host" + hostNum) + } + verify(blacklist, never()).isNodeBlacklistedForStage(anyString(), anyInt()) + verify(blacklist, never()).isExecutorBlacklistedForStage(anyInt(), anyString()) + verify(blacklist, never()).isExecutorBlacklisted(anyString(), anyInt(), anyInt()) + + // we should have aborted the existing stages, since they aren't schedulable + (0 to 2).foreach { stageId => + assert(stageToTsm(stageId).isZombie) + verify(blacklist).taskSetFailed(stageId) + } + } + 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 // test explores a particular corner case -- you may have one task fail, but still be @@ -328,8 +514,8 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B assert(tsm.isZombie) assert(failedTaskSet) val idx = failedTask.index - assert(failedTaskSetReason == s"Aborting TaskSet 0.0 because task $idx (partition $idx) has " + - s"already failed on executors (executor0), and no other executors are available.") + assert(failedTaskSetReason == s"Aborting TaskSet 0.0 because task $idx (partition $idx) " + + s"cannot run anywhere due to node and executor blacklist.") } test("don't abort if there is an executor available, though it hasn't had scheduled tasks yet") { @@ -376,7 +562,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B test("SPARK-16106 locality levels updated if executor added to existing host") { val taskScheduler = setupScheduler() - taskScheduler.submitTasks(FakeTask.createTaskSet(2, 0, + taskScheduler.submitTasks(FakeTask.createTaskSet(2, 0, 0, (0 until 2).map { _ => Seq(TaskLocation("host0", "executor2"))}: _* )) @@ -410,4 +596,16 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B assert(thirdTaskDescs.size === 0) assert(taskScheduler.getExecutorsAliveOnHost("host1") === Some(Set("executor1", "executor3"))) } + + test("check for executors that can be expired from blacklist") { + val blacklist = mock[BlacklistTracker] + taskScheduler = setupScheduler(blacklist) + + taskScheduler.submitTasks(FakeTask.createTaskSet(1, 0, 0)) + taskScheduler.resourceOffers(Seq( + new WorkerOffer("executor0", "host0", 1) + )).flatten + + verify(blacklist).expireExecutorsInBlacklist() + } } 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 5b58241788d68..1d8a5d8fba112 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -113,7 +113,9 @@ class FakeTaskScheduler(sc: SparkContext, liveExecutors: (String, String)* /* ex } } - override def taskSetFinished(manager: TaskSetManager): Unit = finishedManagers += manager + override def taskSetFinished(manager: TaskSetManager, success: Boolean): Unit = { + finishedManagers += manager + } override def isExecutorAlive(execId: String): Boolean = executors.contains(execId) @@ -423,10 +425,8 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val taskSet = FakeTask.createTaskSet(1, Seq(TaskLocation("host1", "exec1"))) val clock = new ManualClock - // spy taskSetManager to set Manual clock for BlacklistTracker - val manager = new TaskSetManager(sched, taskSet, 4, clock) - val tracker = new BlacklistTracker(conf, clock) - manager.setBlacklistTracker(tracker) + val blacklist = new BlacklistTrackerImpl(conf, clock) + val manager = new TaskSetManager(sched, blacklist, taskSet, 4, clock) { val offerResult = manager.resourceOffer("exec1", "host1", PROCESS_LOCAL) @@ -479,20 +479,25 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg assert(manager.resourceOffer("exec2", "host2", ANY).isEmpty) } - // After reschedule delay, scheduling on exec1 should be possible. + // Despite advancing beyond the time for expiring executors from within the blacklist, + // we *never* expire from *within* the stage blacklist clock.advance(rescheduleDelay) - tracker.expireExecutorsInBlackList() + blacklist.expireExecutorsInBlacklist() { val offerResult = manager.resourceOffer("exec1", "host1", PROCESS_LOCAL) - assert(offerResult.isDefined, "Expect resource offer to return a task") + assert(offerResult.isEmpty) + } + { + val offerResult = manager.resourceOffer("exec3", "host1", NODE_LOCAL) + assert(offerResult.isDefined) assert(offerResult.get.index === 0) - assert(offerResult.get.executorId === "exec1") + assert(offerResult.get.executorId === "exec3") - assert(manager.resourceOffer("exec1", "host1", PROCESS_LOCAL).isEmpty) + assert(manager.resourceOffer("exec3", "host1", NODE_LOCAL).isEmpty) - // Cause exec1 to fail : failure 4 + // Cause exec3 to fail : failure 4 manager.handleFailedTask(offerResult.get.taskId, TaskState.FINISHED, TaskResultLost) } @@ -864,6 +869,10 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg assert(sched.endedTasks(3) === Success) } + test("don't update blacklist for shuffle fetch failures") { + pending + } + private def createTaskResult( id: Int, accumUpdates: Seq[AccumulatorV2[_, _]] = Seq.empty): DirectTaskResult[Int] = { diff --git a/docs/configuration.md b/docs/configuration.md index 1e293331a54e3..01142ca5e8577 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1156,24 +1156,10 @@ Apart from these, the following properties are also available, and may be useful - spark.scheduler.blacklist.timeout - 0s + spark.scheduler.blacklist.MORE + TODO - If executor blacklisting is enabled, this controls how long an executor remains in the blacklist before it is returned to the pool of available executors. - - - - spark.scheduler.blacklist.recoverPeriod - 60s - - If executor blacklisting is enabled, this controls how often to check if executors can be returned to the pool of active executors. - - - - spark.scheduler.blacklist.advancedStrategy - false - - set to tree to enable experimental advanced blacklist strategy. Comparing with the standard behavior before spark 1.6, it enables blacklist on node level. + TODO diff --git a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala index 60fdb22f6a000..31fafbc2a586b 100644 --- a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala +++ b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala @@ -126,8 +126,7 @@ private[spark] abstract class YarnSchedulerBackend( */ override def doRequestTotalExecutors(requestedTotal: Int): Boolean = { - val nodeBlacklist: Set[String] = scheduler.sc.blacklistTracker - .map(_.nodeBlacklist()).getOrElse(Set.empty[String]) + val nodeBlacklist: Set[String] = scheduler.blacklistTracker.nodeBlacklist() yarnSchedulerEndpointRef.askWithRetry[Boolean]( RequestExecutors(requestedTotal, localityAwareTasks, hostToLocalTaskCount, nodeBlacklist)) From d7adc6711e15994cefc280666486f54efb0f95cd Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 7 Jul 2016 15:32:51 -0500 Subject: [PATCH 03/57] (node,task) blacklisting --- .../spark/scheduler/BlacklistTracker.scala | 87 ++++++++++++++----- .../spark/scheduler/TaskSetManager.scala | 37 ++++---- .../scheduler/BlacklistIntegrationSuite.scala | 33 +++++-- .../scheduler/BlacklistTrackerSuite.scala | 24 ++--- 4 files changed, 126 insertions(+), 55 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala index 677cea3741b9e..2a9ab999b7d72 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala @@ -55,17 +55,24 @@ private[scheduler] trait BlacklistTracker { def isExecutorBlacklisted( executorId: String, stageId: Int, - partition: Int) : Boolean + indexInTaskSet: Int) : Boolean + + def isNodeBlacklisted( + node: String, + stageId: Int, + indexInTaskSet: Int) : Boolean def taskSucceeded( stageId: Int, - partition: Int, - info: TaskInfo): Unit + indexInTaskSet: Int, + info: TaskInfo, + scheduler: TaskSchedulerImpl): Unit def taskFailed( stageId: Int, - partition: Int, - info: TaskInfo): Unit + indexInTaskSet: Int, + info: TaskInfo, + scheduler: TaskSchedulerImpl): Unit def taskSetSucceeded(stageId: Int, scheduler: TaskSchedulerImpl): Unit @@ -94,6 +101,8 @@ private[scheduler] class BlacklistTrackerImpl( conf: SparkConf, clock: Clock = new SystemClock()) extends BlacklistTracker with Logging { + private val MAX_TASK_FAILURES_PER_NODE = + conf.getInt("spark.blacklist.maxTaskFailuresPerNode", 2) private val MAX_FAILURES_PER_EXEC = conf.getInt("spark.blacklist.maxFailedTasksPerExecutor", 2) private val MAX_FAILURES_PER_EXEC_STAGE = @@ -110,6 +119,8 @@ private[scheduler] class BlacklistTrackerImpl( // failures for each executor by stage. Only tracked while the stage is running. val stageIdToExecToFailures: HashMap[Int, HashMap[String, FailureStatus]] = new HashMap() + val stageIdToNodeBlacklistedTasks: HashMap[Int, HashMap[String, HashSet[Int]]] = + new HashMap() val stageIdToBlacklistedNodes: HashMap[Int, HashSet[String]] = new HashMap() private val executorIdToBlacklistExpiryTime: HashMap[String, Long] = new HashMap() private val nodeIdToBlacklistExpiryTime: HashMap[String, Long] = new HashMap() @@ -222,8 +233,9 @@ private[scheduler] class BlacklistTrackerImpl( override def taskSucceeded( stageId: Int, - partition: Int, - info: TaskInfo): Unit = { + indexInTaskSet: Int, + info: TaskInfo, + scheduler: TaskSchedulerImpl): Unit = { // no-op intentionally, included just for symmetry. success to failure ratio is irrelevant, we // just blacklist based on failures. Furthermore, one success does not override previous // failures, since the bad node / executor may not fail *every* time @@ -231,12 +243,29 @@ private[scheduler] class BlacklistTrackerImpl( override def taskFailed( stageId: Int, - partition: Int, - info: TaskInfo): Unit = { + indexInTaskSet: Int, + info: TaskInfo, + scheduler: TaskSchedulerImpl): Unit = { val stageFailures = stageIdToExecToFailures.getOrElseUpdate(stageId, new HashMap()) val failureStatus = stageFailures.getOrElseUpdate(info.executorId, new FailureStatus()) failureStatus.totalFailures += 1 - failureStatus.failuresByPart += partition + failureStatus.failuresByTask += indexInTaskSet + + // check if this task has also failed on other executors on the same host, and if so, blacklist + // this task from the host + val failuresOnHost = (for { + exec <- scheduler.getExecutorsAliveOnHost(info.host).getOrElse(Set()).toSeq + failures <- stageFailures.get(exec) + } yield { + if (failures.failuresByTask.contains(indexInTaskSet)) 1 else 0 + }).sum + logInfo(s"total failures on host ${info.host} = $failuresOnHost") + if (failuresOnHost > MAX_TASK_FAILURES_PER_NODE) { + stageIdToNodeBlacklistedTasks.getOrElseUpdate(stageId, new HashMap()) + .getOrElseUpdate(info.host, new HashSet()) += indexInTaskSet + } + + if (failureStatus.totalFailures >= MAX_FAILURES_PER_EXEC_STAGE) { // this executor has been pushed into the blacklist for this stage. Lets check if it pushes // the whole node into the blacklist @@ -257,14 +286,14 @@ private[scheduler] class BlacklistTrackerImpl( override def isExecutorBlacklisted( executorId: String, stageId: Int, - partition: Int) : Boolean = { + indexInTaskSet: Int): Boolean = { // intentionally avoiding .getOrElse(..., new HashMap()) to avoid lots of object // creation, since this method gets called a *lot* stageIdToExecToFailures.get(stageId) match { case Some(stageFailures) => stageFailures.get(executorId) match { case Some(failures) => - failures.failuresByPart.contains(partition) + failures.failuresByTask.contains(indexInTaskSet) case None => false } @@ -273,6 +302,15 @@ private[scheduler] class BlacklistTrackerImpl( } } + override def isNodeBlacklisted( + node: String, + stageId: Int, + indexInTaskSet: Int): Boolean = { + stageIdToNodeBlacklistedTasks.get(stageId).flatMap { nodeToFailures => + nodeToFailures.get(node).map{_.contains(indexInTaskSet)} + }.getOrElse(false) + } + override def removeExecutor(executorId: String): Unit = { executorIdToBlacklistExpiryTime -= executorId executorIdToFailureCount -= executorId @@ -324,13 +362,14 @@ private[scheduler] object BlacklistTracker extends Logging { } } -/** Failures for one executor, within one stage */ +/** Failures for one executor, within one taskset */ private[scheduler] final class FailureStatus { - val failuresByPart = HashSet[Int]() + /** index of the tasks in the taskset that have failed on this executor. */ + val failuresByTask = HashSet[Int]() var totalFailures = 0 override def toString(): String = { - s"totalFailures = $totalFailures; partitionsFailed = $failuresByPart" + s"totalFailures = $totalFailures; tasksFailed = $failuresByTask" } } @@ -362,17 +401,25 @@ private[scheduler] object NoopBlacklistTracker extends BlacklistTracker { override def isExecutorBlacklisted( executorId: String, stageId: Int, - partition: Int) : Boolean = false + indexInTaskSet: Int) : Boolean = false + + override def isNodeBlacklisted( + node: String, + stageId: Int, + indexInTaskSet: Int): Boolean = false + override def taskSucceeded( stageId: Int, - partition: Int, - info: TaskInfo): Unit = {} + indexInTaskSet: Int, + info: TaskInfo, + scheduler: TaskSchedulerImpl): Unit = {} override def taskFailed( stageId: Int, - partition: Int, - info: TaskInfo): Unit = {} + indexInTaskSet: Int, + info: TaskInfo, + scheduler: TaskSchedulerImpl): Unit = {} override def removeExecutor(executorId: String): Unit = {} } 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 1c1c1836f16b1..9eb715b51c3ee 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -241,12 +241,16 @@ private[spark] class TaskSetManager( * This method also cleans up any tasks in the list that have already * been launched, since we want that to happen lazily. */ - private def dequeueTaskFromList(execId: String, list: ArrayBuffer[Int]): Option[Int] = { + private def dequeueTaskFromList( + execId: String, + host: String, + list: ArrayBuffer[Int]): Option[Int] = { var indexOffset = list.size while (indexOffset > 0) { indexOffset -= 1 val index = list(indexOffset) - if (!blacklistTracker.isExecutorBlacklisted(execId, stageId, index)) { + if (!blacklistTracker.isNodeBlacklisted(host, stageId, index) && + !blacklistTracker.isExecutorBlacklisted(execId, stageId, index)) { // This should almost always be list.trimEnd(1) to remove tail list.remove(indexOffset) if (copiesRunning(index) == 0 && !successful(index)) { @@ -275,6 +279,7 @@ private[spark] class TaskSetManager( def canRunOnHost(index: Int): Boolean = !hasAttemptOnHost(index, host) && + !blacklistTracker.isNodeBlacklisted(host, stageId, index) && !blacklistTracker.isExecutorBlacklisted(execId, stageId, index) if (!speculatableTasks.isEmpty) { @@ -348,19 +353,19 @@ private[spark] class TaskSetManager( private def dequeueTask(execId: String, host: String, maxLocality: TaskLocality.Value) : Option[(Int, TaskLocality.Value, Boolean)] = { - for (index <- dequeueTaskFromList(execId, getPendingTasksForExecutor(execId))) { + for (index <- dequeueTaskFromList(execId, host, getPendingTasksForExecutor(execId))) { return Some((index, TaskLocality.PROCESS_LOCAL, false)) } if (TaskLocality.isAllowed(maxLocality, TaskLocality.NODE_LOCAL)) { - for (index <- dequeueTaskFromList(execId, getPendingTasksForHost(host))) { + for (index <- dequeueTaskFromList(execId, host, getPendingTasksForHost(host))) { return Some((index, TaskLocality.NODE_LOCAL, false)) } } if (TaskLocality.isAllowed(maxLocality, TaskLocality.NO_PREF)) { // Look for noPref tasks after NODE_LOCAL for minimize cross-rack traffic - for (index <- dequeueTaskFromList(execId, pendingTasksWithNoPrefs)) { + for (index <- dequeueTaskFromList(execId, host, pendingTasksWithNoPrefs)) { return Some((index, TaskLocality.PROCESS_LOCAL, false)) } } @@ -368,14 +373,14 @@ private[spark] class TaskSetManager( if (TaskLocality.isAllowed(maxLocality, TaskLocality.RACK_LOCAL)) { for { rack <- sched.getRackForHost(host) - index <- dequeueTaskFromList(execId, getPendingTasksForRack(rack)) + index <- dequeueTaskFromList(execId, host, getPendingTasksForRack(rack)) } { return Some((index, TaskLocality.RACK_LOCAL, false)) } } if (TaskLocality.isAllowed(maxLocality, TaskLocality.ANY)) { - for (index <- dequeueTaskFromList(execId, allPendingTasks)) { + for (index <- dequeueTaskFromList(execId, host, allPendingTasks)) { return Some((index, TaskLocality.ANY, false)) } } @@ -599,17 +604,17 @@ private[spark] class TaskSetManager( if (executorsByHost.nonEmpty) { // take any task that needs to be scheduled, and see if we can find some executor it *could* // run on - pendingTask.foreach { taskId => + pendingTask.foreach { indexInTaskSet => val stage = taskSet.stageId - val part = tasks(taskId).partitionId executorsByHost.foreach { case (host, execs) => if (!blacklistTracker.isNodeBlacklisted(host) && - !blacklistTracker.isNodeBlacklistedForStage(host, stage)) { + !blacklistTracker.isNodeBlacklistedForStage(host, stage) && + !blacklistTracker.isNodeBlacklisted(host, stage, indexInTaskSet)) { execs.foreach { exec => if ( !blacklistTracker.isExecutorBlacklisted(exec) && !blacklistTracker.isExecutorBlacklistedForStage(stage, exec) && - !blacklistTracker.isExecutorBlacklisted(exec, stageId = stage, partition = part) + !blacklistTracker.isExecutorBlacklisted(exec, stage, indexInTaskSet) ) { // we've found some executor this task can run on. Its possible that some *other* // task isn't schedulable anywhere, but we will discover that in some later call, @@ -619,9 +624,9 @@ private[spark] class TaskSetManager( } } } - val partition = tasks(taskId).partitionId - abort(s"Aborting ${taskSet} because task $taskId (partition $partition) cannot run " + - s"anywhere due to node and executor blacklist.") + val partition = tasks(indexInTaskSet).partitionId + abort(s"Aborting ${taskSet} because task $indexInTaskSet (partition $partition) cannot " + + s"run anywhere due to node and executor blacklist.") } } } @@ -691,7 +696,7 @@ private[spark] class TaskSetManager( " because task " + index + " has already completed successfully") } - blacklistTracker.taskSucceeded(stageId, tasks(index).partitionId, info) + blacklistTracker.taskSucceeded(stageId, index, info, sched) maybeFinishTaskSet() } @@ -776,7 +781,7 @@ private[spark] class TaskSetManager( // always add to failed executors // TODO if there is a fetch failure, does it really make sense to add this? - blacklistTracker.taskFailed(stageId, tasks(index).partitionId, info) + blacklistTracker.taskFailed(stageId, index, info, sched) sched.dagScheduler.taskEnded(tasks(index), reason, null, accumUpdates, info) diff --git a/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala index 09fcd2b3560ce..f4963f9573424 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala @@ -56,21 +56,16 @@ class BlacklistIntegrationSuite extends SchedulerIntegrationSuite[MultiExecutorM // even with the blacklist turned on, bad configs can lead to job failure. To survive one // bad node, you need to make sure that - // maxTaskFailures > spark.blacklist.maxFailedTasksPerExecutorStage * - // spark.blacklist.maxFailedExecutorsPerNodeStage + // maxTaskFailures > min(spark.blacklist.maxTaskFailuresPerNode, nExecutorsPerHost) testScheduler( "With blacklist on, job will still fail if there are too many bad executors on bad host", extraConfs = Seq( "spark.scheduler.blacklist.enabled" -> "true", - // set this to something much longer than the test duration so that executors don't get - // removed from the blacklist during the test - "spark.scheduler.executorTaskBlacklistTime" -> "10000000", "spark.testing.nHosts" -> "2", "spark.testing.nExecutorsPerHost" -> "5", "spark.testing.nCoresPerExecutor" -> "10", "spark.task.maxFailures" -> "4", - "spark.blacklist.maxFailedTasksPerExecutorStage" -> "1", - "spark.blacklist.maxFailedExecutorsPerNodeStage" -> "5" + "spark.blacklist.maxTaskFailuresPerNode" -> "5" ) ) { // to reliably reproduce the failure, we have to use 1 task. That way, we ensure this @@ -86,6 +81,30 @@ class BlacklistIntegrationSuite extends SchedulerIntegrationSuite[MultiExecutorM assertDataStructuresEmpty(noFailure = false) } + + testScheduler( + "With default settings, job can succeed despite multiple bad executors on node", + extraConfs = Seq( + "spark.scheduler.blacklist.enabled" -> "true", + "spark.testing.nHosts" -> "2", + "spark.testing.nExecutorsPerHost" -> "5", + "spark.testing.nCoresPerExecutor" -> "10", + "spark.task.maxFailures" -> "4" + ) + ) { + // to reliably reproduce the failure, 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 -- we never try scheduling tasks on the good host-1, since 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) + awaitJobTermination(jobFuture, duration) + } + assertDataStructuresEmpty(noFailure = true) + } + // Here we run with the blacklist on, and the default config takes care of having this // robust to one bad node. testScheduler( diff --git a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala index 42ec9d4fcc9dc..cf00863d86e7f 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala @@ -83,7 +83,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M // Task 1 failed on executor 1 blacklistTracker = new BlacklistTrackerImpl(conf, clock) - blacklistTracker.taskFailed(stage1, partition1, taskInfo_1_hostA) + blacklistTracker.taskFailed(stage1, partition1, taskInfo_1_hostA, scheduler) for { executor <- (1 to 4).map(_.toString) partition <- 0 until 10 @@ -101,11 +101,11 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M // for all tasks for the stage. Note the api expects multiple checks for each type of // blacklist -- this actually fits naturally with its use in the scheduler blacklistTracker.taskFailed(stage1, partition1, - new TaskInfo(2L, 1, 1, 0L, "2", "hostA", TaskLocality.ANY, false)) + new TaskInfo(2L, 1, 1, 0L, "2", "hostA", TaskLocality.ANY, false), scheduler) blacklistTracker.taskFailed(stage1, partition2, - new TaskInfo(3L, 2, 1, 0L, "2", "hostA", TaskLocality.ANY, false)) + new TaskInfo(3L, 2, 1, 0L, "2", "hostA", TaskLocality.ANY, false), scheduler) blacklistTracker.taskFailed(stage1, partition2, - new TaskInfo(4L, 2, 1, 0L, "1", "hostA", TaskLocality.ANY, false)) + new TaskInfo(4L, 2, 1, 0L, "1", "hostA", TaskLocality.ANY, false), scheduler) // we don't explicitly return the executors in hostA here, but that is OK for { executor <- (1 to 4).map(_.toString) @@ -165,9 +165,9 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M // any particular taskset, but we still blacklist the executor overall eventually (0 until 4).foreach { stage => tracker.taskFailed(stage, 0, - new TaskInfo(stage, 0, 0, 0, "1", "hostA", TaskLocality.ANY, false)) + new TaskInfo(stage, 0, 0, 0, "1", "hostA", TaskLocality.ANY, false), scheduler) tracker.taskSucceeded(stage, 0, - new TaskInfo(stage, 0, 1, 0, "2", "hostA", TaskLocality.ANY, false)) + new TaskInfo(stage, 0, 1, 0, "2", "hostA", TaskLocality.ANY, false), scheduler) tracker.taskSetSucceeded(stage, scheduler) } assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set("1")) @@ -180,7 +180,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M // for 4 different stages, executor 1 fails a task, and then the taskSet fails. (0 until 4).foreach { stage => tracker.taskFailed(stage, 0, - new TaskInfo(stage, 0, 0, 0, "1", "hostA", TaskLocality.ANY, false)) + new TaskInfo(stage, 0, 0, 0, "1", "hostA", TaskLocality.ANY, false), scheduler) tracker.taskSetFailed(stage) } assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set()) @@ -194,7 +194,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M val stageId = 1 + (if (succeedTaskSet) 1 else 0) (0 until 4).foreach { partition => tracker.taskFailed(stageId, partition, new TaskInfo(stageId * 4 + partition, partition, 0, - clock.getTimeMillis(), "1", "hostA", TaskLocality.ANY, false)) + clock.getTimeMillis(), "1", "hostA", TaskLocality.ANY, false), scheduler) } assert(tracker.isExecutorBlacklistedForStage(stageId, "1")) assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set()) @@ -216,7 +216,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M val (tracker, scheduler) = trackerFixture (0 until 4).foreach { partition => tracker.taskFailed(0, partition, new TaskInfo(partition, partition, 0, clock.getTimeMillis(), - "1", "hostA", TaskLocality.ANY, false)) + "1", "hostA", TaskLocality.ANY, false), scheduler) } tracker.taskSetSucceeded(0, scheduler) assert(tracker.nodeBlacklist() === Set()) @@ -225,7 +225,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M (0 until 4).foreach { partition => tracker.taskFailed(1, partition, new TaskInfo(partition + 4, partition, 0, - clock.getTimeMillis(), "2", "hostA", TaskLocality.ANY, false)) + clock.getTimeMillis(), "2", "hostA", TaskLocality.ANY, false), scheduler) } tracker.taskSetSucceeded(1, scheduler) assert(tracker.nodeBlacklist() === Set("hostA")) @@ -242,7 +242,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M // fail one more task, but executor isn't put back into blacklist since count reset to 0 tracker.taskFailed(1, 0, new TaskInfo(5, 0, 0, clock.getTimeMillis(), - "1", "hostA", TaskLocality.ANY, false)) + "1", "hostA", TaskLocality.ANY, false), scheduler) tracker.taskSetSucceeded(1, scheduler) assert(tracker.nodeBlacklist() === Set()) assertEquivalentToSet(tracker.isNodeBlacklisted(_), Set()) @@ -264,7 +264,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M // we want to fail on multiple executors, to trigger node blacklist val exec = (partition % 2).toString tracker.taskFailed(stage, partition, new TaskInfo(tid, partition, 0, clock.getTimeMillis(), - exec, "hostA", TaskLocality.ANY, false)) + exec, "hostA", TaskLocality.ANY, false), scheduler) } when(scheduler.getHostForExecutor("0")).thenReturn("hostA") From a34e9aeb695958c749d306595d1adebe0207fdf9 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 7 Jul 2016 15:52:43 -0500 Subject: [PATCH 04/57] go back to having the blacklist tracker as an option, rather than the no-op implementation --- .../spark/scheduler/BlacklistTracker.scala | 139 +++--------------- .../spark/scheduler/TaskSchedulerImpl.scala | 57 ++++--- .../spark/scheduler/TaskSetManager.scala | 35 +++-- .../scheduler/BlacklistTrackerSuite.scala | 8 +- .../scheduler/TaskSchedulerImplSuite.scala | 2 +- .../spark/scheduler/TaskSetManagerSuite.scala | 4 +- 6 files changed, 77 insertions(+), 168 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala index 2a9ab999b7d72..a36356919220f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala @@ -27,60 +27,6 @@ import org.apache.spark.util.Clock import org.apache.spark.util.SystemClock import org.apache.spark.util.Utils - -/** - * Abstract interface for tracking all info related to executor and node blacklist. Though we only - * have one real implmentation currently, [[BlacklistTrackerImpl]], usage becomes much simpler - * by adding [[NoopBlacklistTracker]] than by making all uses switch to Option[BlacklistTracker]. - * See [[BlacklistTrackerImpl]] for all meaningful documentation. - */ -private[scheduler] trait BlacklistTracker { - def start(): Unit - - def stop(): Unit - - def expireExecutorsInBlacklist(): Unit - - /** Get the full blacklist. This *is* thread-safe, unlike all other methods. */ - def nodeBlacklist(): Set[String] - - def isNodeBlacklisted(node: String): Boolean - - def isNodeBlacklistedForStage(node: String, stageId: Int): Boolean - - def isExecutorBlacklisted(executorId: String): Boolean - - def isExecutorBlacklistedForStage(stageId: Int, executorId: String): Boolean - - def isExecutorBlacklisted( - executorId: String, - stageId: Int, - indexInTaskSet: Int) : Boolean - - def isNodeBlacklisted( - node: String, - stageId: Int, - indexInTaskSet: Int) : Boolean - - def taskSucceeded( - stageId: Int, - indexInTaskSet: Int, - info: TaskInfo, - scheduler: TaskSchedulerImpl): Unit - - def taskFailed( - stageId: Int, - indexInTaskSet: Int, - info: TaskInfo, - scheduler: TaskSchedulerImpl): Unit - - def taskSetSucceeded(stageId: Int, scheduler: TaskSchedulerImpl): Unit - - def taskSetFailed(stageId: Int): Unit - - def removeExecutor(executorId: String): Unit -} - /** * BlacklistTracker is designed to track problematic executors and nodes. It supports blacklisting * specific (executor, task) pairs within a stage, blacklisting entire executors and nodes for a @@ -97,9 +43,9 @@ private[scheduler] trait BlacklistTracker { * 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 BlacklistTrackerImpl( +private[scheduler] class BlacklistTracker ( conf: SparkConf, - clock: Clock = new SystemClock()) extends BlacklistTracker with Logging { + clock: Clock = new SystemClock()) extends Logging { private val MAX_TASK_FAILURES_PER_NODE = conf.getInt("spark.blacklist.maxTaskFailuresPerNode", 2) @@ -127,11 +73,11 @@ private[scheduler] class BlacklistTrackerImpl( private val _nodeBlacklist: AtomicReference[Set[String]] = new AtomicReference(Set()) private var nextExpiryTime: Long = Long.MaxValue - override def start(): Unit = {} + def start(): Unit = {} - override def stop(): Unit = {} + def stop(): Unit = {} - override def expireExecutorsInBlacklist(): Unit = { + def expireExecutorsInBlacklist(): 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) { @@ -155,7 +101,7 @@ private[scheduler] class BlacklistTrackerImpl( } } - override def taskSetSucceeded(stageId: Int, scheduler: TaskSchedulerImpl): Unit = { + def taskSetSucceeded(stageId: Int, scheduler: TaskSchedulerImpl): Unit = { // if any tasks failed, we count them towards the overall failure count for the executor at // this point. Also clean out all data about the stage to avoid increasing memory use. stageIdToExecToFailures.remove(stageId).map { failuresForStage => @@ -197,7 +143,7 @@ private[scheduler] class BlacklistTrackerImpl( stageIdToBlacklistedNodes.remove(stageId) } - override def taskSetFailed(stageId: Int): Unit = { + def taskSetFailed(stageId: Int): Unit = { // just throw away all the info for the failures in this taskSet -- assume the executors were // fine, the failures were just b/c the taskSet itself was bad (eg., bad user code) stageIdToExecToFailures.remove(stageId) @@ -208,14 +154,14 @@ private[scheduler] class BlacklistTrackerImpl( * Return true if this executor is blacklisted for the given stage. Completely ignores whether * the executor is blacklisted overall (or anything to do with the node the executor is on). */ - override def isExecutorBlacklistedForStage( + def isExecutorBlacklistedForStage( stageId: Int, executorId: String): Boolean = { stageIdToExecToFailures.get(stageId).flatMap(_.get(executorId)) .map(_.totalFailures >= MAX_FAILURES_PER_EXEC_STAGE).getOrElse(false) } - override def isExecutorBlacklisted(executorId: String): Boolean = { + def isExecutorBlacklisted(executorId: String): Boolean = { executorIdToBlacklistExpiryTime.contains(executorId) } @@ -223,25 +169,25 @@ private[scheduler] class BlacklistTrackerImpl( stageIdToBlacklistedNodes.get(stageId).map(_.contains(node)).getOrElse(false) } - override def nodeBlacklist(): Set[String] = { + def nodeBlacklist(): Set[String] = { _nodeBlacklist.get() } - override def isNodeBlacklisted(node: String): Boolean = { + def isNodeBlacklisted(node: String): Boolean = { nodeIdToBlacklistExpiryTime.contains(node) } - override def taskSucceeded( + def taskSucceeded( stageId: Int, indexInTaskSet: Int, info: TaskInfo, scheduler: TaskSchedulerImpl): Unit = { // no-op intentionally, included just for symmetry. success to failure ratio is irrelevant, we - // just blacklist based on failures. Furthermore, one success does not override previous + // just blacklist based on failures. Furthermore, one success does not previous // failures, since the bad node / executor may not fail *every* time } - override def taskFailed( + def taskFailed( stageId: Int, indexInTaskSet: Int, info: TaskInfo, @@ -283,7 +229,7 @@ private[scheduler] class BlacklistTrackerImpl( * need to return true if the executor is blacklisted for the entire stage, or blacklisted * altogether. */ - override def isExecutorBlacklisted( + def isExecutorBlacklisted( executorId: String, stageId: Int, indexInTaskSet: Int): Boolean = { @@ -302,7 +248,7 @@ private[scheduler] class BlacklistTrackerImpl( } } - override def isNodeBlacklisted( + def isNodeBlacklisted( node: String, stageId: Int, indexInTaskSet: Int): Boolean = { @@ -311,7 +257,7 @@ private[scheduler] class BlacklistTrackerImpl( }.getOrElse(false) } - override def removeExecutor(executorId: String): Unit = { + def removeExecutor(executorId: String): Unit = { executorIdToBlacklistExpiryTime -= executorId executorIdToFailureCount -= executorId stageIdToExecToFailures.values.foreach { execFailureOneStage => @@ -372,54 +318,3 @@ private[scheduler] final class FailureStatus { s"totalFailures = $totalFailures; tasksFailed = $failuresByTask" } } - -/** Used to turn off blacklisting completely */ -private[scheduler] object NoopBlacklistTracker extends BlacklistTracker { - - override def start: Unit = {} - - override def stop: Unit = {} - - override def expireExecutorsInBlacklist(): Unit = {} - - override def taskSetSucceeded(stageId: Int, scheduler: TaskSchedulerImpl): Unit = {} - - override def taskSetFailed(stageId: Int): Unit = {} - - override def isExecutorBlacklistedForStage(stageId: Int, executorId: String): Boolean = { - false - } - - override def isExecutorBlacklisted(executorId: String): Boolean = false - - override def nodeBlacklist(): Set[String] = Set() - - override def isNodeBlacklisted(node: String): Boolean = false - - override def isNodeBlacklistedForStage(node: String, stageId: Int): Boolean = false - - override def isExecutorBlacklisted( - executorId: String, - stageId: Int, - indexInTaskSet: Int) : Boolean = false - - override def isNodeBlacklisted( - node: String, - stageId: Int, - indexInTaskSet: Int): Boolean = false - - - override def taskSucceeded( - stageId: Int, - indexInTaskSet: Int, - info: TaskInfo, - scheduler: TaskSchedulerImpl): Unit = {} - - override def taskFailed( - stageId: Int, - indexInTaskSet: Int, - info: TaskInfo, - scheduler: TaskSchedulerImpl): Unit = {} - - override def removeExecutor(executorId: String): Unit = {} -} 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 ec1292fe31d9f..d3e13a3760f43 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -54,7 +54,7 @@ import org.apache.spark.util.{AccumulatorV2, Clock, SystemClock, ThreadUtils, Ut private[spark] class TaskSchedulerImpl private[scheduler]( val sc: SparkContext, val maxTaskFailures: Int, - private[scheduler] val blacklistTracker: BlacklistTracker, + private[scheduler] val blacklistTracker: Option[BlacklistTracker], private val clock: Clock = new SystemClock, isLocal: Boolean = false) extends TaskScheduler with Logging @@ -162,7 +162,7 @@ private[spark] class TaskSchedulerImpl private[scheduler]( override def start() { backend.start() - blacklistTracker.start() + blacklistTracker.map(_.start()) if (!isLocal && conf.getBoolean("spark.speculation", false)) { logInfo("Starting speculative execution thread") @@ -255,11 +255,11 @@ private[spark] class TaskSchedulerImpl private[scheduler]( } manager.parent.removeSchedulable(manager) if (success) { - blacklistTracker.taskSetSucceeded(manager.taskSet.stageId, this) + blacklistTracker.map(_.taskSetSucceeded(manager.taskSet.stageId, this)) logInfo(s"Removed TaskSet ${manager.taskSet.id}, whose tasks have all completed, from pool" + s" ${manager.parent.name}") } else { - blacklistTracker.taskSetFailed(manager.taskSet.stageId) + blacklistTracker.map(_.taskSetFailed(manager.taskSet.stageId)) logInfo(s"Removed TaskSet ${manager.taskSet.id}, since it failed, from pool" + s" ${manager.parent.name}") } @@ -277,10 +277,12 @@ private[spark] class TaskSchedulerImpl private[scheduler]( for (i <- 0 until shuffledOffers.size) { val offer = shuffledOffers(i) val host = offer.host - val nodeBlacklisted = blacklistTracker.isNodeBlacklistedForStage(host, taskSet.stageId) val execId = offer.executorId - val execBlacklisted = blacklistTracker.isExecutorBlacklistedForStage(taskSet.stageId, execId) - if (!nodeBlacklisted && !execBlacklisted && availableCpus(i) >= CPUS_PER_TASK) { + val offerBlacklisted = blacklistTracker.map { bl => + bl.isNodeBlacklistedForStage(host, taskSet.stageId) || + bl.isExecutorBlacklistedForStage(taskSet.stageId, execId) + }.getOrElse(false) + if (!offerBlacklisted && availableCpus(i) >= CPUS_PER_TASK) { try { for (task <- taskSet.resourceOffer(execId, host, maxLocality)) { tasks(i) += task @@ -302,19 +304,23 @@ private[spark] class TaskSchedulerImpl private[scheduler]( } } if (!launchedTask) { - taskSet.abortIfCompletelyBlacklisted(executorsByHost, blacklistTracker) + blacklistTracker.foreach(taskSet.abortIfCompletelyBlacklisted(executorsByHost, _)) } return launchedTask } private[scheduler] def areAllExecutorsBlacklisted(): Boolean = { - executorsByHost.foreach { case (host, execs) => - if (!blacklistTracker.isNodeBlacklisted(host) && - execs.exists(!blacklistTracker.isExecutorBlacklisted(_))) { - return false - } + blacklistTracker match { + case Some(bl) => + executorsByHost.foreach { case (host, execs) => + if (!bl.isNodeBlacklisted(host) && + execs.exists(!bl.isExecutorBlacklisted(_))) { + return false + } + } + true + case None => false } - true } /** @@ -344,13 +350,16 @@ private[spark] class TaskSchedulerImpl private[scheduler]( // ensure that we periodically check if executors can be removed from the blacklist, without // requiring a separate thread and added synchronization overhead - blacklistTracker.expireExecutorsInBlacklist() + blacklistTracker.foreach(_.expireExecutorsInBlacklist()) val sortedTaskSets = rootPool.getSortedTaskSetQueue - val filteredOffers: IndexedSeq[WorkerOffer] = offers.filter { offer => - !blacklistTracker.isNodeBlacklisted(offer.host) && - !blacklistTracker.isExecutorBlacklisted(offer.executorId) - } match { + val filteredOffers: IndexedSeq[WorkerOffer] = (blacklistTracker match { + case Some(bl) => offers.filter { offer => + !bl.isNodeBlacklisted(offer.host) && + !bl.isExecutorBlacklisted(offer.executorId) + } + case None => offers + }) match { // toIndexedSeq always makes an *immutable* IndexedSeq, though we don't care if its mutable // or immutable. So we do this to avoid making a pointless copy case is: IndexedSeq[WorkerOffer] => is @@ -517,7 +526,7 @@ private[spark] class TaskSchedulerImpl private[scheduler]( override def stop() { speculationScheduler.shutdown() - blacklistTracker.stop() + blacklistTracker.foreach(_.stop()) if (backend != null) { backend.stop() } @@ -611,7 +620,7 @@ private[spark] class TaskSchedulerImpl private[scheduler]( executorIdToHost -= executorId rootPool.executorLost(executorId, host, reason) } - blacklistTracker.removeExecutor(executorId) + blacklistTracker.foreach(_.removeExecutor(executorId)) } def executorAdded(execId: String, host: String) { @@ -721,11 +730,11 @@ private[spark] object TaskSchedulerImpl { retval.toList } - private def createBlacklistTracker(conf: SparkConf): BlacklistTracker = { + private def createBlacklistTracker(conf: SparkConf): Option[BlacklistTracker] = { if (BlacklistTracker.isBlacklistEnabled(conf)) { - new BlacklistTrackerImpl(conf) + Some(new BlacklistTracker(conf)) } else { - NoopBlacklistTracker + None } } 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 9eb715b51c3ee..320e73a884f9d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -49,7 +49,7 @@ import org.apache.spark.util.{AccumulatorV2, Clock, SystemClock, Utils} */ private[spark] class TaskSetManager( val sched: TaskSchedulerImpl, - val blacklistTracker: BlacklistTracker, + val blacklistTracker: Option[BlacklistTracker], val taskSet: TaskSet, val maxTaskFailures: Int, val clock: Clock) @@ -249,8 +249,11 @@ private[spark] class TaskSetManager( while (indexOffset > 0) { indexOffset -= 1 val index = list(indexOffset) - if (!blacklistTracker.isNodeBlacklisted(host, stageId, index) && - !blacklistTracker.isExecutorBlacklisted(execId, stageId, index)) { + val taskBlacklisted = blacklistTracker.map { bl => + bl.isNodeBlacklisted(host, stageId, index) || + bl.isExecutorBlacklisted(execId, stageId, index) + }.getOrElse(false) + if (!taskBlacklisted) { // This should almost always be list.trimEnd(1) to remove tail list.remove(indexOffset) if (copiesRunning(index) == 0 && !successful(index)) { @@ -277,10 +280,12 @@ private[spark] class TaskSetManager( { speculatableTasks.retain(index => !successful(index)) // Remove finished tasks from set - def canRunOnHost(index: Int): Boolean = - !hasAttemptOnHost(index, host) && - !blacklistTracker.isNodeBlacklisted(host, stageId, index) && - !blacklistTracker.isExecutorBlacklisted(execId, stageId, index) + def canRunOnHost(index: Int): Boolean = { + !hasAttemptOnHost(index, host) && blacklistTracker.map { bl => + !bl.isNodeBlacklisted(host, stageId, index) && + !bl.isExecutorBlacklisted(execId, stageId, index) + }.getOrElse(true) + } if (!speculatableTasks.isEmpty) { // Check for process-local tasks; note that tasks can be process-local @@ -607,14 +612,14 @@ private[spark] class TaskSetManager( pendingTask.foreach { indexInTaskSet => val stage = taskSet.stageId executorsByHost.foreach { case (host, execs) => - if (!blacklistTracker.isNodeBlacklisted(host) && - !blacklistTracker.isNodeBlacklistedForStage(host, stage) && - !blacklistTracker.isNodeBlacklisted(host, stage, indexInTaskSet)) { + if (!blacklist.isNodeBlacklisted(host) && + !blacklist.isNodeBlacklistedForStage(host, stage) && + !blacklist.isNodeBlacklisted(host, stage, indexInTaskSet)) { execs.foreach { exec => if ( - !blacklistTracker.isExecutorBlacklisted(exec) && - !blacklistTracker.isExecutorBlacklistedForStage(stage, exec) && - !blacklistTracker.isExecutorBlacklisted(exec, stage, indexInTaskSet) + !blacklist.isExecutorBlacklisted(exec) && + !blacklist.isExecutorBlacklistedForStage(stage, exec) && + !blacklist.isExecutorBlacklisted(exec, stage, indexInTaskSet) ) { // we've found some executor this task can run on. Its possible that some *other* // task isn't schedulable anywhere, but we will discover that in some later call, @@ -696,7 +701,7 @@ private[spark] class TaskSetManager( " because task " + index + " has already completed successfully") } - blacklistTracker.taskSucceeded(stageId, index, info, sched) + blacklistTracker.foreach(_.taskSucceeded(stageId, index, info, sched)) maybeFinishTaskSet() } @@ -781,7 +786,7 @@ private[spark] class TaskSetManager( // always add to failed executors // TODO if there is a fetch failure, does it really make sense to add this? - blacklistTracker.taskFailed(stageId, index, info, sched) + blacklistTracker.foreach(_.taskFailed(stageId, index, info, sched)) sched.dagScheduler.taskEnded(tasks(index), reason, null, accumUpdates, info) diff --git a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala index cf00863d86e7f..6ca16e982f231 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala @@ -44,7 +44,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M val clock = new ManualClock(0) - var blacklistTracker: BlacklistTrackerImpl = _ + var blacklistTracker: BlacklistTracker = _ override def afterEach(): Unit = { if (blacklistTracker != null) { @@ -82,7 +82,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M } // Task 1 failed on executor 1 - blacklistTracker = new BlacklistTrackerImpl(conf, clock) + blacklistTracker = new BlacklistTracker(conf, clock) blacklistTracker.taskFailed(stage1, partition1, taskInfo_1_hostA, scheduler) for { executor <- (1 to 4).map(_.toString) @@ -142,7 +142,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M assertEquivalentToSet(blacklistTracker.isExecutorBlacklisted(_), Set()) } - def trackerFixture: (BlacklistTrackerImpl, TaskSchedulerImpl) = { + def trackerFixture: (BlacklistTracker, TaskSchedulerImpl) = { val conf = new SparkConf().setAppName("test").setMaster("local") .set("spark.ui.enabled", "false") .set("spark.scheduler.blacklist.advancedStrategy", "true") @@ -154,7 +154,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M } clock.setTime(0) - blacklistTracker = new BlacklistTrackerImpl(conf, clock) + blacklistTracker = new BlacklistTracker(conf, clock) (blacklistTracker, scheduler) } 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 29ed433730a6c..ac00399721a92 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -78,7 +78,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B sc.conf.set(k, v) } taskScheduler = - new TaskSchedulerImpl(sc, sc.conf.getInt("spark.task.maxFailures", 4), blacklist) + new TaskSchedulerImpl(sc, sc.conf.getInt("spark.task.maxFailures", 4), Some(blacklist)) setupHelper() } 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 1d8a5d8fba112..7484bb1366c7a 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -425,8 +425,8 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val taskSet = FakeTask.createTaskSet(1, Seq(TaskLocation("host1", "exec1"))) val clock = new ManualClock - val blacklist = new BlacklistTrackerImpl(conf, clock) - val manager = new TaskSetManager(sched, blacklist, taskSet, 4, clock) + val blacklist = new BlacklistTracker(conf, clock) + val manager = new TaskSetManager(sched, Some(blacklist), taskSet, 4, clock) { val offerResult = manager.resourceOffer("exec1", "host1", PROCESS_LOCAL) From cf5837410818dae093ef15617cb42336a14408db Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 7 Jul 2016 16:32:09 -0500 Subject: [PATCH 05/57] dont count shuffle-fetch failures --- .../apache/spark/scheduler/TaskSetManager.scala | 10 +++++++--- .../spark/scheduler/TaskSchedulerImplSuite.scala | 16 ++++++++++++++++ .../spark/scheduler/TaskSetManagerSuite.scala | 4 ---- 3 files changed, 23 insertions(+), 7 deletions(-) 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 320e73a884f9d..e085b68a27bff 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -784,9 +784,13 @@ private[spark] class TaskSetManager( None } - // always add to failed executors - // TODO if there is a fetch failure, does it really make sense to add this? - blacklistTracker.foreach(_.taskFailed(stageId, index, info, sched)) + // we might rack up a bunch of fetch-failures in rapid succession, due to a bad node. But + // that bad node will get handled separately by spark's stage-failure handling mechanism. It + // shouldn't penalize *this* executor at all, so don't count it as a task-failure as far as + // the blacklist is concerned. + if (!reason.isInstanceOf[FetchFailed]) { + blacklistTracker.foreach(_.taskFailed(stageId, index, info, sched)) + } sched.dagScheduler.taskEnded(tasks(index), reason, null, accumUpdates, info) 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 ac00399721a92..121623c5f7736 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -24,6 +24,7 @@ import org.scalatest.mock.MockitoSugar import org.apache.spark._ import org.apache.spark.internal.Logging +import org.apache.spark.storage.BlockManagerId class FakeSchedulerBackend extends SchedulerBackend { def start() {} @@ -608,4 +609,19 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B verify(blacklist).expireExecutorsInBlacklist() } + + test("don't update blacklist for shuffle-fetch failures") { + val blacklist = mock[BlacklistTracker] + taskScheduler = setupScheduler(blacklist) + val stage0 = FakeTask.createTaskSet(numTasks = 2, stageId = 0, stageAttemptId = 0) + taskScheduler.submitTasks(stage0) + val taskDescs = taskScheduler.resourceOffers( + Seq(new WorkerOffer("executor0", "host0", 10))).flatten + assert(taskDescs.size == 2) + + val mgr = taskScheduler.taskIdToTaskSetManager.get(taskDescs(0).taskId).get + taskScheduler.handleFailedTask(mgr, taskDescs(0).taskId, TaskState.FAILED, + FetchFailed(BlockManagerId("executor1", "host1", 12345), 0, 0, 0, "ignored")) + verify(blacklist, never()).taskFailed(anyInt(), anyInt(), anyObject(), anyObject()) + } } 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 7484bb1366c7a..770dbf1e81a4b 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -869,10 +869,6 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg assert(sched.endedTasks(3) === Success) } - test("don't update blacklist for shuffle fetch failures") { - pending - } - private def createTaskResult( id: Int, accumUpdates: Seq[AccumulatorV2[_, _]] = Seq.empty): DirectTaskResult[Int] = { From 7fcb266f7816de8127807017a755d518ab5d314e Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Fri, 8 Jul 2016 11:57:42 -0500 Subject: [PATCH 06/57] make sure we clear the (node, task) blacklist on stage completion, add to test --- .../apache/spark/scheduler/BlacklistTracker.scala | 4 +++- .../spark/scheduler/BlacklistTrackerSuite.scala | 14 ++++++++++---- 2 files changed, 13 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala index a36356919220f..5a3929a97074e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala @@ -141,6 +141,7 @@ private[scheduler] class BlacklistTracker ( // blacklist the node. That is why we just remove this entry without doing any promotion to // the full app blacklist. stageIdToBlacklistedNodes.remove(stageId) + stageIdToNodeBlacklistedTasks.remove(stageId) } def taskSetFailed(stageId: Int): Unit = { @@ -148,6 +149,7 @@ private[scheduler] class BlacklistTracker ( // fine, the failures were just b/c the taskSet itself was bad (eg., bad user code) stageIdToExecToFailures.remove(stageId) stageIdToBlacklistedNodes.remove(stageId) + stageIdToNodeBlacklistedTasks.remove(stageId) } /** @@ -206,7 +208,7 @@ private[scheduler] class BlacklistTracker ( if (failures.failuresByTask.contains(indexInTaskSet)) 1 else 0 }).sum logInfo(s"total failures on host ${info.host} = $failuresOnHost") - if (failuresOnHost > MAX_TASK_FAILURES_PER_NODE) { + if (failuresOnHost >= MAX_TASK_FAILURES_PER_NODE) { stageIdToNodeBlacklistedTasks.getOrElseUpdate(stageId, new HashMap()) .getOrElseUpdate(info.host, new HashSet()) += indexInTaskSet } diff --git a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala index 6ca16e982f231..c97352a453967 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala @@ -255,6 +255,10 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M // of the internals on what to check (without the overhead of trying to trigger an OOM or // something). val (tracker, scheduler) = trackerFixture + when(scheduler.getHostForExecutor("0")).thenReturn("hostA") + when(scheduler.getHostForExecutor("1")).thenReturn("hostA") + when(scheduler.getExecutorsAliveOnHost("hostA")).thenReturn(Some(Set("0", "1"))) + // fail a couple of tasks in two stages for { stage <- 0 until 2 @@ -266,14 +270,15 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M tracker.taskFailed(stage, partition, new TaskInfo(tid, partition, 0, clock.getTimeMillis(), exec, "hostA", TaskLocality.ANY, false), scheduler) } - - when(scheduler.getHostForExecutor("0")).thenReturn("hostA") - when(scheduler.getHostForExecutor("1")).thenReturn("hostA") - when(scheduler.getExecutorsAliveOnHost("hostA")).thenReturn(Some(Set("0", "1"))) + // also want to fail one task on two different executors on the same node, to trigger + // (node, task) blacklist + tracker.taskFailed(0, 0, new TaskInfo(8, 0, 1, clock.getTimeMillis(), "1", "hostA", + TaskLocality.ANY, false), scheduler) // just make sure our test is even checking something useful -- we expect these data structures // to grow for running task sets with failed tasks assert(tracker.stageIdToExecToFailures.nonEmpty) + assert(tracker.stageIdToNodeBlacklistedTasks.nonEmpty) assert(tracker.stageIdToBlacklistedNodes.nonEmpty) // now say stage 0 fails, and stage 1 completes @@ -283,5 +288,6 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M // datastructures should be empty again assert(tracker.stageIdToExecToFailures.isEmpty) assert(tracker.stageIdToBlacklistedNodes.isEmpty) + assert(tracker.stageIdToNodeBlacklistedTasks.isEmpty) } } From 487eb66f850ba6cf68cc814d03b7daa085f591b4 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 11 Jul 2016 16:42:15 -0500 Subject: [PATCH 07/57] review feedback --- .../scala/org/apache/spark/SparkConf.scala | 4 +- .../spark/internal/config/package.scala | 43 ++++++ .../spark/scheduler/BlacklistTracker.scala | 124 ++++++++++-------- .../spark/scheduler/TaskSchedulerImpl.scala | 8 +- .../spark/scheduler/TaskSetManager.scala | 12 +- .../scheduler/BlacklistTrackerSuite.scala | 33 ++++- .../scheduler/TaskSchedulerImplSuite.scala | 14 +- .../spark/scheduler/TaskSetManagerSuite.scala | 4 +- .../spark/deploy/yarn/ApplicationMaster.scala | 7 +- 9 files changed, 163 insertions(+), 86 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 33ed0d5493e0e..db68f8e71c416 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -621,7 +621,9 @@ private[spark] object SparkConf extends Logging { "Please use spark.kryoserializer.buffer instead. The default value for " + "spark.kryoserializer.buffer.mb was previously specified as '0.064'. Fractional values " + "are no longer accepted. To specify the equivalent now, one may use '64k'."), - DeprecatedConfig("spark.rpc", "2.0", "Not used any more.") + DeprecatedConfig("spark.rpc", "2.0", "Not used any more."), + DeprecatedConfig("spark.scheduler.executorTaskBlacklistTime", "2.1.0", + "Please use the new blacklisting options, spark.scheduler.blacklist.*") ) Map(configs.map { cfg => (cfg.key -> cfg) } : _*) 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 05dd68300f891..6e0c20fcb4ee9 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 @@ -97,6 +97,49 @@ package object config { .toSequence .createWithDefault(Nil) + // Blacklist confs + private[spark] val BLACKLIST_ENABLED = + ConfigBuilder("spark.scheduler.blacklist.enabled") + .booleanConf + .createOptional + + private[spark] val MAX_TASK_FAILURES_PER_NODE = + ConfigBuilder("spark.blacklist.maxTaskFailuresPerNode") + .intConf + .createWithDefault(2) + + private[spark] val MAX_FAILURES_PER_EXEC = + ConfigBuilder("spark.blacklist.maxFailedTasksPerExecutor") + .intConf + .createWithDefault(2) + + private[spark] val MAX_FAILURES_PER_EXEC_STAGE = + ConfigBuilder("spark.blacklist.maxFailedTasksPerExecutorStage") + .intConf + .createWithDefault(2) + + private[spark] val MAX_FAILED_EXEC_PER_NODE = + ConfigBuilder("spark.blacklist.maxFailedExecutorsPerNode") + .intConf + .createWithDefault(2) + + private[spark] val MAX_FAILED_EXEC_PER_NODE_STAGE = + ConfigBuilder("spark.blacklist.maxFailedExecutorsPerNodeStage") + .intConf + .createWithDefault(2) + + private[spark] val BLACKLIST_EXPIRY_TIMEOUT_CONF = + ConfigBuilder("spark.scheduler.blacklist.recoverPeriod") + .timeConf(TimeUnit.MILLISECONDS) + .createOptional + + private[spark] val BLACKLIST_LEGACY_TIMEOUT_CONF = + ConfigBuilder("spark.scheduler.executorTaskBlacklistTime") + .internal() + .longConf + .createOptional + // End blacklist confs + // Note: This is a SQL config but needs to be in core because the REPL depends on it private[spark] val CATALOG_IMPLEMENTATION = ConfigBuilder("spark.sql.catalogImplementation") .internal() diff --git a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala index 5a3929a97074e..1cce9fdcca7f7 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala @@ -23,6 +23,7 @@ import scala.collection.mutable.{HashMap, HashSet} import org.apache.spark.SparkConf import org.apache.spark.internal.Logging +import org.apache.spark.internal.config import org.apache.spark.util.Clock import org.apache.spark.util.SystemClock import org.apache.spark.util.Utils @@ -47,16 +48,11 @@ private[scheduler] class BlacklistTracker ( conf: SparkConf, clock: Clock = new SystemClock()) extends Logging { - private val MAX_TASK_FAILURES_PER_NODE = - conf.getInt("spark.blacklist.maxTaskFailuresPerNode", 2) - private val MAX_FAILURES_PER_EXEC = - conf.getInt("spark.blacklist.maxFailedTasksPerExecutor", 2) - private val MAX_FAILURES_PER_EXEC_STAGE = - conf.getInt("spark.blacklist.maxFailedTasksPerExecutorStage", 2) - private val MAX_FAILED_EXEC_PER_NODE = - conf.getInt("spark.blacklist.maxFailedExecutorsPerNode", 2) - private val MAX_FAILED_EXEC_PER_NODE_STAGE = - conf.getInt("spark.blacklist.maxFailedExecutorsPerNodeStage", 2) + private val MAX_TASK_FAILURES_PER_NODE = conf.get(config.MAX_TASK_FAILURES_PER_NODE) + private val MAX_FAILURES_PER_EXEC = conf.get(config.MAX_FAILURES_PER_EXEC) + private val MAX_FAILURES_PER_EXEC_STAGE = conf.get(config.MAX_FAILURES_PER_EXEC_STAGE) + private val MAX_FAILED_EXEC_PER_NODE = conf.get(config.MAX_FAILED_EXEC_PER_NODE) + private val MAX_FAILED_EXEC_PER_NODE_STAGE = conf.get(config.MAX_FAILED_EXEC_PER_NODE_STAGE) val EXECUTOR_RECOVERY_MILLIS = BlacklistTracker.getBlacklistExpiryTime(conf) // a count of failed tasks for each executor. Only counts failures after tasksets complete @@ -154,7 +150,9 @@ private[scheduler] class BlacklistTracker ( /** * Return true if this executor is blacklisted for the given stage. Completely ignores whether - * the executor is blacklisted overall (or anything to do with the node the executor is on). + * the executor is blacklisted overall (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 isExecutorBlacklistedForStage( stageId: Int, @@ -171,6 +169,10 @@ private[scheduler] class BlacklistTracker ( stageIdToBlacklistedNodes.get(stageId).map(_.contains(node)).getOrElse(false) } + /** + * 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() } @@ -185,7 +187,7 @@ private[scheduler] class BlacklistTracker ( info: TaskInfo, scheduler: TaskSchedulerImpl): Unit = { // no-op intentionally, included just for symmetry. success to failure ratio is irrelevant, we - // just blacklist based on failures. Furthermore, one success does not previous + // just blacklist based on failures. Furthermore, one success does not clear previous // failures, since the bad node / executor may not fail *every* time } @@ -197,7 +199,7 @@ private[scheduler] class BlacklistTracker ( val stageFailures = stageIdToExecToFailures.getOrElseUpdate(stageId, new HashMap()) val failureStatus = stageFailures.getOrElseUpdate(info.executorId, new FailureStatus()) failureStatus.totalFailures += 1 - failureStatus.failuresByTask += indexInTaskSet + failureStatus.tasksWithFailures += indexInTaskSet // check if this task has also failed on other executors on the same host, and if so, blacklist // this task from the host @@ -205,9 +207,8 @@ private[scheduler] class BlacklistTracker ( exec <- scheduler.getExecutorsAliveOnHost(info.host).getOrElse(Set()).toSeq failures <- stageFailures.get(exec) } yield { - if (failures.failuresByTask.contains(indexInTaskSet)) 1 else 0 + if (failures.tasksWithFailures.contains(indexInTaskSet)) 1 else 0 }).sum - logInfo(s"total failures on host ${info.host} = $failuresOnHost") if (failuresOnHost >= MAX_TASK_FAILURES_PER_NODE) { stageIdToNodeBlacklistedTasks.getOrElseUpdate(stageId, new HashMap()) .getOrElseUpdate(info.host, new HashSet()) += indexInTaskSet @@ -215,10 +216,10 @@ private[scheduler] class BlacklistTracker ( if (failureStatus.totalFailures >= MAX_FAILURES_PER_EXEC_STAGE) { - // this executor has been pushed into the blacklist for this stage. Lets check if it pushes + // This executor has been pushed into the blacklist for this stage. Let's check if it pushes // the whole node into the blacklist val blacklistedExecutors = - stageFailures.filter{_._2.totalFailures >= MAX_FAILURES_PER_EXEC_STAGE} + stageFailures.filter(_._2.totalFailures >= MAX_FAILURES_PER_EXEC_STAGE) if (blacklistedExecutors.size >= MAX_FAILED_EXEC_PER_NODE_STAGE) { logInfo(s"Blacklisting ${info.host} for stage $stageId") stageIdToBlacklistedNodes.getOrElseUpdate(stageId, new HashSet()) += info.host @@ -229,34 +230,31 @@ private[scheduler] class BlacklistTracker ( /** * 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 - * altogether. + * altogether. 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 isExecutorBlacklisted( + def isExecutorBlacklistedForTask( executorId: String, stageId: Int, indexInTaskSet: Int): Boolean = { - // intentionally avoiding .getOrElse(..., new HashMap()) to avoid lots of object - // creation, since this method gets called a *lot* - stageIdToExecToFailures.get(stageId) match { - case Some(stageFailures) => - stageFailures.get(executorId) match { - case Some(failures) => - failures.failuresByTask.contains(indexInTaskSet) - case None => - false - } - case None => - false - } + stageIdToExecToFailures.get(stageId) + .flatMap(_.get(executorId)) + .map(_.tasksWithFailures.contains(indexInTaskSet)) + .getOrElse(false) } - def isNodeBlacklisted( + def isNodeBlacklistedForTask( node: String, stageId: Int, indexInTaskSet: Int): Boolean = { - stageIdToNodeBlacklistedTasks.get(stageId).flatMap { nodeToFailures => - nodeToFailures.get(node).map{_.contains(indexInTaskSet)} - }.getOrElse(false) + val n = stageIdToNodeBlacklistedTasks.get(stageId) + .flatMap(_.get(node)) + .map(_.contains(indexInTaskSet)) + .getOrElse(false) + if (n) { + logInfo(s"blacklisting $stageId, $indexInTaskSet on node $node") + } + n } def removeExecutor(executorId: String): Unit = { @@ -270,9 +268,6 @@ private[scheduler] class BlacklistTracker ( private[scheduler] object BlacklistTracker extends Logging { - val LEGACY_TIMEOUT_CONF = "spark.scheduler.executorTaskBlacklistTime" - val EXPIRY_TIMEOUT_CONF = "spark.scheduler.blacklist.recoverPeriod" - val ENABLED_CONF = "spark.scheduler.blacklist.enabled" /** * Return true if the blacklist is enabled, based on the following order of preferences: @@ -283,40 +278,53 @@ private[scheduler] object BlacklistTracker extends Logging { * - on for distributed modes (including local-cluster) */ def isBlacklistEnabled(conf: SparkConf): Boolean = { - val isEnabled = conf.get(ENABLED_CONF, null) - if (isEnabled == null) { + conf.get(config.BLACKLIST_ENABLED) match { + case Some(isEnabled) => + isEnabled + case None => // if they've got a non-zero setting for the legacy conf, always enable the blacklist, // otherwise, use the default based on the cluster-mode (off for local-mode, on otherwise). - val legacyTimeout = conf.getLong(LEGACY_TIMEOUT_CONF, 0L) - if (legacyTimeout > 0) { - // mostly this is necessary just for tests, since real users that want the blacklist will - // get it anyway by default - logWarning(s"Turning on blacklisting due to legacy configuration: $LEGACY_TIMEOUT_CONF > 0") - true - } else { - // local-cluster is *not* considered local for these purposes, we still want the blacklist - // enabled by default - !Utils.isLocalMaster(conf) + val legacyKey = config.BLACKLIST_LEGACY_TIMEOUT_CONF.key + conf.getOption(legacyKey) match { + case Some(legacyTimeout) => + if (legacyTimeout.toLong > 0) { + // mostly this is necessary just for tests, since real users that want the blacklist + // will get it anyway by default + logWarning(s"Turning on blacklisting due to legacy configuration:" + + s" $legacyKey > 0") + true + } else { + logWarning(s"Turning off blacklisting due to legacy configuaration:" + + s" $legacyKey == 0") + false + } + case None => + // local-cluster is *not* considered local for these purposes, we still want the blacklist + // enabled by default + !Utils.isLocalMaster(conf) } - } else { - // always take whatever value is explicitly set by the user - isEnabled.toBoolean } } def getBlacklistExpiryTime(conf: SparkConf): Long = { - conf.getTimeAsMs(BlacklistTracker.EXPIRY_TIMEOUT_CONF, - conf.get(BlacklistTracker.LEGACY_TIMEOUT_CONF, (60 * 60 * 1000).toString)) + val timeoutConf = conf.get(config.BLACKLIST_EXPIRY_TIMEOUT_CONF) + val legacyTimeoutConf = conf.get(config.BLACKLIST_LEGACY_TIMEOUT_CONF) + (timeoutConf, legacyTimeoutConf) match { + case (Some(x), _) => x + case (None, Some(y)) => y + case (None, None) => + Utils.timeStringAsMs("1h") + } } } /** Failures for one executor, within one taskset */ private[scheduler] final class FailureStatus { /** index of the tasks in the taskset that have failed on this executor. */ - val failuresByTask = HashSet[Int]() + val tasksWithFailures = HashSet[Int]() var totalFailures = 0 override def toString(): String = { - s"totalFailures = $totalFailures; tasksFailed = $failuresByTask" + s"totalFailures = $totalFailures; tasksFailed = $tasksWithFailures" } } 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 d3e13a3760f43..ec5913a097b49 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -66,7 +66,7 @@ private[spark] class TaskSchedulerImpl private[scheduler]( def this(sc: SparkContext, maxTaskFailures: Int, isLocal: Boolean) = { this(sc, maxTaskFailures, TaskSchedulerImpl.createBlacklistTracker(sc.conf), - clock = new SystemClock, isLocal) + clock = new SystemClock, isLocal = isLocal) } val conf = sc.conf @@ -162,7 +162,7 @@ private[spark] class TaskSchedulerImpl private[scheduler]( override def start() { backend.start() - blacklistTracker.map(_.start()) + blacklistTracker.foreach(_.start()) if (!isLocal && conf.getBoolean("spark.speculation", false)) { logInfo("Starting speculative execution thread") @@ -255,11 +255,11 @@ private[spark] class TaskSchedulerImpl private[scheduler]( } manager.parent.removeSchedulable(manager) if (success) { - blacklistTracker.map(_.taskSetSucceeded(manager.taskSet.stageId, this)) + blacklistTracker.foreach(_.taskSetSucceeded(manager.taskSet.stageId, this)) logInfo(s"Removed TaskSet ${manager.taskSet.id}, whose tasks have all completed, from pool" + s" ${manager.parent.name}") } else { - blacklistTracker.map(_.taskSetFailed(manager.taskSet.stageId)) + blacklistTracker.foreach(_.taskSetFailed(manager.taskSet.stageId)) logInfo(s"Removed TaskSet ${manager.taskSet.id}, since it failed, from pool" + s" ${manager.parent.name}") } 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 e085b68a27bff..6da40e9a24b2a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -250,8 +250,8 @@ private[spark] class TaskSetManager( indexOffset -= 1 val index = list(indexOffset) val taskBlacklisted = blacklistTracker.map { bl => - bl.isNodeBlacklisted(host, stageId, index) || - bl.isExecutorBlacklisted(execId, stageId, index) + bl.isNodeBlacklistedForTask(host, stageId, index) || + bl.isExecutorBlacklistedForTask(execId, stageId, index) }.getOrElse(false) if (!taskBlacklisted) { // This should almost always be list.trimEnd(1) to remove tail @@ -282,8 +282,8 @@ private[spark] class TaskSetManager( def canRunOnHost(index: Int): Boolean = { !hasAttemptOnHost(index, host) && blacklistTracker.map { bl => - !bl.isNodeBlacklisted(host, stageId, index) && - !bl.isExecutorBlacklisted(execId, stageId, index) + !bl.isNodeBlacklistedForTask(host, stageId, index) && + !bl.isExecutorBlacklistedForTask(execId, stageId, index) }.getOrElse(true) } @@ -614,12 +614,12 @@ private[spark] class TaskSetManager( executorsByHost.foreach { case (host, execs) => if (!blacklist.isNodeBlacklisted(host) && !blacklist.isNodeBlacklistedForStage(host, stage) && - !blacklist.isNodeBlacklisted(host, stage, indexInTaskSet)) { + !blacklist.isNodeBlacklistedForTask(host, stage, indexInTaskSet)) { execs.foreach { exec => if ( !blacklist.isExecutorBlacklisted(exec) && !blacklist.isExecutorBlacklistedForStage(stage, exec) && - !blacklist.isExecutorBlacklisted(exec, stage, indexInTaskSet) + !blacklist.isExecutorBlacklistedForTask(exec, stage, indexInTaskSet) ) { // we've found some executor this task can run on. Its possible that some *other* // task isn't schedulable anywhere, but we will discover that in some later call, diff --git a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala index c97352a453967..8168fd6852587 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala @@ -23,6 +23,7 @@ import org.scalatest.mock.MockitoSugar import org.apache.spark.SparkConf import org.apache.spark.SparkFunSuite +import org.apache.spark.internal.config.{BLACKLIST_EXPIRY_TIMEOUT_CONF, BLACKLIST_LEGACY_TIMEOUT_CONF} import org.apache.spark.util.ManualClock class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with MockitoSugar { @@ -73,7 +74,6 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M test("Blacklisting individual tasks") { val conf = new SparkConf().setAppName("test").setMaster("local") .set("spark.ui.enabled", "false") - .set("spark.scheduler.blacklist.advancedStrategy", "true") .set("spark.scheduler.executorTaskBlacklistTime", "1000") val scheduler = mock[TaskSchedulerImpl] when(scheduler.getExecutorsAliveOnHost("hostA")).thenReturn(Some(Set("1", "2", "4"))) @@ -90,7 +90,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M stage <- (1 to 2) } { val exp = (executor == "1" && stage == stage1 && partition == 1) - assert(blacklistTracker.isExecutorBlacklisted(executor, stage, partition) === exp) + assert(blacklistTracker.isExecutorBlacklistedForTask(executor, stage, partition) === exp) } assert(blacklistTracker.nodeBlacklist() === Set()) assertEquivalentToSet(blacklistTracker.isNodeBlacklisted(_), Set()) @@ -116,7 +116,8 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M val badExec = (executor == "1" || executor == "2") val badPart = (partition == 1 || partition == 2) val taskExp = (badExec && stage == stage1 && badPart) - assert(blacklistTracker.isExecutorBlacklisted(executor, stage, partition) === taskExp) + assert( + blacklistTracker.isExecutorBlacklistedForTask(executor, stage, partition) === taskExp) val executorExp = badExec && stage == stage1 assert(blacklistTracker.isExecutorBlacklistedForStage(stage, executor) === executorExp) } @@ -145,7 +146,6 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M def trackerFixture: (BlacklistTracker, TaskSchedulerImpl) = { val conf = new SparkConf().setAppName("test").setMaster("local") .set("spark.ui.enabled", "false") - .set("spark.scheduler.blacklist.advancedStrategy", "true") .set("spark.scheduler.executorTaskBlacklistTime", "1000") val scheduler = mock[TaskSchedulerImpl] when(scheduler.getExecutorsAliveOnHost("hostA")).thenReturn(Some(Set("1", "2", "4"))) @@ -290,4 +290,29 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M assert(tracker.stageIdToBlacklistedNodes.isEmpty) assert(tracker.stageIdToNodeBlacklistedTasks.isEmpty) } + + test("blacklist still respects legacy configs") { + val legacyKey = BLACKLIST_LEGACY_TIMEOUT_CONF.key + + { + val localConf = new SparkConf().setMaster("local") + assert(!BlacklistTracker.isBlacklistEnabled(localConf)) + localConf.set(legacyKey, "5000") + assert(BlacklistTracker.isBlacklistEnabled(localConf)) + assert(5000 == BlacklistTracker.getBlacklistExpiryTime(localConf)) + + localConf.set(legacyKey, "0") + assert(!BlacklistTracker.isBlacklistEnabled(localConf)) + } + + { + val distConf = new SparkConf().setMaster("yarn-cluster") + assert(BlacklistTracker.isBlacklistEnabled(distConf)) + assert(60 * 60 * 1000L == BlacklistTracker.getBlacklistExpiryTime(distConf)) + distConf.set(legacyKey, "5000") + assert(5000 == BlacklistTracker.getBlacklistExpiryTime(distConf)) + distConf.set(BLACKLIST_EXPIRY_TIMEOUT_CONF.key, "10h") + assert(10 * 60 * 60 * 1000L == BlacklistTracker.getBlacklistExpiryTime(distConf)) + } + } } 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 121623c5f7736..f67275d3d63dc 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -319,7 +319,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B // setup our mock blacklist: // stage 0 is blacklisted on node "host1" - // stage 1 is blacklist on executor "executor3" + // stage 1 is blacklisted on executor "executor3" // stage 0, part 0 is blacklisted on executor 0 // (later stubs take precedence over earlier ones) when(blacklist.isNodeBlacklisted(anyString())).thenReturn(false) @@ -328,8 +328,8 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B when(blacklist.isNodeBlacklistedForStage("host1", 0)).thenReturn(true) when(blacklist.isExecutorBlacklistedForStage(anyInt(), anyString())).thenReturn(false) when(blacklist.isExecutorBlacklistedForStage(1, "executor3")).thenReturn(true) - when(blacklist.isExecutorBlacklisted(anyString(), anyInt(), anyInt())).thenReturn(false) - when(blacklist.isExecutorBlacklisted("executor0", 0, 0)).thenReturn(true) + when(blacklist.isExecutorBlacklistedForTask(anyString(), anyInt(), anyInt())).thenReturn(false) + when(blacklist.isExecutorBlacklistedForTask("executor0", 0, 0)).thenReturn(true) val firstTaskAttempts = taskScheduler.resourceOffers(offers).flatten // these verifications are tricky b/c we reference them multiple times -- also invoked when we @@ -348,13 +348,13 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B // the node blacklist should ensure we never check the task blacklist. This is important // for performance, otherwise we end up changing an O(1) operation into a // O(numPendingTasks) one - verify(blacklist, never).isExecutorBlacklisted(exec, 0, part) + verify(blacklist, never).isExecutorBlacklistedForTask(exec, 0, part) } // similarly, the executor blacklist for an entire stage should prevent us from ever checking // the blacklist for specific parts in a stage. (0 to 1).foreach { part => - verify(blacklist, never).isExecutorBlacklisted("executor3", 1, part) + verify(blacklist, never).isExecutorBlacklistedForTask("executor3", 1, part) } // we should schedule all tasks. @@ -444,7 +444,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B when(blacklist.isExecutorBlacklisted("executor3")).thenReturn(true) when(blacklist.isNodeBlacklistedForStage(anyString(), anyInt())).thenReturn(false) when(blacklist.isExecutorBlacklistedForStage(anyInt(), anyString())).thenReturn(false) - when(blacklist.isExecutorBlacklisted(anyString(), anyInt(), anyInt())).thenReturn(false) + when(blacklist.isExecutorBlacklistedForTask(anyString(), anyInt(), anyInt())).thenReturn(false) val stageToTsm = (0 to 2).map { stageId => val tsm = taskScheduler.taskSetManagerForAttempt(stageId, 0).get @@ -459,7 +459,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B } verify(blacklist, never()).isNodeBlacklistedForStage(anyString(), anyInt()) verify(blacklist, never()).isExecutorBlacklistedForStage(anyInt(), anyString()) - verify(blacklist, never()).isExecutorBlacklisted(anyString(), anyInt(), anyInt()) + verify(blacklist, never()).isExecutorBlacklistedForTask(anyString(), anyInt(), anyInt()) // we should have aborted the existing stages, since they aren't schedulable (0 to 2).foreach { stageId => 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 770dbf1e81a4b..18165d4f53e9e 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -490,12 +490,12 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg } { - val offerResult = manager.resourceOffer("exec3", "host1", NODE_LOCAL) + val offerResult = manager.resourceOffer("exec3", "host3", ANY) assert(offerResult.isDefined) assert(offerResult.get.index === 0) assert(offerResult.get.executorId === "exec3") - assert(manager.resourceOffer("exec3", "host1", NODE_LOCAL).isEmpty) + assert(manager.resourceOffer("exec3", "host3", ANY).isEmpty) // Cause exec3 to fail : failure 4 manager.handleFailedTask(offerResult.get.taskId, TaskState.FINISHED, TaskResultLost) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index f2fa5b44cd638..80ec78e1f0a90 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -682,12 +682,11 @@ private[spark] class ApplicationMaster( } override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { - case RequestExecutors( - requestedTotal, localityAwareTasks, hostToLocalTaskCount, nodeBlacklist) => + case r: RequestExecutors => Option(allocator) match { case Some(a) => - if (a.requestTotalExecutorsWithPreferredLocalities(requestedTotal, - localityAwareTasks, hostToLocalTaskCount, nodeBlacklist)) { + if (a.requestTotalExecutorsWithPreferredLocalities(r.requestedTotal, + r.localityAwareTasks, r.hostToLocalTaskCount, r.nodeBlacklist)) { resetAllocatorInterval() } context.reply(true) From c22aaad76f07cbe58ea455d18959470e7afb1498 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 11 Jul 2016 20:55:39 -0500 Subject: [PATCH 08/57] fix --- .../apache/spark/scheduler/cluster/YarnSchedulerBackend.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala index 31fafbc2a586b..a5a66caa13b08 100644 --- a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala +++ b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala @@ -126,7 +126,8 @@ private[spark] abstract class YarnSchedulerBackend( */ override def doRequestTotalExecutors(requestedTotal: Int): Boolean = { - val nodeBlacklist: Set[String] = scheduler.blacklistTracker.nodeBlacklist() + val nodeBlacklist: Set[String] = + scheduler.blacklistTracker.map(_.nodeBlacklist()).getOrElse(Set()) yarnSchedulerEndpointRef.askWithRetry[Boolean]( RequestExecutors(requestedTotal, localityAwareTasks, hostToLocalTaskCount, nodeBlacklist)) From dc2b3ed4cc256fdb588947bf293d7517c754c45b Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 13 Jul 2016 06:28:49 -0500 Subject: [PATCH 09/57] all taskset specific blacklisting is now in TaskSetManager --- .../spark/scheduler/BlacklistTracker.scala | 165 +++--------------- .../spark/scheduler/TaskSchedulerImpl.scala | 6 +- .../spark/scheduler/TaskSetManager.scala | 95 +++++++++- .../scheduler/BlacklistTrackerSuite.scala | 150 ++++++---------- .../scheduler/TaskSchedulerImplSuite.scala | 101 ++++++----- 5 files changed, 235 insertions(+), 282 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala index 1cce9fdcca7f7..065a336c115dc 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala @@ -48,22 +48,13 @@ private[scheduler] class BlacklistTracker ( conf: SparkConf, clock: Clock = new SystemClock()) extends Logging { - private val MAX_TASK_FAILURES_PER_NODE = conf.get(config.MAX_TASK_FAILURES_PER_NODE) private val MAX_FAILURES_PER_EXEC = conf.get(config.MAX_FAILURES_PER_EXEC) - private val MAX_FAILURES_PER_EXEC_STAGE = conf.get(config.MAX_FAILURES_PER_EXEC_STAGE) private val MAX_FAILED_EXEC_PER_NODE = conf.get(config.MAX_FAILED_EXEC_PER_NODE) - private val MAX_FAILED_EXEC_PER_NODE_STAGE = conf.get(config.MAX_FAILED_EXEC_PER_NODE_STAGE) val EXECUTOR_RECOVERY_MILLIS = BlacklistTracker.getBlacklistExpiryTime(conf) // a count of failed tasks for each executor. Only counts failures after tasksets complete // successfully private val executorIdToFailureCount: HashMap[String, Int] = HashMap() - // failures for each executor by stage. Only tracked while the stage is running. - val stageIdToExecToFailures: HashMap[Int, HashMap[String, FailureStatus]] = - new HashMap() - val stageIdToNodeBlacklistedTasks: HashMap[Int, HashMap[String, HashSet[Int]]] = - new HashMap() - val stageIdToBlacklistedNodes: HashMap[Int, HashSet[String]] = new HashMap() private val executorIdToBlacklistExpiryTime: HashMap[String, Long] = new HashMap() private val nodeIdToBlacklistExpiryTime: HashMap[String, Long] = new HashMap() private val _nodeBlacklist: AtomicReference[Set[String]] = new AtomicReference(Set()) @@ -97,78 +88,49 @@ private[scheduler] class BlacklistTracker ( } } - def taskSetSucceeded(stageId: Int, scheduler: TaskSchedulerImpl): Unit = { + def taskSetSucceeded( + failuresByExec: HashMap[String, FailureStatus], + scheduler: TaskSchedulerImpl): Unit = { // if any tasks failed, we count them towards the overall failure count for the executor at - // this point. Also clean out all data about the stage to avoid increasing memory use. - stageIdToExecToFailures.remove(stageId).map { failuresForStage => - failuresForStage.foreach { case (exec, newFailures) => - val prevFailures = executorIdToFailureCount.getOrElse(exec, 0) - val newTotal = prevFailures + newFailures.totalFailures - - if (newTotal >= MAX_FAILURES_PER_EXEC) { - logInfo(s"Blacklisting executor $exec because it has $newTotal" + - s" task failures in successful task sets") - val now = clock.getTimeMillis() - val expiryTime = now + EXECUTOR_RECOVERY_MILLIS - executorIdToBlacklistExpiryTime.put(exec, expiryTime) - executorIdToFailureCount.remove(exec) - if (expiryTime < nextExpiryTime) { - nextExpiryTime = expiryTime - } + // this point. + failuresByExec.foreach { case (exec, newFailures) => + val prevFailures = executorIdToFailureCount.getOrElse(exec, 0) + val newTotal = prevFailures + newFailures.totalFailures + + if (newTotal >= MAX_FAILURES_PER_EXEC) { + logInfo(s"Blacklisting executor $exec because it has $newTotal" + + s" task failures in successful task sets") + val now = clock.getTimeMillis() + val expiryTime = now + EXECUTOR_RECOVERY_MILLIS + executorIdToBlacklistExpiryTime.put(exec, expiryTime) + executorIdToFailureCount.remove(exec) + if (expiryTime < nextExpiryTime) { + nextExpiryTime = expiryTime + } - val node = scheduler.getHostForExecutor(exec) - val execs = scheduler.getExecutorsAliveOnHost(node).getOrElse(Set()) - val blacklistedExecs = execs.filter(executorIdToBlacklistExpiryTime.contains(_)) - if (blacklistedExecs.size >= MAX_FAILED_EXEC_PER_NODE) { - logInfo(s"Blacklisting node $node because it has ${blacklistedExecs.size} executors " + - s"blacklisted: ${blacklistedExecs}") - nodeIdToBlacklistExpiryTime.put(node, expiryTime) - // make a copy of the blacklisted nodes so nodeBlacklist() is threadsafe - _nodeBlacklist.set(nodeIdToBlacklistExpiryTime.keySet.toSet) - } - } else { - executorIdToFailureCount.put(exec, newTotal) + val node = scheduler.getHostForExecutor(exec) + val execs = scheduler.getExecutorsAliveOnHost(node).getOrElse(Set()) + val blacklistedExecs = execs.filter(executorIdToBlacklistExpiryTime.contains(_)) + if (blacklistedExecs.size >= MAX_FAILED_EXEC_PER_NODE) { + logInfo(s"Blacklisting node $node because it has ${blacklistedExecs.size} executors " + + s"blacklisted: ${blacklistedExecs}") + nodeIdToBlacklistExpiryTime.put(node, expiryTime) + // make a copy of the blacklisted nodes so nodeBlacklist() is threadsafe + _nodeBlacklist.set(nodeIdToBlacklistExpiryTime.keySet.toSet) } + } else { + executorIdToFailureCount.put(exec, newTotal) } } - // when we blacklist a node within a stage, we don't directly promote that node to being - // blacklisted for the app. Instead, we use the mechanism above to decide whether or not to - // blacklist any executors for the app, and when doing so we'll check whether or not to also - // blacklist the node. That is why we just remove this entry without doing any promotion to - // the full app blacklist. - stageIdToBlacklistedNodes.remove(stageId) - stageIdToNodeBlacklistedTasks.remove(stageId) } def taskSetFailed(stageId: Int): Unit = { - // just throw away all the info for the failures in this taskSet -- assume the executors were - // fine, the failures were just b/c the taskSet itself was bad (eg., bad user code) - stageIdToExecToFailures.remove(stageId) - stageIdToBlacklistedNodes.remove(stageId) - stageIdToNodeBlacklistedTasks.remove(stageId) - } - - /** - * Return true if this executor is blacklisted for the given stage. Completely ignores whether - * the executor is blacklisted overall (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 isExecutorBlacklistedForStage( - stageId: Int, - executorId: String): Boolean = { - stageIdToExecToFailures.get(stageId).flatMap(_.get(executorId)) - .map(_.totalFailures >= MAX_FAILURES_PER_EXEC_STAGE).getOrElse(false) } def isExecutorBlacklisted(executorId: String): Boolean = { executorIdToBlacklistExpiryTime.contains(executorId) } - def isNodeBlacklistedForStage(node: String, stageId: Int): Boolean = { - stageIdToBlacklistedNodes.get(stageId).map(_.contains(node)).getOrElse(false) - } - /** * 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. @@ -191,78 +153,9 @@ private[scheduler] class BlacklistTracker ( // failures, since the bad node / executor may not fail *every* time } - def taskFailed( - stageId: Int, - indexInTaskSet: Int, - info: TaskInfo, - scheduler: TaskSchedulerImpl): Unit = { - val stageFailures = stageIdToExecToFailures.getOrElseUpdate(stageId, new HashMap()) - val failureStatus = stageFailures.getOrElseUpdate(info.executorId, new FailureStatus()) - failureStatus.totalFailures += 1 - failureStatus.tasksWithFailures += indexInTaskSet - - // check if this task has also failed on other executors on the same host, and if so, blacklist - // this task from the host - val failuresOnHost = (for { - exec <- scheduler.getExecutorsAliveOnHost(info.host).getOrElse(Set()).toSeq - failures <- stageFailures.get(exec) - } yield { - if (failures.tasksWithFailures.contains(indexInTaskSet)) 1 else 0 - }).sum - if (failuresOnHost >= MAX_TASK_FAILURES_PER_NODE) { - stageIdToNodeBlacklistedTasks.getOrElseUpdate(stageId, new HashMap()) - .getOrElseUpdate(info.host, new HashSet()) += indexInTaskSet - } - - - if (failureStatus.totalFailures >= MAX_FAILURES_PER_EXEC_STAGE) { - // This executor has been pushed into the blacklist for this stage. Let's check if it pushes - // the whole node into the blacklist - val blacklistedExecutors = - stageFailures.filter(_._2.totalFailures >= MAX_FAILURES_PER_EXEC_STAGE) - if (blacklistedExecutors.size >= MAX_FAILED_EXEC_PER_NODE_STAGE) { - logInfo(s"Blacklisting ${info.host} for stage $stageId") - stageIdToBlacklistedNodes.getOrElseUpdate(stageId, new HashSet()) += info.host - } - } - } - - /** - * 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 - * altogether. 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, - stageId: Int, - indexInTaskSet: Int): Boolean = { - stageIdToExecToFailures.get(stageId) - .flatMap(_.get(executorId)) - .map(_.tasksWithFailures.contains(indexInTaskSet)) - .getOrElse(false) - } - - def isNodeBlacklistedForTask( - node: String, - stageId: Int, - indexInTaskSet: Int): Boolean = { - val n = stageIdToNodeBlacklistedTasks.get(stageId) - .flatMap(_.get(node)) - .map(_.contains(indexInTaskSet)) - .getOrElse(false) - if (n) { - logInfo(s"blacklisting $stageId, $indexInTaskSet on node $node") - } - n - } - def removeExecutor(executorId: String): Unit = { executorIdToBlacklistExpiryTime -= executorId executorIdToFailureCount -= executorId - stageIdToExecToFailures.values.foreach { execFailureOneStage => - execFailureOneStage -= executorId - } } } 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 ec5913a097b49..d0dc92f9c0416 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -255,7 +255,7 @@ private[spark] class TaskSchedulerImpl private[scheduler]( } manager.parent.removeSchedulable(manager) if (success) { - blacklistTracker.foreach(_.taskSetSucceeded(manager.taskSet.stageId, this)) + blacklistTracker.foreach(_.taskSetSucceeded(manager.execToFailures, this)) logInfo(s"Removed TaskSet ${manager.taskSet.id}, whose tasks have all completed, from pool" + s" ${manager.parent.name}") } else { @@ -279,8 +279,8 @@ private[spark] class TaskSchedulerImpl private[scheduler]( val host = offer.host val execId = offer.executorId val offerBlacklisted = blacklistTracker.map { bl => - bl.isNodeBlacklistedForStage(host, taskSet.stageId) || - bl.isExecutorBlacklistedForStage(taskSet.stageId, execId) + taskSet.isNodeBlacklistedForTaskSet(host) || + taskSet.isExecutorBlacklistedForTaskSet(execId) }.getOrElse(false) if (!offerBlacklisted && availableCpus(i) >= CPUS_PER_TASK) { try { 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 6da40e9a24b2a..c214258827878 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -28,6 +28,7 @@ import scala.util.control.NonFatal import org.apache.spark._ import org.apache.spark.internal.Logging +import org.apache.spark.internal.config import org.apache.spark.scheduler.SchedulingMode._ import org.apache.spark.TaskState.TaskState import org.apache.spark.util.{AccumulatorV2, Clock, SystemClock, Utils} @@ -64,6 +65,9 @@ private[spark] class TaskSetManager( } val conf = sched.sc.conf + private val MAX_TASK_FAILURES_PER_NODE = conf.get(config.MAX_TASK_FAILURES_PER_NODE) + private val MAX_FAILURES_PER_EXEC_STAGE = conf.get(config.MAX_FAILURES_PER_EXEC_STAGE) + private val MAX_FAILED_EXEC_PER_NODE_STAGE = conf.get(config.MAX_FAILED_EXEC_PER_NODE_STAGE) // Quantile of tasks at which to start speculation val SPECULATION_QUANTILE = conf.getDouble("spark.speculation.quantile", 0.75) @@ -81,6 +85,10 @@ private[spark] class TaskSetManager( val copiesRunning = new Array[Int](numTasks) val successful = new Array[Boolean](numTasks) private val numFailures = new Array[Int](numTasks) + val execToFailures: HashMap[String, FailureStatus] = new HashMap() + val nodeBlacklistedTasks: HashMap[String, HashSet[Int]] = new HashMap() + val blacklistedNodes: HashSet[String] = new HashSet() + val taskAttempts = Array.fill[List[TaskInfo]](numTasks)(Nil) var tasksSuccessful = 0 @@ -250,8 +258,8 @@ private[spark] class TaskSetManager( indexOffset -= 1 val index = list(indexOffset) val taskBlacklisted = blacklistTracker.map { bl => - bl.isNodeBlacklistedForTask(host, stageId, index) || - bl.isExecutorBlacklistedForTask(execId, stageId, index) + isNodeBlacklistedForTask(host, index) || + isExecutorBlacklistedForTask(execId, index) }.getOrElse(false) if (!taskBlacklisted) { // This should almost always be list.trimEnd(1) to remove tail @@ -282,8 +290,8 @@ private[spark] class TaskSetManager( def canRunOnHost(index: Int): Boolean = { !hasAttemptOnHost(index, host) && blacklistTracker.map { bl => - !bl.isNodeBlacklistedForTask(host, stageId, index) && - !bl.isExecutorBlacklistedForTask(execId, stageId, index) + !isNodeBlacklistedForTask(host, index) && + !isExecutorBlacklistedForTask(execId, index) }.getOrElse(true) } @@ -613,13 +621,13 @@ private[spark] class TaskSetManager( val stage = taskSet.stageId executorsByHost.foreach { case (host, execs) => if (!blacklist.isNodeBlacklisted(host) && - !blacklist.isNodeBlacklistedForStage(host, stage) && - !blacklist.isNodeBlacklistedForTask(host, stage, indexInTaskSet)) { + !isNodeBlacklistedForTaskSet(host) && + !isNodeBlacklistedForTask(host, indexInTaskSet)) { execs.foreach { exec => if ( !blacklist.isExecutorBlacklisted(exec) && - !blacklist.isExecutorBlacklistedForStage(stage, exec) && - !blacklist.isExecutorBlacklistedForTask(exec, stage, indexInTaskSet) + !isExecutorBlacklistedForTaskSet(exec) && + !isExecutorBlacklistedForTask(exec, indexInTaskSet) ) { // we've found some executor this task can run on. Its possible that some *other* // task isn't schedulable anywhere, but we will discover that in some later call, @@ -789,7 +797,7 @@ private[spark] class TaskSetManager( // shouldn't penalize *this* executor at all, so don't count it as a task-failure as far as // the blacklist is concerned. if (!reason.isInstanceOf[FetchFailed]) { - blacklistTracker.foreach(_.taskFailed(stageId, index, info, sched)) + updateBlacklistForFailedTask(info.host, info.executorId, index) } sched.dagScheduler.taskEnded(tasks(index), reason, null, accumUpdates, info) @@ -819,6 +827,75 @@ private[spark] class TaskSetManager( maybeFinishTaskSet() } + private[scheduler] def updateBlacklistForFailedTask( + host: String, + exec: String, + index: Int): Unit = { + val failureStatus = execToFailures.getOrElseUpdate(exec, new FailureStatus()) + failureStatus.totalFailures += 1 + failureStatus.tasksWithFailures += index + + // check if this task has also failed on other executors on the same host, and if so, blacklist + // this task from the host + val failuresOnHost = (for { + exec <- sched.getExecutorsAliveOnHost(host).getOrElse(Set()).toSeq + failures <- execToFailures.get(exec) + } yield { + if (failures.tasksWithFailures.contains(index)) 1 else 0 + }).sum + if (failuresOnHost >= MAX_TASK_FAILURES_PER_NODE) { + nodeBlacklistedTasks.getOrElseUpdate(host, new HashSet()) += index + } + + if (failureStatus.totalFailures >= MAX_FAILURES_PER_EXEC_STAGE) { + // This executor has been pushed into the blacklist for this stage. Let's check if it pushes + // the whole node into the blacklist + val blacklistedExecutors = + execToFailures.filter(_._2.totalFailures >= MAX_FAILURES_PER_EXEC_STAGE) + if (blacklistedExecutors.size >= MAX_FAILED_EXEC_PER_NODE_STAGE) { + logInfo(s"Blacklisting ${host} for stage $stageId") + blacklistedNodes += host + } + } + } + + /** + * 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 + * altogether. 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 = { + execToFailures.get(executorId) + .map(_.tasksWithFailures.contains(index)) + .getOrElse(false) + } + + def isNodeBlacklistedForTask( + node: String, + index: Int): Boolean = { + nodeBlacklistedTasks.get(node) + .map(_.contains(index)) + .getOrElse(false) + } + + /** + * Return true if this executor is blacklisted for the given stage. Completely ignores whether + * the executor is blacklisted overall (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 = { + execToFailures.get(executorId) + .map(_.totalFailures >= MAX_FAILURES_PER_EXEC_STAGE).getOrElse(false) + } + + def isNodeBlacklistedForTaskSet(node: String): Boolean = { + blacklistedNodes.contains(node) + } + def abort(message: String, exception: Option[Throwable] = None): Unit = sched.synchronized { // TODO: Kill running tasks if we were not terminated due to a Mesos error sched.dagScheduler.taskSetFailed(taskSet, message, exception) diff --git a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala index 8168fd6852587..7fbe8451913b3 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala @@ -21,12 +21,12 @@ import org.mockito.Mockito.when import org.scalatest.BeforeAndAfterEach import org.scalatest.mock.MockitoSugar -import org.apache.spark.SparkConf -import org.apache.spark.SparkFunSuite +import org.apache.spark._ import org.apache.spark.internal.config.{BLACKLIST_EXPIRY_TIMEOUT_CONF, BLACKLIST_LEGACY_TIMEOUT_CONF} import org.apache.spark.util.ManualClock -class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with MockitoSugar { +class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with MockitoSugar + with LocalSparkContext { val stage1 = 1 val stage2 = 2 @@ -71,11 +71,19 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M } } + 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) + scheduler + } + test("Blacklisting individual tasks") { val conf = new SparkConf().setAppName("test").setMaster("local") .set("spark.ui.enabled", "false") .set("spark.scheduler.executorTaskBlacklistTime", "1000") - val scheduler = mock[TaskSchedulerImpl] + val scheduler = mockTaskSchedWithConf(conf) when(scheduler.getExecutorsAliveOnHost("hostA")).thenReturn(Some(Set("1", "2", "4"))) Set("1", "2", "4").foreach { execId => when(scheduler.getHostForExecutor(execId)).thenReturn("hostA") @@ -83,47 +91,43 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M // Task 1 failed on executor 1 blacklistTracker = new BlacklistTracker(conf, clock) - blacklistTracker.taskFailed(stage1, partition1, taskInfo_1_hostA, scheduler) + val taskSet = FakeTask.createTaskSet(10) + val tsm = new TaskSetManager(scheduler, Some(blacklistTracker), taskSet, 4, clock) + tsm.updateBlacklistForFailedTask("hostA", "1", 0) for { executor <- (1 to 4).map(_.toString) - partition <- 0 until 10 - stage <- (1 to 2) + index <- 0 until 10 } { - val exp = (executor == "1" && stage == stage1 && partition == 1) - assert(blacklistTracker.isExecutorBlacklistedForTask(executor, stage, partition) === exp) + val exp = (executor == "1" && index == 0) + assert(tsm.isExecutorBlacklistedForTask(executor, index) === exp) } assert(blacklistTracker.nodeBlacklist() === Set()) assertEquivalentToSet(blacklistTracker.isNodeBlacklisted(_), Set()) - assertEquivalentToSet(blacklistTracker.isNodeBlacklistedForStage(_, stage1), Set()) - assertEquivalentToSet(blacklistTracker.isNodeBlacklistedForStage(_, stage2), Set()) + assertEquivalentToSet(tsm.isNodeBlacklistedForTaskSet, Set()) + assertEquivalentToSet(tsm.isExecutorBlacklistedForTaskSet, Set()) // Task 1 & 2 failed on both executor 1 & 2, so we blacklist all executors on that host, // for all tasks for the stage. Note the api expects multiple checks for each type of // blacklist -- this actually fits naturally with its use in the scheduler - blacklistTracker.taskFailed(stage1, partition1, - new TaskInfo(2L, 1, 1, 0L, "2", "hostA", TaskLocality.ANY, false), scheduler) - blacklistTracker.taskFailed(stage1, partition2, - new TaskInfo(3L, 2, 1, 0L, "2", "hostA", TaskLocality.ANY, false), scheduler) - blacklistTracker.taskFailed(stage1, partition2, - new TaskInfo(4L, 2, 1, 0L, "1", "hostA", TaskLocality.ANY, false), scheduler) + tsm.updateBlacklistForFailedTask("hostA", "1", 1) + tsm.updateBlacklistForFailedTask("hostA", "2", 0) + tsm.updateBlacklistForFailedTask("hostA", "2", 1) // we don't explicitly return the executors in hostA here, but that is OK for { executor <- (1 to 4).map(_.toString) - stage <- (1 to 2) - partition <- 0 until 10 + index <- 0 until 10 } { - withClue(s"exec = $executor; stage = $stage; part = $partition") { + withClue(s"exec = $executor; index = $index") { val badExec = (executor == "1" || executor == "2") - val badPart = (partition == 1 || partition == 2) - val taskExp = (badExec && stage == stage1 && badPart) + val badPart = (index == 0 || index == 1) + val taskExp = (badExec && badPart) assert( - blacklistTracker.isExecutorBlacklistedForTask(executor, stage, partition) === taskExp) - val executorExp = badExec && stage == stage1 - assert(blacklistTracker.isExecutorBlacklistedForStage(stage, executor) === executorExp) + tsm.isExecutorBlacklistedForTask(executor, index) === taskExp) + val executorExp = badExec + assert(tsm.isExecutorBlacklistedForTaskSet(executor) === executorExp) } } - assertEquivalentToSet(blacklistTracker.isNodeBlacklistedForStage(_, stage1), Set("hostA")) - assertEquivalentToSet(blacklistTracker.isNodeBlacklistedForStage(_, stage2), Set()) + assertEquivalentToSet(tsm.isNodeBlacklistedForTaskSet, Set("hostA")) // we dont' blacklist the nodes or executors till the stages complete assert(blacklistTracker.nodeBlacklist() === Set()) assertEquivalentToSet(blacklistTracker.isNodeBlacklisted(_), Set()) @@ -131,7 +135,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M // when the stage completes successfully, now there is sufficient evidence we've got // bad executors and node - blacklistTracker.taskSetSucceeded(stage1, scheduler) + blacklistTracker.taskSetSucceeded(tsm.execToFailures, scheduler) assert(blacklistTracker.nodeBlacklist() === Set("hostA")) assertEquivalentToSet(blacklistTracker.isNodeBlacklisted(_), Set("hostA")) assertEquivalentToSet(blacklistTracker.isExecutorBlacklisted(_), Set("1", "2")) @@ -147,7 +151,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M val conf = new SparkConf().setAppName("test").setMaster("local") .set("spark.ui.enabled", "false") .set("spark.scheduler.executorTaskBlacklistTime", "1000") - val scheduler = mock[TaskSchedulerImpl] + val scheduler = mockTaskSchedWithConf(conf) when(scheduler.getExecutorsAliveOnHost("hostA")).thenReturn(Some(Set("1", "2", "4"))) Set("1", "2", "4").foreach { execId => when(scheduler.getHostForExecutor(execId)).thenReturn("hostA") @@ -164,11 +168,10 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M // 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 (0 until 4).foreach { stage => - tracker.taskFailed(stage, 0, - new TaskInfo(stage, 0, 0, 0, "1", "hostA", TaskLocality.ANY, false), scheduler) - tracker.taskSucceeded(stage, 0, - new TaskInfo(stage, 0, 1, 0, "2", "hostA", TaskLocality.ANY, false), scheduler) - tracker.taskSetSucceeded(stage, scheduler) + val taskSet = FakeTask.createTaskSet(1) + val tsm = new TaskSetManager(scheduler, Some(tracker), taskSet, 4, clock) + tsm.updateBlacklistForFailedTask("hostA", "1", 0) + tracker.taskSetSucceeded(tsm.execToFailures, scheduler) } assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set("1")) } @@ -179,8 +182,9 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M val (tracker, scheduler) = trackerFixture // for 4 different stages, executor 1 fails a task, and then the taskSet fails. (0 until 4).foreach { stage => - tracker.taskFailed(stage, 0, - new TaskInfo(stage, 0, 0, 0, "1", "hostA", TaskLocality.ANY, false), scheduler) + val taskSet = FakeTask.createTaskSet(1) + val tsm = new TaskSetManager(scheduler, Some(tracker), taskSet, 4, clock) + tsm.updateBlacklistForFailedTask("hostA", "1", 0) tracker.taskSetFailed(stage) } assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set()) @@ -192,16 +196,17 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M // but if the taskset fails, we don't blacklist the executor after the stage. val (tracker, scheduler) = trackerFixture val stageId = 1 + (if (succeedTaskSet) 1 else 0) + val taskSet = FakeTask.createTaskSet(4, stageId, 0) + val tsm = new TaskSetManager(scheduler, Some(tracker), taskSet, 4, clock) (0 until 4).foreach { partition => - tracker.taskFailed(stageId, partition, new TaskInfo(stageId * 4 + partition, partition, 0, - clock.getTimeMillis(), "1", "hostA", TaskLocality.ANY, false), scheduler) + tsm.updateBlacklistForFailedTask("hostA", "1", partition) } - assert(tracker.isExecutorBlacklistedForStage(stageId, "1")) + assert(tsm.isExecutorBlacklistedForTaskSet("1")) assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set()) if (succeedTaskSet) { // the task set succeeded elsewhere, so we count those failures against our executor, // and blacklist it across stages - tracker.taskSetSucceeded(stageId, scheduler) + tracker.taskSetSucceeded(tsm.execToFailures, scheduler) assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set("1")) } else { // the task set failed, so we don't count these failures against the executor for other @@ -214,83 +219,42 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M test("blacklisted executors and nodes get recovered with time") { val (tracker, scheduler) = trackerFixture + val taskSet0 = FakeTask.createTaskSet(4) + val tsm0 = new TaskSetManager(scheduler, Some(tracker), taskSet0, 4, clock) (0 until 4).foreach { partition => - tracker.taskFailed(0, partition, new TaskInfo(partition, partition, 0, clock.getTimeMillis(), - "1", "hostA", TaskLocality.ANY, false), scheduler) + tsm0.updateBlacklistForFailedTask("hostA", "1", partition) } - tracker.taskSetSucceeded(0, scheduler) + tracker.taskSetSucceeded(tsm0.execToFailures, scheduler) assert(tracker.nodeBlacklist() === Set()) assertEquivalentToSet(tracker.isNodeBlacklisted(_), Set()) assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set("1")) + val taskSet1 = FakeTask.createTaskSet(4, 1, 0) + val tsm1 = new TaskSetManager(scheduler, Some(tracker), taskSet1, 4, clock) (0 until 4).foreach { partition => - tracker.taskFailed(1, partition, new TaskInfo(partition + 4, partition, 0, - clock.getTimeMillis(), "2", "hostA", TaskLocality.ANY, false), scheduler) + tsm1.updateBlacklistForFailedTask("hostA", "2", partition) } - tracker.taskSetSucceeded(1, scheduler) + tracker.taskSetSucceeded(tsm1.execToFailures, scheduler) assert(tracker.nodeBlacklist() === Set("hostA")) assertEquivalentToSet(tracker.isNodeBlacklisted(_), Set("hostA")) assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set("1", "2")) clock.advance(tracker.EXECUTOR_RECOVERY_MILLIS + 1) - // TODO might want to change this to avoid the need for expiry thread, if that eliminates the - // need for the locks. In which case, expiry would happen automatically. tracker.expireExecutorsInBlacklist() assert(tracker.nodeBlacklist() === Set()) assertEquivalentToSet(tracker.isNodeBlacklisted(_), Set()) assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set()) // fail one more task, but executor isn't put back into blacklist since count reset to 0 - tracker.taskFailed(1, 0, new TaskInfo(5, 0, 0, clock.getTimeMillis(), - "1", "hostA", TaskLocality.ANY, false), scheduler) - tracker.taskSetSucceeded(1, scheduler) + val taskSet2 = FakeTask.createTaskSet(4, 2, 0) + val tsm2 = new TaskSetManager(scheduler, Some(tracker), taskSet2, 4, clock) + tsm2.updateBlacklistForFailedTask("hostA", "1", 0) + tracker.taskSetSucceeded(tsm2.execToFailures, scheduler) assert(tracker.nodeBlacklist() === Set()) assertEquivalentToSet(tracker.isNodeBlacklisted(_), Set()) assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set()) } - test("memory cleaned up as tasksets complete") { - // We want to make sure that memory used by the blacklist tracker is not O(nTotalTaskSetsRun), - // that would be really bad for long-lived applications. This test just requires some knowledge - // of the internals on what to check (without the overhead of trying to trigger an OOM or - // something). - val (tracker, scheduler) = trackerFixture - when(scheduler.getHostForExecutor("0")).thenReturn("hostA") - when(scheduler.getHostForExecutor("1")).thenReturn("hostA") - when(scheduler.getExecutorsAliveOnHost("hostA")).thenReturn(Some(Set("0", "1"))) - - // fail a couple of tasks in two stages - for { - stage <- 0 until 2 - partition <- 0 until 4 - } { - val tid = stage * 4 + partition - // we want to fail on multiple executors, to trigger node blacklist - val exec = (partition % 2).toString - tracker.taskFailed(stage, partition, new TaskInfo(tid, partition, 0, clock.getTimeMillis(), - exec, "hostA", TaskLocality.ANY, false), scheduler) - } - // also want to fail one task on two different executors on the same node, to trigger - // (node, task) blacklist - tracker.taskFailed(0, 0, new TaskInfo(8, 0, 1, clock.getTimeMillis(), "1", "hostA", - TaskLocality.ANY, false), scheduler) - - // just make sure our test is even checking something useful -- we expect these data structures - // to grow for running task sets with failed tasks - assert(tracker.stageIdToExecToFailures.nonEmpty) - assert(tracker.stageIdToNodeBlacklistedTasks.nonEmpty) - assert(tracker.stageIdToBlacklistedNodes.nonEmpty) - - // now say stage 0 fails, and stage 1 completes - tracker.taskSetFailed(0) - tracker.taskSetSucceeded(1, scheduler) - - // datastructures should be empty again - assert(tracker.stageIdToExecToFailures.isEmpty) - assert(tracker.stageIdToBlacklistedNodes.isEmpty) - assert(tracker.stageIdToNodeBlacklistedTasks.isEmpty) - } - test("blacklist still respects legacy configs") { val legacyKey = BLACKLIST_LEGACY_TIMEOUT_CONF.key 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 f67275d3d63dc..1f110b6c2120e 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -17,8 +17,10 @@ package org.apache.spark.scheduler +import scala.collection.mutable.HashMap + import org.mockito.Matchers._ -import org.mockito.Mockito.{atLeast, never, times, verify, when} +import org.mockito.Mockito.{atLeast, never, spy, times, verify, when} import org.scalatest.BeforeAndAfterEach import org.scalatest.mock.MockitoSugar @@ -36,7 +38,6 @@ class FakeSchedulerBackend extends SchedulerBackend { class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with BeforeAndAfterEach with Logging with MockitoSugar { - var failedTaskSetException: Option[Throwable] = None var failedTaskSetReason: String = null var failedTaskSet = false @@ -44,11 +45,14 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B var taskScheduler: TaskSchedulerImpl = null var dagScheduler: DAGScheduler = null + val stageToMockTsm = new HashMap[Int, TaskSetManager]() + override def beforeEach(): Unit = { super.beforeEach() failedTaskSet = false failedTaskSetException = None failedTaskSetReason = null + stageToMockTsm.clear() } override def afterEach(): Unit = { @@ -83,6 +87,24 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B setupHelper() } + def setupSchedulerWithMockTsm(blacklist: BlacklistTracker): TaskSchedulerImpl = { + sc = new SparkContext("local", "TaskSchedulerImplSuite") + taskScheduler = + new TaskSchedulerImpl(sc, sc.conf.getInt("spark.task.maxFailures", 4), Some(blacklist)) { + override def createTaskSetManager(taskSet: TaskSet, maxFailures: Int): TaskSetManager = { + val tsm = super.createTaskSetManager(taskSet, maxFailures) + val tsmSpy = spy(tsm) + stageToMockTsm(taskSet.stageId) = tsmSpy + // intentionally bogus, just lets us easily verify + val execToFailures = new HashMap[String, FailureStatus]() + execToFailures(taskSet.stageId.toString) = new FailureStatus() + when(tsmSpy.execToFailures).thenReturn(execToFailures) + tsmSpy + } + } + setupHelper() + } + def setupHelper(): TaskSchedulerImpl = { taskScheduler.initialize(new FakeSchedulerBackend) // Need to initialize a DAGScheduler for the taskScheduler to use for callbacks. @@ -302,13 +324,11 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B test("scheduled tasks obey task and stage blacklists") { val blacklist = mock[BlacklistTracker] - taskScheduler = setupScheduler(blacklist) - val stage0 = FakeTask.createTaskSet(numTasks = 2, stageId = 0, stageAttemptId = 0) - val stage1 = FakeTask.createTaskSet(numTasks = 2, stageId = 1, stageAttemptId = 0) - val stage2 = FakeTask.createTaskSet(numTasks = 2, stageId = 2, stageAttemptId = 0) - taskScheduler.submitTasks(stage0) - taskScheduler.submitTasks(stage1) - taskScheduler.submitTasks(stage2) + taskScheduler = setupSchedulerWithMockTsm(blacklist) + (0 to 2).foreach { stageId => + val taskSet = FakeTask.createTaskSet(numTasks = 2, stageId = stageId, stageAttemptId = 0) + taskScheduler.submitTasks(taskSet) + } val offers = Seq( new WorkerOffer("executor0", "host0", 1), @@ -324,12 +344,15 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B // (later stubs take precedence over earlier ones) when(blacklist.isNodeBlacklisted(anyString())).thenReturn(false) when(blacklist.isExecutorBlacklisted(anyString())).thenReturn(false) - when(blacklist.isNodeBlacklistedForStage(anyString(), anyInt())).thenReturn(false) - when(blacklist.isNodeBlacklistedForStage("host1", 0)).thenReturn(true) - when(blacklist.isExecutorBlacklistedForStage(anyInt(), anyString())).thenReturn(false) - when(blacklist.isExecutorBlacklistedForStage(1, "executor3")).thenReturn(true) - when(blacklist.isExecutorBlacklistedForTask(anyString(), anyInt(), anyInt())).thenReturn(false) - when(blacklist.isExecutorBlacklistedForTask("executor0", 0, 0)).thenReturn(true) + // setup some defaults, then override them with particulars + stageToMockTsm.values.foreach { tsm => + when(tsm.isNodeBlacklistedForTaskSet(anyString())).thenReturn(false) + when(tsm.isExecutorBlacklistedForTaskSet(anyString())).thenReturn(false) + when(tsm.isExecutorBlacklistedForTask(anyString(), anyInt())).thenReturn(false) + } + when(stageToMockTsm(0).isNodeBlacklistedForTaskSet("host1")).thenReturn(true) + when(stageToMockTsm(1).isExecutorBlacklistedForTaskSet("executor3")).thenReturn(true) + when(stageToMockTsm(0).isExecutorBlacklistedForTask("executor0", 0)).thenReturn(true) val firstTaskAttempts = taskScheduler.resourceOffers(offers).flatten // these verifications are tricky b/c we reference them multiple times -- also invoked when we @@ -337,9 +360,9 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B ('0' until '2').foreach { hostNum => val host = s"host$hostNum" verify(blacklist, atLeast(1)).isNodeBlacklisted(host) - verify(blacklist, atLeast(1)).isNodeBlacklistedForStage(host, 0) - verify(blacklist, atLeast(1)).isNodeBlacklistedForStage(host, 1) - verify(blacklist, atLeast(1)).isNodeBlacklistedForStage(host, 2) + (0 to 2).foreach { stageId => + verify(stageToMockTsm(stageId), atLeast(1)).isNodeBlacklistedForTaskSet(host) + } } for { exec <- Seq("executor1", "executor2") @@ -348,13 +371,13 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B // the node blacklist should ensure we never check the task blacklist. This is important // for performance, otherwise we end up changing an O(1) operation into a // O(numPendingTasks) one - verify(blacklist, never).isExecutorBlacklistedForTask(exec, 0, part) + verify(stageToMockTsm(0), never).isExecutorBlacklistedForTask(exec, part) } // similarly, the executor blacklist for an entire stage should prevent us from ever checking // the blacklist for specific parts in a stage. (0 to 1).foreach { part => - verify(blacklist, never).isExecutorBlacklistedForTask("executor3", 1, part) + verify(stageToMockTsm(1), never).isExecutorBlacklistedForTask("executor3", part) } // we should schedule all tasks. @@ -412,21 +435,18 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B } // the tasksSets complete, so the tracker should be notified - verify(blacklist, times(1)).taskSetSucceeded(0, taskScheduler) - verify(blacklist, times(1)).taskSetSucceeded(1, taskScheduler) + verify(blacklist, times(1)).taskSetSucceeded(stageToMockTsm(0).execToFailures, taskScheduler) + verify(blacklist, times(1)).taskSetSucceeded(stageToMockTsm(1).execToFailures, taskScheduler) verify(blacklist, times(1)).taskSetFailed(2) } test("scheduled tasks obey node and executor blacklists") { - // another case with full node & executor blacklist val blacklist = mock[BlacklistTracker] - taskScheduler = setupScheduler(blacklist) - val stage0 = FakeTask.createTaskSet(numTasks = 2, stageId = 0, stageAttemptId = 0) - val stage1 = FakeTask.createTaskSet(numTasks = 2, stageId = 1, stageAttemptId = 0) - val stage2 = FakeTask.createTaskSet(numTasks = 2, stageId = 2, stageAttemptId = 0) - taskScheduler.submitTasks(stage0) - taskScheduler.submitTasks(stage1) - taskScheduler.submitTasks(stage2) + taskScheduler = setupSchedulerWithMockTsm(blacklist) + (0 to 2).foreach { stageId => + val taskSet = FakeTask.createTaskSet(numTasks = 2, stageId = stageId, stageAttemptId = 0) + taskScheduler.submitTasks(taskSet) + } val offers = Seq( new WorkerOffer("executor0", "host0", 1), @@ -436,15 +456,12 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B ) // setup our mock blacklist: - // host1, executor0 & executor3 are completely blacklisted + // host1, executor0 & executor3 are completely blacklisted (which covers all the executors) when(blacklist.isNodeBlacklisted(anyString())).thenReturn(false) when(blacklist.isNodeBlacklisted("host1")).thenReturn(true) when(blacklist.isExecutorBlacklisted(anyString())).thenReturn(false) when(blacklist.isExecutorBlacklisted("executor0")).thenReturn(true) when(blacklist.isExecutorBlacklisted("executor3")).thenReturn(true) - when(blacklist.isNodeBlacklistedForStage(anyString(), anyInt())).thenReturn(false) - when(blacklist.isExecutorBlacklistedForStage(anyInt(), anyString())).thenReturn(false) - when(blacklist.isExecutorBlacklistedForTask(anyString(), anyInt(), anyInt())).thenReturn(false) val stageToTsm = (0 to 2).map { stageId => val tsm = taskScheduler.taskSetManagerForAttempt(stageId, 0).get @@ -457,9 +474,11 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B ('0' until '2').foreach { hostNum => verify(blacklist, atLeast(1)).isNodeBlacklisted("host" + hostNum) } - verify(blacklist, never()).isNodeBlacklistedForStage(anyString(), anyInt()) - verify(blacklist, never()).isExecutorBlacklistedForStage(anyInt(), anyString()) - verify(blacklist, never()).isExecutorBlacklistedForTask(anyString(), anyInt(), anyInt()) + stageToMockTsm.values.foreach { tsm => + verify(tsm, never()).isNodeBlacklistedForTaskSet(anyString()) + verify(tsm, never()).isExecutorBlacklistedForTaskSet(anyString()) + verify(tsm, never()).isExecutorBlacklistedForTask(anyString(), anyInt()) + } // we should have aborted the existing stages, since they aren't schedulable (0 to 2).foreach { stageId => @@ -612,16 +631,16 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B test("don't update blacklist for shuffle-fetch failures") { val blacklist = mock[BlacklistTracker] - taskScheduler = setupScheduler(blacklist) + taskScheduler = setupSchedulerWithMockTsm(blacklist) val stage0 = FakeTask.createTaskSet(numTasks = 2, stageId = 0, stageAttemptId = 0) taskScheduler.submitTasks(stage0) val taskDescs = taskScheduler.resourceOffers( Seq(new WorkerOffer("executor0", "host0", 10))).flatten assert(taskDescs.size == 2) - val mgr = taskScheduler.taskIdToTaskSetManager.get(taskDescs(0).taskId).get - taskScheduler.handleFailedTask(mgr, taskDescs(0).taskId, TaskState.FAILED, + val tsm = stageToMockTsm(0) + taskScheduler.handleFailedTask(tsm, taskDescs(0).taskId, TaskState.FAILED, FetchFailed(BlockManagerId("executor1", "host1", 12345), 0, 0, 0, "ignored")) - verify(blacklist, never()).taskFailed(anyInt(), anyInt(), anyObject(), anyObject()) + verify(tsm, never()).updateBlacklistForFailedTask(anyString(), anyString(), anyInt()) } } From 338db65e75c57bdc4af34ee342bc4ab843468b90 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 13 Jul 2016 06:31:03 -0500 Subject: [PATCH 10/57] fix --- .../main/scala/org/apache/spark/internal/config/package.scala | 2 +- .../scala/org/apache/spark/scheduler/BlacklistTracker.scala | 4 ++-- .../apache/spark/scheduler/BlacklistIntegrationSuite.scala | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) 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 6e0c20fcb4ee9..763529a9baedb 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 @@ -104,7 +104,7 @@ package object config { .createOptional private[spark] val MAX_TASK_FAILURES_PER_NODE = - ConfigBuilder("spark.blacklist.maxTaskFailuresPerNode") + ConfigBuilder("spark.blacklist.maxFailedTasksPerNode") .intConf .createWithDefault(2) diff --git a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala index 065a336c115dc..c2a3f90014783 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala @@ -41,8 +41,8 @@ import org.apache.spark.util.Utils * faulty; etc. * * 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. + * 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 ( conf: SparkConf, diff --git a/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala index f4963f9573424..6db8d1c41b90b 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala @@ -65,7 +65,7 @@ class BlacklistIntegrationSuite extends SchedulerIntegrationSuite[MultiExecutorM "spark.testing.nExecutorsPerHost" -> "5", "spark.testing.nCoresPerExecutor" -> "10", "spark.task.maxFailures" -> "4", - "spark.blacklist.maxTaskFailuresPerNode" -> "5" + "spark.blacklist.maxFailedTasksPerNode" -> "5" ) ) { // to reliably reproduce the failure, we have to use 1 task. That way, we ensure this From 16afb431f3827d10c622d0f8ec436a8c01c961fb Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 14 Jul 2016 10:58:39 -0500 Subject: [PATCH 11/57] review feedback --- .../spark/internal/config/package.scala | 2 +- .../spark/scheduler/BlacklistTracker.scala | 32 +++++---- .../spark/scheduler/TaskSchedulerImpl.scala | 9 +-- .../spark/scheduler/TaskSetManager.scala | 57 ++++++++-------- .../scheduler/BlacklistIntegrationSuite.scala | 8 +-- .../scheduler/BlacklistTrackerSuite.scala | 33 +++------- .../scheduler/TaskSchedulerImplSuite.scala | 65 +++++++++++++++---- .../spark/scheduler/TaskSetManagerSuite.scala | 4 +- 8 files changed, 111 insertions(+), 99 deletions(-) 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 763529a9baedb..b3394938b97b9 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 @@ -136,7 +136,7 @@ package object config { private[spark] val BLACKLIST_LEGACY_TIMEOUT_CONF = ConfigBuilder("spark.scheduler.executorTaskBlacklistTime") .internal() - .longConf + .timeConf(TimeUnit.MILLISECONDS) .createOptional // End blacklist confs diff --git a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala index c2a3f90014783..c70e8122cac18 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala @@ -175,27 +175,25 @@ private[scheduler] object BlacklistTracker extends Logging { case Some(isEnabled) => isEnabled case None => - // if they've got a non-zero setting for the legacy conf, always enable the blacklist, - // otherwise, use the default based on the cluster-mode (off for local-mode, on otherwise). + // if they've got a non-zero setting for the legacy conf, always enable the blacklist, + // otherwise, use the default based on the cluster-mode (off for local-mode, on otherwise). val legacyKey = config.BLACKLIST_LEGACY_TIMEOUT_CONF.key - conf.getOption(legacyKey) match { - case Some(legacyTimeout) => - if (legacyTimeout.toLong > 0) { - // mostly this is necessary just for tests, since real users that want the blacklist - // will get it anyway by default - logWarning(s"Turning on blacklisting due to legacy configuration:" + - s" $legacyKey > 0") - true - } else { + conf.get(config.BLACKLIST_LEGACY_TIMEOUT_CONF) match { + case Some(legacyTimeout) if legacyTimeout == 0 => logWarning(s"Turning off blacklisting due to legacy configuaration:" + s" $legacyKey == 0") false - } - case None => - // local-cluster is *not* considered local for these purposes, we still want the blacklist - // enabled by default - !Utils.isLocalMaster(conf) - } + case Some(legacyTimeout) => + // mostly this is necessary just for tests, since real users that want the blacklist + // will get it anyway by default + logWarning(s"Turning on blacklisting due to legacy configuration:" + + s" $legacyKey > 0") + true + case None => + // local-cluster is *not* considered local for these purposes, we still want the + // blacklist enabled by default + !Utils.isLocalMaster(conf) + } } } 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 d0dc92f9c0416..41cfb236d01d8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -312,13 +312,10 @@ private[spark] class TaskSchedulerImpl private[scheduler]( private[scheduler] def areAllExecutorsBlacklisted(): Boolean = { blacklistTracker match { case Some(bl) => - executorsByHost.foreach { case (host, execs) => - if (!bl.isNodeBlacklisted(host) && - execs.exists(!bl.isExecutorBlacklisted(_))) { - return false - } + executorsByHost.forall { case (host, execs) => + bl.isNodeBlacklisted(host) || + execs.forall(bl.isExecutorBlacklisted(_)) } - true case None => false } } 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 c214258827878..9002b62828ff1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -257,11 +257,7 @@ private[spark] class TaskSetManager( while (indexOffset > 0) { indexOffset -= 1 val index = list(indexOffset) - val taskBlacklisted = blacklistTracker.map { bl => - isNodeBlacklistedForTask(host, index) || - isExecutorBlacklistedForTask(execId, index) - }.getOrElse(false) - if (!taskBlacklisted) { + if (!blacklistedOnExec(execId, host, index)) { // This should almost always be list.trimEnd(1) to remove tail list.remove(indexOffset) if (copiesRunning(index) == 0 && !successful(index)) { @@ -277,6 +273,13 @@ private[spark] class TaskSetManager( taskAttempts(taskIndex).exists(_.host == host) } + private def blacklistedOnExec(execId: String, host: String, index: Int): Boolean = { + blacklistTracker.map { bl => + isNodeBlacklistedForTask(host, index) || + isExecutorBlacklistedForTask(execId, index) + }.getOrElse(false) + } + /** * Return a speculative task for a given executor if any are available. The task should not have * an attempt running on this host, in case the host is slow. In addition, the task should meet @@ -289,10 +292,8 @@ private[spark] class TaskSetManager( speculatableTasks.retain(index => !successful(index)) // Remove finished tasks from set def canRunOnHost(index: Int): Boolean = { - !hasAttemptOnHost(index, host) && blacklistTracker.map { bl => - !isNodeBlacklistedForTask(host, index) && - !isExecutorBlacklistedForTask(execId, index) - }.getOrElse(true) + !hasAttemptOnHost(index, host) && + !blacklistedOnExec(execId, host, index) } if (!speculatableTasks.isEmpty) { @@ -618,28 +619,28 @@ private[spark] class TaskSetManager( // take any task that needs to be scheduled, and see if we can find some executor it *could* // run on pendingTask.foreach { indexInTaskSet => - val stage = taskSet.stageId - executorsByHost.foreach { case (host, execs) => - if (!blacklist.isNodeBlacklisted(host) && - !isNodeBlacklistedForTaskSet(host) && - !isNodeBlacklistedForTask(host, indexInTaskSet)) { - execs.foreach { exec => - if ( - !blacklist.isExecutorBlacklisted(exec) && - !isExecutorBlacklistedForTaskSet(exec) && - !isExecutorBlacklistedForTask(exec, indexInTaskSet) - ) { - // we've found some executor this task can run on. Its possible that some *other* - // task isn't schedulable anywhere, but we will discover that in some later call, - // when that unschedulable task is the last task remaining. - return - } + // try to find some executor this task can run on. Its possible that some *other* + // task isn't schedulable anywhere, but we will discover that in some later call, + // when that unschedulable task is the last task remaining. + val blacklistedEverywhere = executorsByHost.forall { case (host, execs) => + val nodeBlacklisted = blacklist.isNodeBlacklisted(host) || + isNodeBlacklistedForTaskSet(host) || + isNodeBlacklistedForTask(host, indexInTaskSet) + if (nodeBlacklisted) { + true + } else { + execs.forall { exec => + blacklist.isExecutorBlacklisted(exec) || + isExecutorBlacklistedForTaskSet(exec) || + isExecutorBlacklistedForTask(exec, indexInTaskSet) } } } - val partition = tasks(indexInTaskSet).partitionId - abort(s"Aborting ${taskSet} because task $indexInTaskSet (partition $partition) cannot " + - s"run anywhere due to node and executor blacklist.") + if (blacklistedEverywhere) { + val partition = tasks(indexInTaskSet).partitionId + abort(s"Aborting ${taskSet} because task $indexInTaskSet (partition $partition) cannot " + + s"run anywhere due to node and executor blacklist.") + } } } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala index 6db8d1c41b90b..29e7d33f8d0d4 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala @@ -16,7 +16,6 @@ */ package org.apache.spark.scheduler -import scala.concurrent.Await import scala.concurrent.duration._ import org.apache.spark._ @@ -111,8 +110,6 @@ class BlacklistIntegrationSuite extends SchedulerIntegrationSuite[MultiExecutorM "Bad node with multiple executors, job will still succeed with the right confs", extraConfs = Seq( "spark.scheduler.blacklist.enabled" -> "true", - // just set this to something much longer than the test duration - "spark.scheduler.executorTaskBlacklistTime" -> "10000000", // just to avoid this test taking too long "spark.locality.wait" -> "10ms" ) @@ -132,9 +129,6 @@ class BlacklistIntegrationSuite extends SchedulerIntegrationSuite[MultiExecutorM "SPARK-15865 Progress with fewer executors than maxTaskFailures", extraConfs = Seq( "spark.scheduler.blacklist.enabled" -> "true", - // set this to something much longer than the test duration so that executors don't get - // removed from the blacklist during the test - "spark.scheduler.executorTaskBlacklistTime" -> "10000000", "spark.testing.nHosts" -> "2", "spark.testing.nExecutorsPerHost" -> "1", "spark.testing.nCoresPerExecutor" -> "1" @@ -146,7 +140,7 @@ class BlacklistIntegrationSuite extends SchedulerIntegrationSuite[MultiExecutorM } withBackend(runBackend _) { val jobFuture = submit(new MockRDD(sc, 10, Nil), (0 until 10).toArray) - Await.ready(jobFuture, duration) + awaitJobTermination(jobFuture, duration) val pattern = ("Aborting TaskSet 0.0 because task .* " + "cannot run anywhere due to node and executor blacklist").r assert(pattern.findFirstIn(failure.getMessage).isDefined, diff --git a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala index 7fbe8451913b3..4f2bf9a841450 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala @@ -22,30 +22,15 @@ import org.scalatest.BeforeAndAfterEach import org.scalatest.mock.MockitoSugar import org.apache.spark._ -import org.apache.spark.internal.config.{BLACKLIST_EXPIRY_TIMEOUT_CONF, BLACKLIST_LEGACY_TIMEOUT_CONF} +import org.apache.spark.internal.config.{BLACKLIST_ENABLED, BLACKLIST_EXPIRY_TIMEOUT_CONF, BLACKLIST_LEGACY_TIMEOUT_CONF} import org.apache.spark.util.ManualClock class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with MockitoSugar with LocalSparkContext { - val stage1 = 1 - val stage2 = 2 + private val clock = new ManualClock(0) - val partition1 = 1 - val partition2 = 2 - val partition3 = 3 - - // Variable name can indicate basic information of taskInfo - // hostA: executor 1, 2, 4 - // hostB: executor 3 - // The format is "taskInfo_executorId_hostName" - val taskInfo_1_hostA = new TaskInfo(1L, 1, 1, 0L, "1", "hostA", TaskLocality.ANY, false) - val taskInfo_2_hostA = new TaskInfo(2L, 1, 1, 0L, "2", "hostA", TaskLocality.ANY, false) - val taskInfo_3_hostB = new TaskInfo(3L, 3, 1, 0L, "3", "hostB", TaskLocality.ANY, false) - - val clock = new ManualClock(0) - - var blacklistTracker: BlacklistTracker = _ + private var blacklistTracker: BlacklistTracker = _ override def afterEach(): Unit = { if (blacklistTracker != null) { @@ -81,8 +66,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M test("Blacklisting individual tasks") { val conf = new SparkConf().setAppName("test").setMaster("local") - .set("spark.ui.enabled", "false") - .set("spark.scheduler.executorTaskBlacklistTime", "1000") + .set(BLACKLIST_ENABLED.key, "true") val scheduler = mockTaskSchedWithConf(conf) when(scheduler.getExecutorsAliveOnHost("hostA")).thenReturn(Some(Set("1", "2", "4"))) Set("1", "2", "4").foreach { execId => @@ -149,8 +133,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M def trackerFixture: (BlacklistTracker, TaskSchedulerImpl) = { val conf = new SparkConf().setAppName("test").setMaster("local") - .set("spark.ui.enabled", "false") - .set("spark.scheduler.executorTaskBlacklistTime", "1000") + .set(BLACKLIST_ENABLED.key, "true") val scheduler = mockTaskSchedWithConf(conf) when(scheduler.getExecutorsAliveOnHost("hostA")).thenReturn(Some(Set("1", "2", "4"))) Set("1", "2", "4").foreach { execId => @@ -263,7 +246,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M assert(!BlacklistTracker.isBlacklistEnabled(localConf)) localConf.set(legacyKey, "5000") assert(BlacklistTracker.isBlacklistEnabled(localConf)) - assert(5000 == BlacklistTracker.getBlacklistExpiryTime(localConf)) + assert(5000 === BlacklistTracker.getBlacklistExpiryTime(localConf)) localConf.set(legacyKey, "0") assert(!BlacklistTracker.isBlacklistEnabled(localConf)) @@ -272,9 +255,9 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M { val distConf = new SparkConf().setMaster("yarn-cluster") assert(BlacklistTracker.isBlacklistEnabled(distConf)) - assert(60 * 60 * 1000L == BlacklistTracker.getBlacklistExpiryTime(distConf)) + assert(60 * 60 * 1000L === BlacklistTracker.getBlacklistExpiryTime(distConf)) distConf.set(legacyKey, "5000") - assert(5000 == BlacklistTracker.getBlacklistExpiryTime(distConf)) + assert(5000 === BlacklistTracker.getBlacklistExpiryTime(distConf)) distConf.set(BLACKLIST_EXPIRY_TIMEOUT_CONF.key, "10h") assert(10 * 60 * 60 * 1000L == BlacklistTracker.getBlacklistExpiryTime(distConf)) } 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 1f110b6c2120e..93569c8d9e7f1 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -25,6 +25,7 @@ import org.scalatest.BeforeAndAfterEach import org.scalatest.mock.MockitoSugar import org.apache.spark._ +import org.apache.spark.internal.config import org.apache.spark.internal.Logging import org.apache.spark.storage.BlockManagerId @@ -381,7 +382,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B } // we should schedule all tasks. - assert(firstTaskAttempts.size == 6) + assert(firstTaskAttempts.size === 6) def tasksForStage(stageId: Int): Seq[TaskDescription] = { firstTaskAttempts.filter{_.name.contains(s"stage $stageId")} } @@ -389,7 +390,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B // exec 1 & 2 blacklisted for node // exec 0 blacklisted just for part 0 if (task.index == 0) { - assert(task.executorId == "executor3") + assert(task.executorId === "executor3") } else { assert(Set("executor0", "executor3").contains(task.executorId)) } @@ -410,16 +411,16 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B var task = tasks(0) val taskIndex = task.index (0 until 4).foreach { attempt => - assert(task.attemptNumber == attempt) + assert(task.attemptNumber === attempt) tsm.handleFailedTask(task.taskId, TaskState.FAILED, TaskResultLost) val nextAttempts = taskScheduler.resourceOffers(Seq(WorkerOffer("executor4", "host4", 1))).flatten if (attempt < 3) { - assert(nextAttempts.size == 1) + assert(nextAttempts.size === 1) task = nextAttempts(0) - assert(task.index == taskIndex) + assert(task.index === taskIndex) } else { - assert(nextAttempts.size == 0) + assert(nextAttempts.size === 0) } } // end the other task of the taskset, doesn't matter whether it succeeds or fails @@ -487,15 +488,53 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B } } + test("abort stage when all executors are blacklisted") { + val blacklist = mock[BlacklistTracker] + taskScheduler = setupSchedulerWithMockTsm(blacklist) + val taskSet = FakeTask.createTaskSet(numTasks = 2, stageId = 0, stageAttemptId = 0) + taskScheduler.submitTasks(taskSet) + val tsm = stageToMockTsm(0) + + // first just submit some offers so the scheduler knows about all the executors + taskScheduler.resourceOffers(Seq( + WorkerOffer("executor0", "host0", 2), + WorkerOffer("executor1", "host0", 2), + WorkerOffer("executor2", "host0", 2), + WorkerOffer("executor3", "host1", 2) + )) + + // now say our blacklist updates to blacklist a bunch of resources, but *not* everything + when(blacklist.isNodeBlacklisted(anyString())).thenReturn(false) + when(blacklist.isNodeBlacklisted("host1")).thenReturn(true) + when(blacklist.isExecutorBlacklisted(anyString())).thenReturn(false) + when(blacklist.isExecutorBlacklisted("executor0")).thenReturn(true) + + // make an offer on the blacklisted resources. We won't schedule anything, but also won't + // abort yet, since we know of other resources that work + assert(taskScheduler.resourceOffers(Seq( + WorkerOffer("executor0", "host0", 2), + WorkerOffer("executor3", "host1", 2) + )).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) + assert(taskScheduler.resourceOffers(Seq( + WorkerOffer("executor0", "host0", 2), + WorkerOffer("executor3", "host1", 2) + )).flatten.size === 0) + assert(tsm.isZombie) + verify(tsm).abort(anyString(), anyObject()) + } + 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 // 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( - // set this to something much longer than the test duration so that executors don't get - // removed from the blacklist during the test - "spark.scheduler.executorTaskBlacklistTime" -> "10000000" + config.BLACKLIST_ENABLED.key -> "true" ) val taskSet = FakeTask.createTaskSet(2) @@ -534,7 +573,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B assert(tsm.isZombie) assert(failedTaskSet) val idx = failedTask.index - assert(failedTaskSetReason == s"Aborting TaskSet 0.0 because task $idx (partition $idx) " + + assert(failedTaskSetReason === s"Aborting TaskSet 0.0 because task $idx (partition $idx) " + s"cannot run anywhere due to node and executor blacklist.") } @@ -545,9 +584,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B // available and not bail on the job val taskScheduler = setupScheduler( - // set this to something much longer than the test duration so that executors don't get - // removed from the blacklist during the test - "spark.scheduler.executorTaskBlacklistTime" -> "10000000" + config.BLACKLIST_ENABLED.key -> "true" ) val taskSet = FakeTask.createTaskSet(2, (0 until 2).map { _ => Seq(TaskLocation("host0")) }: _*) @@ -636,7 +673,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B taskScheduler.submitTasks(stage0) val taskDescs = taskScheduler.resourceOffers( Seq(new WorkerOffer("executor0", "host0", 10))).flatten - assert(taskDescs.size == 2) + assert(taskDescs.size === 2) val tsm = stageToMockTsm(0) taskScheduler.handleFailedTask(tsm, taskDescs(0).taskId, TaskState.FAILED, 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 18165d4f53e9e..8cecd3282ff2a 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -25,6 +25,7 @@ import scala.collection.mutable.ArrayBuffer import org.mockito.Mockito.{mock, verify} import org.apache.spark._ +import org.apache.spark.internal.config import org.apache.spark.internal.Logging import org.apache.spark.util.{AccumulatorV2, ManualClock} @@ -413,7 +414,8 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg test("executors should be blacklisted after task failure, in spite of locality preferences") { val rescheduleDelay = 300L val conf = new SparkConf(). - set("spark.scheduler.executorTaskBlacklistTime", rescheduleDelay.toString). + set(config.BLACKLIST_ENABLED.key, "true"). + set(config.BLACKLIST_EXPIRY_TIMEOUT_CONF.key, rescheduleDelay.toString). // don't wait to jump locality levels in this test set("spark.locality.wait", "0") From 7aff08a52099334eb6ac242c7ba70a9873aef624 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 14 Jul 2016 16:22:57 -0500 Subject: [PATCH 12/57] review feedback --- .../scala/org/apache/spark/scheduler/TaskSetManager.scala | 5 +++-- .../scala/org/apache/spark/deploy/yarn/YarnAllocator.scala | 6 ++++++ .../spark/scheduler/cluster/YarnSchedulerBackend.scala | 5 +++-- 3 files changed, 12 insertions(+), 4 deletions(-) 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 9002b62828ff1..37161c1e573a3 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -854,8 +854,9 @@ private[spark] class TaskSetManager( val blacklistedExecutors = execToFailures.filter(_._2.totalFailures >= MAX_FAILURES_PER_EXEC_STAGE) if (blacklistedExecutors.size >= MAX_FAILED_EXEC_PER_NODE_STAGE) { - logInfo(s"Blacklisting ${host} for stage $stageId") - blacklistedNodes += host + if (blacklistedNodes.add(host)) { + logInfo(s"Blacklisting ${host} for stage $stageId") + } } } } diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala index 5f4a9e6403d5e..0575c74c6c835 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -239,6 +239,12 @@ private[yarn] class YarnAllocator( // in order to avoid allocating new Container on the problematic nodes. val blacklistAdditions = nodeBlacklist -- currentNodeBlacklist val blacklistRemovals = currentNodeBlacklist -- nodeBlacklist + if (blacklistAdditions.nonEmpty) { + logInfo(s"adding nodes to blacklist: $blacklistAdditions") + } + if (blacklistRemovals.nonEmpty) { + logInfo(s"removing nodes from blacklist: $blacklistRemovals") + } amClient.updateBlacklist(blacklistAdditions.toList.asJava, blacklistRemovals.toList.asJava) currentNodeBlacklist = nodeBlacklist true diff --git a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala index a5a66caa13b08..b503d89335caf 100644 --- a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala +++ b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala @@ -128,9 +128,10 @@ private[spark] abstract class YarnSchedulerBackend( val nodeBlacklist: Set[String] = scheduler.blacklistTracker.map(_.nodeBlacklist()).getOrElse(Set()) + val filteredHostToLocalTaskCount = hostToLocalTaskCount.filterKeys(!nodeBlacklist.contains(_)) - yarnSchedulerEndpointRef.askWithRetry[Boolean]( - RequestExecutors(requestedTotal, localityAwareTasks, hostToLocalTaskCount, nodeBlacklist)) + yarnSchedulerEndpointRef.askWithRetry[Boolean](RequestExecutors( + requestedTotal, localityAwareTasks, filteredHostToLocalTaskCount, nodeBlacklist)) } /** From e181546bbcce249ace26c952666493a187f95437 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 14 Jul 2016 16:26:54 -0500 Subject: [PATCH 13/57] rename conf --- .../main/scala/org/apache/spark/internal/config/package.scala | 4 ++-- .../scala/org/apache/spark/scheduler/TaskSetManager.scala | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) 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 b3394938b97b9..bf947664b9083 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 @@ -103,8 +103,8 @@ package object config { .booleanConf .createOptional - private[spark] val MAX_TASK_FAILURES_PER_NODE = - ConfigBuilder("spark.blacklist.maxFailedTasksPerNode") + private[spark] val MAX_TASK_ATTEMPTS_PER_NODE = + ConfigBuilder("spark.blacklist.maxTaskAttemptsPerNode") .intConf .createWithDefault(2) 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 37161c1e573a3..d25f2b5f8a7ef 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -65,7 +65,7 @@ private[spark] class TaskSetManager( } val conf = sched.sc.conf - private val MAX_TASK_FAILURES_PER_NODE = conf.get(config.MAX_TASK_FAILURES_PER_NODE) + private val MAX_TASK_ATTEMPTS_PER_NODE = conf.get(config.MAX_TASK_ATTEMPTS_PER_NODE) private val MAX_FAILURES_PER_EXEC_STAGE = conf.get(config.MAX_FAILURES_PER_EXEC_STAGE) private val MAX_FAILED_EXEC_PER_NODE_STAGE = conf.get(config.MAX_FAILED_EXEC_PER_NODE_STAGE) @@ -844,7 +844,7 @@ private[spark] class TaskSetManager( } yield { if (failures.tasksWithFailures.contains(index)) 1 else 0 }).sum - if (failuresOnHost >= MAX_TASK_FAILURES_PER_NODE) { + if (failuresOnHost >= MAX_TASK_ATTEMPTS_PER_NODE) { nodeBlacklistedTasks.getOrElseUpdate(host, new HashSet()) += index } From 351a9a7e2893a0b90c57233d5e44a52c147bb2a8 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 14 Jul 2016 16:45:24 -0500 Subject: [PATCH 14/57] use typed confs consistently --- .../scheduler/BlacklistIntegrationSuite.scala | 21 ++++++++++--------- 1 file changed, 11 insertions(+), 10 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala index 29e7d33f8d0d4..c703e4f80846d 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.scheduler import scala.concurrent.duration._ import org.apache.spark._ +import org.apache.spark.internal.config class BlacklistIntegrationSuite extends SchedulerIntegrationSuite[MultiExecutorMockBackend]{ @@ -43,7 +44,7 @@ class BlacklistIntegrationSuite extends SchedulerIntegrationSuite[MultiExecutorM // according to locality preferences, and so the job fails testScheduler("If preferred node is bad, without blacklist job will fail", extraConfs = Seq( - "spark.scheduler.blacklist.enabled" -> "false" + config.BLACKLIST_ENABLED.key -> "false" )) { val rdd = new MockRDDWithLocalityPrefs(sc, 10, Nil, badHost) withBackend(badHostBackend _) { @@ -59,12 +60,12 @@ class BlacklistIntegrationSuite extends SchedulerIntegrationSuite[MultiExecutorM testScheduler( "With blacklist on, job will still fail if there are too many bad executors on bad host", extraConfs = Seq( - "spark.scheduler.blacklist.enabled" -> "true", + config.BLACKLIST_ENABLED.key -> "true", + config.MAX_TASK_ATTEMPTS_PER_NODE.key -> "5", + "spark.task.maxFailures" -> "4", "spark.testing.nHosts" -> "2", "spark.testing.nExecutorsPerHost" -> "5", - "spark.testing.nCoresPerExecutor" -> "10", - "spark.task.maxFailures" -> "4", - "spark.blacklist.maxFailedTasksPerNode" -> "5" + "spark.testing.nCoresPerExecutor" -> "10" ) ) { // to reliably reproduce the failure, we have to use 1 task. That way, we ensure this @@ -84,11 +85,11 @@ class BlacklistIntegrationSuite extends SchedulerIntegrationSuite[MultiExecutorM testScheduler( "With default settings, job can succeed despite multiple bad executors on node", extraConfs = Seq( - "spark.scheduler.blacklist.enabled" -> "true", + config.BLACKLIST_ENABLED.key -> "true", + "spark.task.maxFailures" -> "4", "spark.testing.nHosts" -> "2", "spark.testing.nExecutorsPerHost" -> "5", - "spark.testing.nCoresPerExecutor" -> "10", - "spark.task.maxFailures" -> "4" + "spark.testing.nCoresPerExecutor" -> "10" ) ) { // to reliably reproduce the failure, we have to use 1 task. That way, we ensure this @@ -109,7 +110,7 @@ class BlacklistIntegrationSuite extends SchedulerIntegrationSuite[MultiExecutorM testScheduler( "Bad node with multiple executors, job will still succeed with the right confs", extraConfs = Seq( - "spark.scheduler.blacklist.enabled" -> "true", + config.BLACKLIST_ENABLED.key -> "true", // just to avoid this test taking too long "spark.locality.wait" -> "10ms" ) @@ -128,7 +129,7 @@ class BlacklistIntegrationSuite extends SchedulerIntegrationSuite[MultiExecutorM testScheduler( "SPARK-15865 Progress with fewer executors than maxTaskFailures", extraConfs = Seq( - "spark.scheduler.blacklist.enabled" -> "true", + config.BLACKLIST_ENABLED.key -> "true", "spark.testing.nHosts" -> "2", "spark.testing.nExecutorsPerHost" -> "1", "spark.testing.nCoresPerExecutor" -> "1" From 572c7773c44010428e4c1b39a4b03fdbf103a3d6 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 20 Jul 2016 12:11:15 -0500 Subject: [PATCH 15/57] docs --- .../spark/internal/config/package.scala | 2 +- docs/configuration.md | 58 +++++++++++++++++-- 2 files changed, 53 insertions(+), 7 deletions(-) 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 bf947664b9083..1781862b6c4c3 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 @@ -99,7 +99,7 @@ package object config { // Blacklist confs private[spark] val BLACKLIST_ENABLED = - ConfigBuilder("spark.scheduler.blacklist.enabled") + ConfigBuilder("spark.blacklist.enabled") .booleanConf .createOptional diff --git a/docs/configuration.md b/docs/configuration.md index 01142ca5e8577..5f8a7cb8450f1 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1149,17 +1149,63 @@ Apart from these, the following properties are also available, and may be useful - spark.scheduler.blacklist.enabled - true + spark.blacklist.enabled + + true in cluster mode;
+ false in local mode + + + 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. + + + + spark.blacklist.recoveryTime + 1h + + 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. + + + + spark.blacklist.maxTaskAttemptsPerNode + 2 + + For a given task, how many executors the task can fail on, before the entire node is blacklisted + for the given task + + + + spark.blacklist.maxFailedTasksPerExecutorStage + 2 + + How many different tasks must fail on one executor, within one stage, before the executor is + blacklisted for that stage. + + + + spark.blacklist.maxFailedExecutorsPerNodeStage + 2 - 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.scheduler.blacklist" configuration options. + How many different executors are marked as failed for a given stage, before the entire node + is marked as failed for the stage. - spark.scheduler.blacklist.MORE - TODO + spark.blacklist.maxFailedTasksPerExecutor + 2 + + How many different tasks must fail on one executor, in successful task sets, before the executor + is blacklisted for the entire application. + + + + spark.blacklist.maxFailedExecutorsPerNode + 2 - TODO + How many different executors must be blacklisted for the entire application, before the node is + blacklisted for the entire application. From 8cebb01a0ad19eeccd4143557b7cf22836041c70 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 20 Jul 2016 12:35:04 -0500 Subject: [PATCH 16/57] api simplification --- .../spark/scheduler/BlacklistTracker.scala | 19 +------------------ .../spark/scheduler/TaskSchedulerImpl.scala | 3 --- .../spark/scheduler/TaskSetManager.scala | 2 -- .../scheduler/BlacklistTrackerSuite.scala | 3 --- .../scheduler/TaskSchedulerImplSuite.scala | 2 -- 5 files changed, 1 insertion(+), 28 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala index c70e8122cac18..bd91611e27a8e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala @@ -60,10 +60,6 @@ private[scheduler] class BlacklistTracker ( private val _nodeBlacklist: AtomicReference[Set[String]] = new AtomicReference(Set()) private var nextExpiryTime: Long = Long.MaxValue - def start(): Unit = {} - - def stop(): Unit = {} - def expireExecutorsInBlacklist(): Unit = { val now = clock.getTimeMillis() // quickly check if we've got anything to expire from blacklist -- if not, avoid doing any work @@ -98,7 +94,7 @@ private[scheduler] class BlacklistTracker ( val newTotal = prevFailures + newFailures.totalFailures if (newTotal >= MAX_FAILURES_PER_EXEC) { - logInfo(s"Blacklisting executor $exec because it has $newTotal" + + logInfo(s"Blacklisting executor id: $exec because it has $newTotal" + s" task failures in successful task sets") val now = clock.getTimeMillis() val expiryTime = now + EXECUTOR_RECOVERY_MILLIS @@ -124,9 +120,6 @@ private[scheduler] class BlacklistTracker ( } } - def taskSetFailed(stageId: Int): Unit = { - } - def isExecutorBlacklisted(executorId: String): Boolean = { executorIdToBlacklistExpiryTime.contains(executorId) } @@ -143,16 +136,6 @@ private[scheduler] class BlacklistTracker ( nodeIdToBlacklistExpiryTime.contains(node) } - def taskSucceeded( - stageId: Int, - indexInTaskSet: Int, - info: TaskInfo, - scheduler: TaskSchedulerImpl): Unit = { - // no-op intentionally, included just for symmetry. success to failure ratio is irrelevant, we - // just blacklist based on failures. Furthermore, one success does not clear previous - // failures, since the bad node / executor may not fail *every* time - } - def removeExecutor(executorId: String): Unit = { executorIdToBlacklistExpiryTime -= executorId executorIdToFailureCount -= executorId 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 41cfb236d01d8..7584a699857a2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -162,7 +162,6 @@ private[spark] class TaskSchedulerImpl private[scheduler]( override def start() { backend.start() - blacklistTracker.foreach(_.start()) if (!isLocal && conf.getBoolean("spark.speculation", false)) { logInfo("Starting speculative execution thread") @@ -259,7 +258,6 @@ private[spark] class TaskSchedulerImpl private[scheduler]( logInfo(s"Removed TaskSet ${manager.taskSet.id}, whose tasks have all completed, from pool" + s" ${manager.parent.name}") } else { - blacklistTracker.foreach(_.taskSetFailed(manager.taskSet.stageId)) logInfo(s"Removed TaskSet ${manager.taskSet.id}, since it failed, from pool" + s" ${manager.parent.name}") } @@ -523,7 +521,6 @@ private[spark] class TaskSchedulerImpl private[scheduler]( override def stop() { speculationScheduler.shutdown() - blacklistTracker.foreach(_.stop()) if (backend != null) { backend.stop() } 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 d25f2b5f8a7ef..4cc527c3a21a8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -709,8 +709,6 @@ private[spark] class TaskSetManager( logInfo("Ignoring task-finished event for " + info.id + " in stage " + taskSet.id + " because task " + index + " has already completed successfully") } - - blacklistTracker.foreach(_.taskSucceeded(stageId, index, info, sched)) maybeFinishTaskSet() } diff --git a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala index 4f2bf9a841450..831ae672e9363 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala @@ -34,7 +34,6 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M override def afterEach(): Unit = { if (blacklistTracker != null) { - blacklistTracker.stop() blacklistTracker = null } super.afterEach() @@ -168,7 +167,6 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M val taskSet = FakeTask.createTaskSet(1) val tsm = new TaskSetManager(scheduler, Some(tracker), taskSet, 4, clock) tsm.updateBlacklistForFailedTask("hostA", "1", 0) - tracker.taskSetFailed(stage) } assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set()) } @@ -194,7 +192,6 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M } else { // the task set failed, so we don't count these failures against the executor for other // stages - tracker.taskSetFailed(stageId) assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set()) } } 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 93569c8d9e7f1..55babcac79163 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -438,7 +438,6 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B // the tasksSets complete, so the tracker should be notified verify(blacklist, times(1)).taskSetSucceeded(stageToMockTsm(0).execToFailures, taskScheduler) verify(blacklist, times(1)).taskSetSucceeded(stageToMockTsm(1).execToFailures, taskScheduler) - verify(blacklist, times(1)).taskSetFailed(2) } test("scheduled tasks obey node and executor blacklists") { @@ -484,7 +483,6 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B // we should have aborted the existing stages, since they aren't schedulable (0 to 2).foreach { stageId => assert(stageToTsm(stageId).isZombie) - verify(blacklist).taskSetFailed(stageId) } } From dbf904e80ad892b76d29bf0092db5810d14b3271 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 20 Jul 2016 13:49:07 -0500 Subject: [PATCH 17/57] review feedback --- .../spark/scheduler/BlacklistTracker.scala | 50 +++++++++------ .../spark/scheduler/TaskSchedulerImpl.scala | 2 +- .../spark/scheduler/TaskSetManager.scala | 15 +++-- .../scheduler/BlacklistTrackerSuite.scala | 64 ++++++++++++++----- .../scheduler/TaskSchedulerImplSuite.scala | 6 +- 5 files changed, 95 insertions(+), 42 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala index bd91611e27a8e..b06a568fe00aa 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala @@ -54,23 +54,34 @@ private[scheduler] class BlacklistTracker ( // a count of failed tasks for each executor. Only counts failures after tasksets complete // successfully - private val executorIdToFailureCount: HashMap[String, Int] = HashMap() - private val executorIdToBlacklistExpiryTime: HashMap[String, Long] = new HashMap() + private val executorIdToFailureCount: HashMap[String, Int] = new HashMap() + private val executorIdToBlacklistStatus: HashMap[String, BlacklistedExecutor] = new HashMap() private val nodeIdToBlacklistExpiryTime: HashMap[String, Long] = new HashMap() private val _nodeBlacklist: AtomicReference[Set[String]] = new AtomicReference(Set()) private var nextExpiryTime: Long = Long.MaxValue + // for blacklisted executors, the node it is on. 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. + val nodeToFailedExecs: HashMap[String, HashSet[String]] = new HashMap() def expireExecutorsInBlacklist(): 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) { - val execsToClear = executorIdToBlacklistExpiryTime.filter(_._2 < now).keys + val execsToClear = executorIdToBlacklistStatus.filter(_._2.expiryTime < now).keys if (execsToClear.nonEmpty) { logInfo(s"Removing executors $execsToClear from blacklist during periodic recovery") - execsToClear.foreach { exec => executorIdToBlacklistExpiryTime.remove(exec) } + execsToClear.foreach { exec => + val status = executorIdToBlacklistStatus.remove(exec).get + val failedExecsOnNode = nodeToFailedExecs(status.node) + failedExecsOnNode.remove(exec) + if (failedExecsOnNode.isEmpty) { + nodeToFailedExecs.remove(status.node) + } + } } - if (executorIdToBlacklistExpiryTime.nonEmpty) { - nextExpiryTime = executorIdToBlacklistExpiryTime.map{_._2}.min + if (executorIdToBlacklistStatus.nonEmpty) { + nextExpiryTime = executorIdToBlacklistStatus.map{_._2.expiryTime}.min } else { nextExpiryTime = Long.MaxValue } @@ -84,32 +95,30 @@ private[scheduler] class BlacklistTracker ( } } - def taskSetSucceeded( - failuresByExec: HashMap[String, FailureStatus], - scheduler: TaskSchedulerImpl): Unit = { + def taskSetSucceeded(failuresByExec: HashMap[String, FailureStatus]): Unit = { // if any tasks failed, we count them towards the overall failure count for the executor at // this point. failuresByExec.foreach { case (exec, newFailures) => val prevFailures = executorIdToFailureCount.getOrElse(exec, 0) val newTotal = prevFailures + newFailures.totalFailures + val node = newFailures.node if (newTotal >= MAX_FAILURES_PER_EXEC) { logInfo(s"Blacklisting executor id: $exec because it has $newTotal" + s" task failures in successful task sets") val now = clock.getTimeMillis() val expiryTime = now + EXECUTOR_RECOVERY_MILLIS - executorIdToBlacklistExpiryTime.put(exec, expiryTime) + executorIdToBlacklistStatus.put(exec, BlacklistedExecutor(node, expiryTime)) + val blacklistedExecsOnNode = nodeToFailedExecs.getOrElseUpdate(node, HashSet[String]()) + blacklistedExecsOnNode += exec executorIdToFailureCount.remove(exec) if (expiryTime < nextExpiryTime) { nextExpiryTime = expiryTime } - val node = scheduler.getHostForExecutor(exec) - val execs = scheduler.getExecutorsAliveOnHost(node).getOrElse(Set()) - val blacklistedExecs = execs.filter(executorIdToBlacklistExpiryTime.contains(_)) - if (blacklistedExecs.size >= MAX_FAILED_EXEC_PER_NODE) { - logInfo(s"Blacklisting node $node because it has ${blacklistedExecs.size} executors " + - s"blacklisted: ${blacklistedExecs}") + if (blacklistedExecsOnNode.size >= MAX_FAILED_EXEC_PER_NODE) { + logInfo(s"Blacklisting node $node because it has ${blacklistedExecsOnNode.size} " + + s"executors blacklisted: ${blacklistedExecsOnNode}") nodeIdToBlacklistExpiryTime.put(node, expiryTime) // make a copy of the blacklisted nodes so nodeBlacklist() is threadsafe _nodeBlacklist.set(nodeIdToBlacklistExpiryTime.keySet.toSet) @@ -121,7 +130,7 @@ private[scheduler] class BlacklistTracker ( } def isExecutorBlacklisted(executorId: String): Boolean = { - executorIdToBlacklistExpiryTime.contains(executorId) + executorIdToBlacklistStatus.contains(executorId) } /** @@ -137,7 +146,8 @@ private[scheduler] class BlacklistTracker ( } def removeExecutor(executorId: String): Unit = { - executorIdToBlacklistExpiryTime -= executorId + // we intentionally do not clean up executors that are already blacklisted, so that if another + // executor on the same node gets blacklisted, we can blacklist the entire node. executorIdToFailureCount -= executorId } } @@ -193,7 +203,7 @@ private[scheduler] object BlacklistTracker extends Logging { } /** Failures for one executor, within one taskset */ -private[scheduler] final class FailureStatus { +private[scheduler] final class FailureStatus(val node: String) { /** index of the tasks in the taskset that have failed on this executor. */ val tasksWithFailures = HashSet[Int]() var totalFailures = 0 @@ -202,3 +212,5 @@ private[scheduler] final class FailureStatus { s"totalFailures = $totalFailures; tasksFailed = $tasksWithFailures" } } + +private final case class BlacklistedExecutor(node: String, expiryTime: Long) 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 7584a699857a2..1ef1d9bf76502 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -254,7 +254,7 @@ private[spark] class TaskSchedulerImpl private[scheduler]( } manager.parent.removeSchedulable(manager) if (success) { - blacklistTracker.foreach(_.taskSetSucceeded(manager.execToFailures, this)) + blacklistTracker.foreach(_.taskSetSucceeded(manager.execToFailures)) logInfo(s"Removed TaskSet ${manager.taskSet.id}, whose tasks have all completed, from pool" + s" ${manager.parent.name}") } else { 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 4cc527c3a21a8..56641e010e675 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -86,8 +86,13 @@ private[spark] class TaskSetManager( val successful = new Array[Boolean](numTasks) private val numFailures = new Array[Int](numTasks) val execToFailures: HashMap[String, FailureStatus] = new HashMap() - val nodeBlacklistedTasks: HashMap[String, HashSet[Int]] = new HashMap() - val blacklistedNodes: HashSet[String] = new HashSet() + /** + * Map from node to all executors on it with failures. Needed because we want to know about + * executors on a node even after they have died. + */ + private val nodesToExecsWithFailures: HashMap[String, HashSet[String]] = new HashMap() + private val nodeBlacklistedTasks: HashMap[String, HashSet[Int]] = new HashMap() + private val blacklistedNodes: HashSet[String] = new HashSet() val taskAttempts = Array.fill[List[TaskInfo]](numTasks)(Nil) @@ -830,14 +835,16 @@ private[spark] class TaskSetManager( host: String, exec: String, index: Int): Unit = { - val failureStatus = execToFailures.getOrElseUpdate(exec, new FailureStatus()) + val failureStatus = execToFailures.getOrElseUpdate(exec, new FailureStatus(host)) failureStatus.totalFailures += 1 failureStatus.tasksWithFailures += index // check if this task has also failed on other executors on the same host, and if so, blacklist // this task from the host + val execsWithFailuresOnNode = nodesToExecsWithFailures.getOrElseUpdate(host, new HashSet()) + execsWithFailuresOnNode += exec val failuresOnHost = (for { - exec <- sched.getExecutorsAliveOnHost(host).getOrElse(Set()).toSeq + exec <- execsWithFailuresOnNode.toIterator failures <- execToFailures.get(exec) } yield { if (failures.tasksWithFailures.contains(index)) 1 else 0 diff --git a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala index 831ae672e9363..5c71021388078 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala @@ -67,11 +67,6 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M val conf = new SparkConf().setAppName("test").setMaster("local") .set(BLACKLIST_ENABLED.key, "true") val scheduler = mockTaskSchedWithConf(conf) - when(scheduler.getExecutorsAliveOnHost("hostA")).thenReturn(Some(Set("1", "2", "4"))) - Set("1", "2", "4").foreach { execId => - when(scheduler.getHostForExecutor(execId)).thenReturn("hostA") - } - // Task 1 failed on executor 1 blacklistTracker = new BlacklistTracker(conf, clock) val taskSet = FakeTask.createTaskSet(10) @@ -118,7 +113,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M // when the stage completes successfully, now there is sufficient evidence we've got // bad executors and node - blacklistTracker.taskSetSucceeded(tsm.execToFailures, scheduler) + blacklistTracker.taskSetSucceeded(tsm.execToFailures) assert(blacklistTracker.nodeBlacklist() === Set("hostA")) assertEquivalentToSet(blacklistTracker.isNodeBlacklisted(_), Set("hostA")) assertEquivalentToSet(blacklistTracker.isExecutorBlacklisted(_), Set("1", "2")) @@ -134,10 +129,6 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M val conf = new SparkConf().setAppName("test").setMaster("local") .set(BLACKLIST_ENABLED.key, "true") val scheduler = mockTaskSchedWithConf(conf) - when(scheduler.getExecutorsAliveOnHost("hostA")).thenReturn(Some(Set("1", "2", "4"))) - Set("1", "2", "4").foreach { execId => - when(scheduler.getHostForExecutor(execId)).thenReturn("hostA") - } clock.setTime(0) blacklistTracker = new BlacklistTracker(conf, clock) @@ -153,7 +144,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M val taskSet = FakeTask.createTaskSet(1) val tsm = new TaskSetManager(scheduler, Some(tracker), taskSet, 4, clock) tsm.updateBlacklistForFailedTask("hostA", "1", 0) - tracker.taskSetSucceeded(tsm.execToFailures, scheduler) + tracker.taskSetSucceeded(tsm.execToFailures) } assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set("1")) } @@ -187,7 +178,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M if (succeedTaskSet) { // the task set succeeded elsewhere, so we count those failures against our executor, // and blacklist it across stages - tracker.taskSetSucceeded(tsm.execToFailures, scheduler) + tracker.taskSetSucceeded(tsm.execToFailures) assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set("1")) } else { // the task set failed, so we don't count these failures against the executor for other @@ -204,7 +195,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M (0 until 4).foreach { partition => tsm0.updateBlacklistForFailedTask("hostA", "1", partition) } - tracker.taskSetSucceeded(tsm0.execToFailures, scheduler) + tracker.taskSetSucceeded(tsm0.execToFailures) assert(tracker.nodeBlacklist() === Set()) assertEquivalentToSet(tracker.isNodeBlacklisted(_), Set()) assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set("1")) @@ -214,7 +205,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M (0 until 4).foreach { partition => tsm1.updateBlacklistForFailedTask("hostA", "2", partition) } - tracker.taskSetSucceeded(tsm1.execToFailures, scheduler) + tracker.taskSetSucceeded(tsm1.execToFailures) assert(tracker.nodeBlacklist() === Set("hostA")) assertEquivalentToSet(tracker.isNodeBlacklisted(_), Set("hostA")) assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set("1", "2")) @@ -229,12 +220,55 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M val taskSet2 = FakeTask.createTaskSet(4, 2, 0) val tsm2 = new TaskSetManager(scheduler, Some(tracker), taskSet2, 4, clock) tsm2.updateBlacklistForFailedTask("hostA", "1", 0) - tracker.taskSetSucceeded(tsm2.execToFailures, scheduler) + tracker.taskSetSucceeded(tsm2.execToFailures) assert(tracker.nodeBlacklist() === Set()) assertEquivalentToSet(tracker.isNodeBlacklisted(_), Set()) assertEquivalentToSet(tracker.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 (tracker, scheduler) = trackerFixture + when(scheduler.getExecutorsAliveOnHost("hostA")).thenReturn(Some(Set("1"))) + val taskSet0 = FakeTask.createTaskSet(4) + val tsm0 = new TaskSetManager(scheduler, Some(tracker), taskSet0, 4, clock) + (0 until 3).foreach { partition => + tsm0.updateBlacklistForFailedTask("hostA", "1", partition) + } + // now lets say that executor 1 dies completely + when(scheduler.getExecutorsAliveOnHost("hostA")).thenReturn(Some(Set[String]())) + when(scheduler.getHostForExecutor("1")).thenThrow(new NoSuchElementException("1")) + // we get a task failure for the last task + tsm0.updateBlacklistForFailedTask("hostA", "1", 3) + tracker.taskSetSucceeded(tsm0.execToFailures) + assert(tracker.isExecutorBlacklisted("1")) + clock.advance(tracker.EXECUTOR_RECOVERY_MILLIS / 2) + + // say another executor gets spun up on that host + when(scheduler.getExecutorsAliveOnHost("hostA")).thenReturn(Some(Set("2"))) + val taskSet1 = FakeTask.createTaskSet(4) + val tsm1 = new TaskSetManager(scheduler, Some(tracker), taskSet1, 4, clock) + (0 until 4).foreach { partition => + tsm1.updateBlacklistForFailedTask("hostA", "2", partition) + } + tracker.taskSetSucceeded(tsm1.execToFailures) + // we've now had two bad executors on the hostA, so we should blacklist the entire node + assert(tracker.isExecutorBlacklisted("1")) + assert(tracker.isExecutorBlacklisted("2")) + assert(tracker.isNodeBlacklisted("hostA")) + + clock.advance(tracker.EXECUTOR_RECOVERY_MILLIS / 2 + 1) + tracker.expireExecutorsInBlacklist() + // executor 1 is no longer explicitly blacklisted, since we've gone past its recovery time, + // but everything else is still blacklisted. + assert(!tracker.isExecutorBlacklisted("1")) + assert(tracker.isExecutorBlacklisted("2")) + assert(tracker.isNodeBlacklisted("hostA")) + // make sure we don't leak memory + assert(!tracker.nodeToFailedExecs("hostA").contains("1")) + } + test("blacklist still respects legacy configs") { val legacyKey = BLACKLIST_LEGACY_TIMEOUT_CONF.key 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 55babcac79163..436a70e76207a 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -98,7 +98,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B stageToMockTsm(taskSet.stageId) = tsmSpy // intentionally bogus, just lets us easily verify val execToFailures = new HashMap[String, FailureStatus]() - execToFailures(taskSet.stageId.toString) = new FailureStatus() + execToFailures(taskSet.stageId.toString) = new FailureStatus("dummy") when(tsmSpy.execToFailures).thenReturn(execToFailures) tsmSpy } @@ -436,8 +436,8 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B } // the tasksSets complete, so the tracker should be notified - verify(blacklist, times(1)).taskSetSucceeded(stageToMockTsm(0).execToFailures, taskScheduler) - verify(blacklist, times(1)).taskSetSucceeded(stageToMockTsm(1).execToFailures, taskScheduler) + verify(blacklist, times(1)).taskSetSucceeded(stageToMockTsm(0).execToFailures) + verify(blacklist, times(1)).taskSetSucceeded(stageToMockTsm(1).execToFailures) } test("scheduled tasks obey node and executor blacklists") { From f0de0db8c54ac2dc66b0ab59b1b6c155a6775014 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 20 Jul 2016 16:02:14 -0500 Subject: [PATCH 18/57] fix for config name change --- .../src/test/scala/org/apache/spark/DistributedSuite.scala | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/DistributedSuite.scala b/core/src/test/scala/org/apache/spark/DistributedSuite.scala index 6193019dd9c2d..e03c37f319887 100644 --- a/core/src/test/scala/org/apache/spark/DistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/DistributedSuite.scala @@ -21,6 +21,7 @@ import org.scalatest.concurrent.Timeouts._ import org.scalatest.Matchers import org.scalatest.time.{Millis, Span} +import org.apache.spark.internal.config.BLACKLIST_ENABLED import org.apache.spark.storage.{RDDBlockId, StorageLevel} import org.apache.spark.util.io.ChunkedByteBuffer @@ -109,7 +110,7 @@ class DistributedSuite extends SparkFunSuite with Matchers with LocalSparkContex test("repeatedly failing task") { val conf = new SparkConf().setAppName("test").setMaster(clusterUrl) - .set("spark.scheduler.blacklist.enabled", "false") + .set(BLACKLIST_ENABLED, false) sc = new SparkContext(conf) val thrown = intercept[SparkException] { // scalastyle:off println @@ -125,7 +126,9 @@ class DistributedSuite extends SparkFunSuite with Matchers with LocalSparkContex // than hanging due to retrying the failed task infinitely many times (eventually the // standalone scheduler will remove the application, causing the job to hang waiting to // reconnect to the master). - sc = new SparkContext(clusterUrl, "test") + val conf = new SparkConf().setAppName("test").setMaster(clusterUrl) + .set(BLACKLIST_ENABLED, false) + sc = new SparkContext(conf) failAfter(Span(100000, Millis)) { val thrown = intercept[SparkException] { // One of the tasks always fails. From 8a12adf445b00e8841eb3df071c0b6adee6c16da Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Fri, 22 Jul 2016 12:16:46 -0500 Subject: [PATCH 19/57] exclude killed tasks and preempted tasks from blacklist --- .../org/apache/spark/TaskEndReason.scala | 10 +++ .../spark/scheduler/TaskSetManager.scala | 11 +-- .../scheduler/TaskSchedulerImplSuite.scala | 13 +++- .../spark/scheduler/TaskSetManagerSuite.scala | 76 ++++++++++++++++--- 4 files changed, 92 insertions(+), 18 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala index 42690844f9610..59a25398a039b 100644 --- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala +++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala @@ -92,6 +92,15 @@ case class FetchFailed( s"FetchFailed($bmAddressString, shuffleId=$shuffleId, mapId=$mapId, reduceId=$reduceId, " + s"message=\n$message\n)" } + + /** + * 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. + */ + override def countTowardsTaskFailures: Boolean = false } /** @@ -204,6 +213,7 @@ case object TaskResultLost extends TaskFailedReason { @DeveloperApi case object TaskKilled extends TaskFailedReason { override def toErrorString: String = "TaskKilled (killed intentionally)" + override def countTowardsTaskFailures: Boolean = false } /** 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 56641e010e675..53ad18718614b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -740,7 +740,6 @@ private[spark] class TaskSetManager( successful(index) = true tasksSuccessful += 1 } - // Not adding to failed executors for FetchFailed. isZombie = true None @@ -800,7 +799,11 @@ private[spark] class TaskSetManager( // that bad node will get handled separately by spark's stage-failure handling mechanism. It // shouldn't penalize *this* executor at all, so don't count it as a task-failure as far as // the blacklist is concerned. - if (!reason.isInstanceOf[FetchFailed]) { + val countTowardsTaskFailures = reason match { + case fail: TaskFailedReason => fail.countTowardsTaskFailures + case Success => false + } + if (countTowardsTaskFailures && blacklistTracker.isDefined) { updateBlacklistForFailedTask(info.host, info.executorId, index) } @@ -815,9 +818,7 @@ private[spark] class TaskSetManager( addPendingTask(index) } - if (!isZombie && state != TaskState.KILLED - && reason.isInstanceOf[TaskFailedReason] - && reason.asInstanceOf[TaskFailedReason].countTowardsTaskFailures) { + if (!isZombie && countTowardsTaskFailures) { assert (null != failureReason) numFailures(index) += 1 if (numFailures(index) >= maxTaskFailures) { 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 436a70e76207a..32a069933dce0 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -664,18 +664,25 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B verify(blacklist).expireExecutorsInBlacklist() } - test("don't update blacklist for shuffle-fetch failures") { + test("don't update blacklist for shuffle-fetch failures, preemption, denied commits, " + + "or killed tasks") { val blacklist = mock[BlacklistTracker] taskScheduler = setupSchedulerWithMockTsm(blacklist) - val stage0 = FakeTask.createTaskSet(numTasks = 2, stageId = 0, stageAttemptId = 0) + val stage0 = FakeTask.createTaskSet(numTasks = 4, stageId = 0, stageAttemptId = 0) taskScheduler.submitTasks(stage0) val taskDescs = taskScheduler.resourceOffers( Seq(new WorkerOffer("executor0", "host0", 10))).flatten - assert(taskDescs.size === 2) + assert(taskDescs.size === 4) val tsm = stageToMockTsm(0) taskScheduler.handleFailedTask(tsm, taskDescs(0).taskId, TaskState.FAILED, FetchFailed(BlockManagerId("executor1", "host1", 12345), 0, 0, 0, "ignored")) + taskScheduler.handleFailedTask(tsm, taskDescs(1).taskId, TaskState.FAILED, + ExecutorLostFailure("executor0", exitCausedByApp = false, reason = None)) + taskScheduler.handleFailedTask(tsm, taskDescs(2).taskId, TaskState.FAILED, + TaskCommitDenied(0, 2, 0)) + taskScheduler.handleFailedTask(tsm, taskDescs(3).taskId, TaskState.KILLED, + TaskKilled) verify(tsm, never()).updateBlacklistForFailedTask(anyString(), anyString(), anyInt()) } } 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 8cecd3282ff2a..57435cded6d95 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -821,16 +821,19 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg test("Kill other task attempts when one attempt belonging to the same task succeeds") { sc = new SparkContext("local", "test") sched = new FakeTaskScheduler(sc, ("exec1", "host1"), ("exec2", "host2")) - val taskSet = FakeTask.createTaskSet(4) + val taskSet = FakeTask.createTaskSet(5) // Set the speculation multiplier to be 0 so speculative tasks are launched immediately sc.conf.set("spark.speculation.multiplier", "0.0") + sc.conf.set("spark.speculation.quantile", "0.6") val clock = new ManualClock() val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) val accumUpdatesByTask: Array[Seq[AccumulatorV2[_, _]]] = taskSet.tasks.map { task => task.metrics.internalAccums } // Offer resources for 4 tasks to start + val tasks = new ArrayBuffer[TaskDescription]() for ((k, v) <- List( + "exec1" -> "host1", "exec1" -> "host1", "exec1" -> "host1", "exec2" -> "host2", @@ -839,14 +842,44 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg assert(taskOption.isDefined) val task = taskOption.get assert(task.executorId === k) + tasks += task } - assert(sched.startedTasks.toSet === Set(0, 1, 2, 3)) - // Complete the 3 tasks and leave 1 task in running + assert(sched.startedTasks.toSet === (0 until 5).toSet) + // Complete 3 tasks and leave 2 task in running for (id <- Set(0, 1, 2)) { manager.handleSuccessfulTask(id, createTaskResult(id, accumUpdatesByTask(id))) assert(sched.endedTasks(id) === Success) } + def runningTaskForIndex(index: Int): TaskDescription = { + val t = tasks.find { task => task.index == index && !sched.endedTasks.contains(task.taskId) } + t match { + case Some(x) => x + case None => + throw new RuntimeException(s"couldn't find index $index in " + + s"tasks: ${tasks.map{t => t.index -> t.taskId}} with endedTasks:" + + s" ${sched.endedTasks.keys}") + } + } + + // have each of the running tasks fail 3 times (not enough to abort the stage) + (3 until 6).foreach { attempt => + Seq(3, 4).foreach { index => + val task = runningTaskForIndex(index) + logInfo(s"failing task $task") + val endReason = ExceptionFailure("a", "b", Array(), "c", None) + manager.handleFailedTask(task.taskId, TaskState.FAILED, endReason) + sched.endedTasks(task.taskId) = endReason + assert(!manager.isZombie) + val nextTask = manager.resourceOffer(s"exec2", s"host2", NO_PREF) + assert(nextTask.isDefined, s"no offer for attempt $attempt of $index") + tasks += nextTask.get + } + } + + // we can't be sure which one of our running tasks will get anothe speculative copy + val originalTasks = Seq(3, 4).map { index => index -> runningTaskForIndex(index) }.toMap + // checkSpeculatableTasks checks that the task runtime is greater than the threshold for // speculating. Since we use a threshold of 0 for speculation, tasks need to be running for // > 0ms, so advance the clock by 1ms here. @@ -855,20 +888,43 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg // Offer resource to start the speculative attempt for the running task val taskOption5 = manager.resourceOffer("exec1", "host1", NO_PREF) assert(taskOption5.isDefined) - val task5 = taskOption5.get - assert(task5.index === 3) - assert(task5.taskId === 4) - assert(task5.executorId === "exec1") - assert(task5.attemptNumber === 1) + val speculativeTask = taskOption5.get + assert(speculativeTask.index === 3 || speculativeTask.index === 4) + assert(speculativeTask.taskId === 11) + assert(speculativeTask.executorId === "exec1") + assert(speculativeTask.attemptNumber === 4) sched.backend = mock(classOf[SchedulerBackend]) // Complete the speculative attempt for the running task - manager.handleSuccessfulTask(4, createTaskResult(3, accumUpdatesByTask(3))) + manager.handleSuccessfulTask(speculativeTask.taskId, createTaskResult(3, accumUpdatesByTask(3))) // Verify that it kills other running attempt - verify(sched.backend).killTask(3, "exec2", true) + val origTask = originalTasks(speculativeTask.index) + verify(sched.backend).killTask(origTask.taskId, "exec2", true) // Because the SchedulerBackend was a mock, the 2nd copy of the task won't actually be // killed, so the FakeTaskScheduler is only told about the successful completion // of the speculated task. assert(sched.endedTasks(3) === Success) + // also because the scheduler is a mock, our manager isn't notified about the task killed event, + // so we do that manually + manager.handleFailedTask(origTask.taskId, TaskState.KILLED, TaskKilled) + // this task has "failed" 4 times, but one of them doesn't count, so keep running the stage + assert(manager.tasksSuccessful === 4) + assert(!manager.isZombie) + + // now run another speculative task + val taskOpt6 = manager.resourceOffer("exec1", "host1", NO_PREF) + assert(taskOpt6.isDefined) + val speculativeTask2 = taskOpt6.get + assert(speculativeTask2.index === 3 || speculativeTask2.index === 4) + assert(speculativeTask2.index !== speculativeTask.index) + assert(speculativeTask2.attemptNumber === 4) + // Complete the speculative attempt for the running task + manager.handleSuccessfulTask(speculativeTask2.taskId, + createTaskResult(3, accumUpdatesByTask(3))) + // Verify that it kills other running attempt + val origTask2 = originalTasks(speculativeTask2.index) + verify(sched.backend).killTask(origTask2.taskId, "exec2", true) + assert(manager.tasksSuccessful === 5) + assert(manager.isZombie) } private def createTaskResult( From c9e3662dd468fe262fcb663ca1898cac0029f8d7 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 26 Jul 2016 11:20:58 -0500 Subject: [PATCH 20/57] combine imports --- .../scala/org/apache/spark/scheduler/BlacklistTracker.scala | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala index b06a568fe00aa..ae7e359ddc79d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala @@ -24,9 +24,7 @@ import scala.collection.mutable.{HashMap, HashSet} import org.apache.spark.SparkConf import org.apache.spark.internal.Logging import org.apache.spark.internal.config -import org.apache.spark.util.Clock -import org.apache.spark.util.SystemClock -import org.apache.spark.util.Utils +import org.apache.spark.util.{Clock, SystemClock, Utils} /** * BlacklistTracker is designed to track problematic executors and nodes. It supports blacklisting From 497e6268b46549502b615b487ebf26361ecf4623 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 11 Aug 2016 16:26:16 -0500 Subject: [PATCH 21/57] review feedback --- .../org/apache/spark/TaskEndReason.scala | 4 +- .../spark/internal/config/package.scala | 53 ++--- .../spark/scheduler/BlacklistTracker.scala | 138 ++++++++----- .../spark/scheduler/TaskResultGetter.scala | 4 +- .../spark/scheduler/TaskSchedulerImpl.scala | 101 +++------ .../spark/scheduler/TaskSetManager.scala | 194 ++++++++++-------- .../scheduler/BlacklistIntegrationSuite.scala | 4 +- .../scheduler/BlacklistTrackerSuite.scala | 169 ++++++++++++--- .../scheduler/SchedulerIntegrationSuite.scala | 4 +- .../scheduler/TaskSchedulerImplSuite.scala | 38 ++-- .../spark/scheduler/TaskSetManagerSuite.scala | 8 +- docs/configuration.md | 50 +++-- .../spark/deploy/yarn/YarnAllocator.scala | 7 +- 13 files changed, 455 insertions(+), 319 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala index 59a25398a039b..20ea7d04d6480 100644 --- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala +++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala @@ -213,7 +213,7 @@ case object TaskResultLost extends TaskFailedReason { @DeveloperApi case object TaskKilled extends TaskFailedReason { override def toErrorString: String = "TaskKilled (killed intentionally)" - override def countTowardsTaskFailures: Boolean = false + override val countTowardsTaskFailures: Boolean = false } /** @@ -232,7 +232,7 @@ case class TaskCommitDenied( * towards failing the stage. This is intended to prevent spurious stage failures in cases * where many speculative tasks are launched and denied to commit. */ - override def countTowardsTaskFailures: Boolean = false + override val countTowardsTaskFailures: Boolean = false } /** 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 1781862b6c4c3..ad22b6bc0863b 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 @@ -100,44 +100,49 @@ package object config { // Blacklist confs private[spark] val BLACKLIST_ENABLED = ConfigBuilder("spark.blacklist.enabled") - .booleanConf - .createOptional + .booleanConf + .createOptional + + private[spark] val MAX_TASK_ATTEMPTS_PER_EXECUTOR = + ConfigBuilder("spark.blacklist.task.maxTaskAttemptsPerExecutor") + .intConf + .createWithDefault(1) private[spark] val MAX_TASK_ATTEMPTS_PER_NODE = - ConfigBuilder("spark.blacklist.maxTaskAttemptsPerNode") - .intConf - .createWithDefault(2) + ConfigBuilder("spark.blacklist.task.maxTaskAttemptsPerNode") + .intConf + .createWithDefault(2) private[spark] val MAX_FAILURES_PER_EXEC = - ConfigBuilder("spark.blacklist.maxFailedTasksPerExecutor") - .intConf - .createWithDefault(2) + ConfigBuilder("spark.blacklist.application.maxFailedTasksPerExecutor") + .intConf + .createWithDefault(2) private[spark] val MAX_FAILURES_PER_EXEC_STAGE = - ConfigBuilder("spark.blacklist.maxFailedTasksPerExecutorStage") - .intConf - .createWithDefault(2) + ConfigBuilder("spark.blacklist.stage.maxFailedTasksPerExecutor") + .intConf + .createWithDefault(2) private[spark] val MAX_FAILED_EXEC_PER_NODE = - ConfigBuilder("spark.blacklist.maxFailedExecutorsPerNode") - .intConf - .createWithDefault(2) + ConfigBuilder("spark.blacklist.application.maxFailedExecutorsPerNode") + .intConf + .createWithDefault(2) private[spark] val MAX_FAILED_EXEC_PER_NODE_STAGE = - ConfigBuilder("spark.blacklist.maxFailedExecutorsPerNodeStage") - .intConf - .createWithDefault(2) + ConfigBuilder("spark.blacklist.stage.maxFailedExecutorsPerNode") + .intConf + .createWithDefault(2) - private[spark] val BLACKLIST_EXPIRY_TIMEOUT_CONF = - ConfigBuilder("spark.scheduler.blacklist.recoverPeriod") - .timeConf(TimeUnit.MILLISECONDS) - .createOptional + private[spark] val BLACKLIST_TIMEOUT_CONF = + ConfigBuilder("spark.blacklist.timeout") + .timeConf(TimeUnit.MILLISECONDS) + .createOptional private[spark] val BLACKLIST_LEGACY_TIMEOUT_CONF = ConfigBuilder("spark.scheduler.executorTaskBlacklistTime") - .internal() - .timeConf(TimeUnit.MILLISECONDS) - .createOptional + .internal() + .timeConf(TimeUnit.MILLISECONDS) + .createOptional // End blacklist confs // Note: This is a SQL config but needs to be in core because the REPL depends on it diff --git a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala index ae7e359ddc79d..6a79a89e690f0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala @@ -28,15 +28,17 @@ import org.apache.spark.util.{Clock, SystemClock, Utils} /** * BlacklistTracker is designed to track problematic executors and nodes. It supports blacklisting - * specific (executor, task) pairs within a stage, blacklisting entire executors and nodes for a - * stage, and blacklisting executors and nodes across an entire application (with a periodic - * expiry). + * 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, which may lead to many - * task failures, but that should not count against individual executors; many small stages, which - * may prevent a bad executor for having many failures within one stage, but still many failures - * over the entire application; "flaky" executors, that don't fail every task, but are still - * faulty; etc. + * The tracker needs to deal with a variety of workloads, eg.: + * * bad user code, which may lead to many task failures, but that should not count against + * individual executors + * * many small stages, which may prevent a bad executor for having many failures within one stage, + * but still many failures over the entire application + * * "flaky" executors, that don't fail every task, but are still faulty + * * etc. * * 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 @@ -48,28 +50,44 @@ private[scheduler] class BlacklistTracker ( private val MAX_FAILURES_PER_EXEC = conf.get(config.MAX_FAILURES_PER_EXEC) private val MAX_FAILED_EXEC_PER_NODE = conf.get(config.MAX_FAILED_EXEC_PER_NODE) - val EXECUTOR_RECOVERY_MILLIS = BlacklistTracker.getBlacklistExpiryTime(conf) + val BLACKLIST_TIMEOUT_MILLIS = BlacklistTracker.getBlacklistTimeout(conf) - // a count of failed tasks for each executor. Only counts failures after tasksets complete - // successfully + /** + * 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. + */ private val executorIdToFailureCount: HashMap[String, Int] = new HashMap() private val executorIdToBlacklistStatus: HashMap[String, BlacklistedExecutor] = new HashMap() private val nodeIdToBlacklistExpiryTime: HashMap[String, Long] = new HashMap() + /** + * An immutable copy of the set of nodes that are currently blacklisted. Kept in an + * AtomicReference to make [[nodeBlacklist()]] thread-safe. + */ private val _nodeBlacklist: AtomicReference[Set[String]] = new AtomicReference(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. + */ private var nextExpiryTime: Long = Long.MaxValue - // for blacklisted executors, the node it is on. 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. + /** + * 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 periodically clean up the list of blacklisted executors. + */ val nodeToFailedExecs: HashMap[String, HashSet[String]] = new HashMap() def expireExecutorsInBlacklist(): 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) { - val execsToClear = executorIdToBlacklistStatus.filter(_._2.expiryTime < now).keys - if (execsToClear.nonEmpty) { - logInfo(s"Removing executors $execsToClear from blacklist during periodic recovery") - execsToClear.foreach { exec => + 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"has timed out") + execsToUnblacklist.foreach { exec => val status = executorIdToBlacklistStatus.remove(exec).get val failedExecsOnNode = nodeToFailedExecs(status.node) failedExecsOnNode.remove(exec) @@ -83,46 +101,48 @@ private[scheduler] class BlacklistTracker ( } else { nextExpiryTime = Long.MaxValue } - val nodesToClear = nodeIdToBlacklistExpiryTime.filter(_._2 < now).keys - if (nodesToClear.nonEmpty) { - logInfo(s"Removing nodes $nodesToClear from blacklist during periodic recovery") - nodesToClear.foreach { node => nodeIdToBlacklistExpiryTime.remove(node) } - // make a copy of the blacklisted nodes so nodeBlacklist() is threadsafe + 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) } _nodeBlacklist.set(nodeIdToBlacklistExpiryTime.keySet.toSet) } } } - def taskSetSucceeded(failuresByExec: HashMap[String, FailureStatus]): Unit = { + def updateBlacklistForSuccessfulTaskSet( + failuresByExec: HashMap[String, ExecutorFailuresInTaskSet]): Unit = { // if any tasks failed, we count them towards the overall failure count for the executor at // this point. - failuresByExec.foreach { case (exec, newFailures) => + failuresByExec.foreach { case (exec, failuresInTaskSet) => val prevFailures = executorIdToFailureCount.getOrElse(exec, 0) - val newTotal = prevFailures + newFailures.totalFailures - val node = newFailures.node + val newTotal = prevFailures + failuresInTaskSet.numUniqueTasksWithFailures + executorIdToFailureCount.put(exec, newTotal) if (newTotal >= MAX_FAILURES_PER_EXEC) { logInfo(s"Blacklisting executor id: $exec because it has $newTotal" + s" task failures in successful task sets") val now = clock.getTimeMillis() - val expiryTime = now + EXECUTOR_RECOVERY_MILLIS + val expiryTime = now + BLACKLIST_TIMEOUT_MILLIS + val node = failuresInTaskSet.node executorIdToBlacklistStatus.put(exec, BlacklistedExecutor(node, expiryTime)) - val blacklistedExecsOnNode = nodeToFailedExecs.getOrElseUpdate(node, HashSet[String]()) - blacklistedExecsOnNode += exec executorIdToFailureCount.remove(exec) if (expiryTime < nextExpiryTime) { nextExpiryTime = expiryTime } + // 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 = nodeToFailedExecs.getOrElseUpdate(node, HashSet[String]()) + blacklistedExecsOnNode += exec if (blacklistedExecsOnNode.size >= MAX_FAILED_EXEC_PER_NODE) { logInfo(s"Blacklisting node $node because it has ${blacklistedExecsOnNode.size} " + s"executors blacklisted: ${blacklistedExecsOnNode}") nodeIdToBlacklistExpiryTime.put(node, expiryTime) - // make a copy of the blacklisted nodes so nodeBlacklist() is threadsafe _nodeBlacklist.set(nodeIdToBlacklistExpiryTime.keySet.toSet) } - } else { - executorIdToFailureCount.put(exec, newTotal) } } } @@ -143,9 +163,13 @@ private[scheduler] class BlacklistTracker ( nodeIdToBlacklistExpiryTime.contains(node) } - def removeExecutor(executorId: String): Unit = { - // we intentionally do not clean up executors that are already blacklisted, so that if another - // executor on the same node gets blacklisted, we can blacklist the entire node. + def handleRemovedExecutor(executorId: String): Unit = { + // We intentionally do not clean up executors that are already blacklisted in nodeToFailedExecs, + // 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. executorIdToFailureCount -= executorId } } @@ -153,8 +177,11 @@ private[scheduler] class BlacklistTracker ( private[scheduler] object BlacklistTracker extends Logging { + private val DEFAULT_TIMEOUT = "1h" + /** - * Return true if the blacklist is enabled, based on the following order of preferences: + * 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. Use the default for the spark-master: @@ -170,16 +197,18 @@ private[scheduler] object BlacklistTracker extends Logging { // otherwise, use the default based on the cluster-mode (off for local-mode, on otherwise). val legacyKey = config.BLACKLIST_LEGACY_TIMEOUT_CONF.key conf.get(config.BLACKLIST_LEGACY_TIMEOUT_CONF) match { - case Some(legacyTimeout) if legacyTimeout == 0 => - logWarning(s"Turning off blacklisting due to legacy configuaration:" + - s" $legacyKey == 0") - false case Some(legacyTimeout) => + if (legacyTimeout == 0) { + logWarning(s"Turning off blacklisting due to legacy configuaration:" + + s" $legacyKey == 0") + false + } else { // mostly this is necessary just for tests, since real users that want the blacklist // will get it anyway by default logWarning(s"Turning on blacklisting due to legacy configuration:" + s" $legacyKey > 0") true + } case None => // local-cluster is *not* considered local for these purposes, we still want the // blacklist enabled by default @@ -188,26 +217,27 @@ private[scheduler] object BlacklistTracker extends Logging { } } - def getBlacklistExpiryTime(conf: SparkConf): Long = { - val timeoutConf = conf.get(config.BLACKLIST_EXPIRY_TIMEOUT_CONF) - val legacyTimeoutConf = conf.get(config.BLACKLIST_LEGACY_TIMEOUT_CONF) - (timeoutConf, legacyTimeoutConf) match { - case (Some(x), _) => x - case (None, Some(y)) => y - case (None, None) => - Utils.timeStringAsMs("1h") + def getBlacklistTimeout(conf: SparkConf): Long = { + conf.get(config.BLACKLIST_TIMEOUT_CONF).getOrElse { + conf.get(config.BLACKLIST_LEGACY_TIMEOUT_CONF).getOrElse { + Utils.timeStringAsMs(DEFAULT_TIMEOUT) + } } } } /** Failures for one executor, within one taskset */ -private[scheduler] final class FailureStatus(val node: String) { - /** index of the tasks in the taskset that have failed on this executor. */ - val tasksWithFailures = HashSet[Int]() - var totalFailures = 0 +private[scheduler] final class ExecutorFailuresInTaskSet(val node: String) { + /** + * Mapping from index of the tasks in the taskset, to the number of times it has failed on this + * executor. + */ + val taskToFailureCount = HashMap[Int, Int]() + def numUniqueTasksWithFailures: Int = taskToFailureCount.size override def toString(): String = { - s"totalFailures = $totalFailures; tasksFailed = $tasksWithFailures" + s"numUniqueTasksWithFailures= $numUniqueTasksWithFailures; " + + s"tasksToFailureCount = $taskToFailureCount" } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala index 685ef55c66876..1c3fcbd4612a0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala @@ -118,14 +118,14 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul def enqueueFailedTask(taskSetManager: TaskSetManager, tid: Long, taskState: TaskState, serializedData: ByteBuffer) { - var reason : TaskEndReason = UnknownReason + var reason : TaskFailedReason = UnknownReason try { getTaskResultExecutor.execute(new Runnable { override def run(): Unit = Utils.logUncaughtExceptions { val loader = Utils.getContextOrSparkClassLoader try { if (serializedData != null && serializedData.limit() > 0) { - reason = serializer.get().deserialize[TaskEndReason]( + reason = serializer.get().deserialize[TaskFailedReason]( serializedData, loader) } } catch { 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 1ef1d9bf76502..07e143069c866 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -54,8 +54,7 @@ import org.apache.spark.util.{AccumulatorV2, Clock, SystemClock, ThreadUtils, Ut private[spark] class TaskSchedulerImpl private[scheduler]( val sc: SparkContext, val maxTaskFailures: Int, - private[scheduler] val blacklistTracker: Option[BlacklistTracker], - private val clock: Clock = new SystemClock, + private val blacklistTracker: Option[BlacklistTracker], isLocal: Boolean = false) extends TaskScheduler with Logging { @@ -66,7 +65,7 @@ private[spark] class TaskSchedulerImpl private[scheduler]( def this(sc: SparkContext, maxTaskFailures: Int, isLocal: Boolean) = { this(sc, maxTaskFailures, TaskSchedulerImpl.createBlacklistTracker(sc.conf), - clock = new SystemClock, isLocal = isLocal) + isLocal = isLocal) } val conf = sc.conf @@ -110,7 +109,7 @@ private[spark] class TaskSchedulerImpl private[scheduler]( // The set of executors we have on each host; this is used to compute hostsAlive, which // in turn is used to decide when we can attain data locality on a given host - protected val executorsByHost = new HashMap[String, HashSet[String]] + protected val hostToExecutors = new HashMap[String, HashSet[String]] protected val hostsByRack = new HashMap[String, HashSet[String]] @@ -217,7 +216,7 @@ private[spark] class TaskSchedulerImpl private[scheduler]( private[scheduler] def createTaskSetManager( taskSet: TaskSet, maxTaskFailures: Int): TaskSetManager = { - new TaskSetManager(this, taskSet, maxTaskFailures, clock) + new TaskSetManager(this, blacklistTracker, taskSet, maxTaskFailures, new SystemClock) } override def cancelTasks(stageId: Int, interruptThread: Boolean): Unit = synchronized { @@ -245,7 +244,7 @@ private[spark] class TaskSchedulerImpl private[scheduler]( * given TaskSetManager have completed, so state associated with the TaskSetManager should be * cleaned up. */ - def taskSetFinished(manager: TaskSetManager, success: Boolean): Unit = synchronized { + def taskSetFinished(manager: TaskSetManager): Unit = synchronized { taskSetsByStageIdAndAttempt.get(manager.taskSet.stageId).foreach { taskSetsForStage => taskSetsForStage -= manager.taskSet.stageAttemptId if (taskSetsForStage.isEmpty) { @@ -253,20 +252,14 @@ private[spark] class TaskSchedulerImpl private[scheduler]( } } manager.parent.removeSchedulable(manager) - if (success) { - blacklistTracker.foreach(_.taskSetSucceeded(manager.execToFailures)) - logInfo(s"Removed TaskSet ${manager.taskSet.id}, whose tasks have all completed, from pool" + - s" ${manager.parent.name}") - } else { - logInfo(s"Removed TaskSet ${manager.taskSet.id}, since it failed, from pool" + - s" ${manager.parent.name}") - } + logInfo(s"Removed TaskSet ${manager.taskSet.id}, whose tasks have all completed, from pool" + + s" ${manager.parent.name}") } private def resourceOfferSingleTaskSet( taskSet: TaskSetManager, maxLocality: TaskLocality, - shuffledOffers: IndexedSeq[WorkerOffer], + shuffledOffers: Seq[WorkerOffer], availableCpus: Array[Int], tasks: Seq[ArrayBuffer[TaskDescription]]) : Boolean = { var launchedTask = false @@ -276,11 +269,7 @@ private[spark] class TaskSchedulerImpl private[scheduler]( val offer = shuffledOffers(i) val host = offer.host val execId = offer.executorId - val offerBlacklisted = blacklistTracker.map { bl => - taskSet.isNodeBlacklistedForTaskSet(host) || - taskSet.isExecutorBlacklistedForTaskSet(execId) - }.getOrElse(false) - if (!offerBlacklisted && availableCpus(i) >= CPUS_PER_TASK) { + if (availableCpus(i) >= CPUS_PER_TASK) { try { for (task <- taskSet.resourceOffer(execId, host, maxLocality)) { tasks(i) += task @@ -301,23 +290,12 @@ private[spark] class TaskSchedulerImpl private[scheduler]( } } } - if (!launchedTask) { - blacklistTracker.foreach(taskSet.abortIfCompletelyBlacklisted(executorsByHost, _)) + if (!launchedTask && blacklistTracker.isDefined) { + taskSet.abortIfCompletelyBlacklisted(hostToExecutors) } return launchedTask } - private[scheduler] def areAllExecutorsBlacklisted(): Boolean = { - blacklistTracker match { - case Some(bl) => - executorsByHost.forall { case (host, execs) => - bl.isNodeBlacklisted(host) || - execs.forall(bl.isExecutorBlacklisted(_)) - } - case None => false - } - } - /** * Called by cluster manager to offer resources on slaves. We respond by asking our active task * sets for tasks in order of priority. We fill each node with tasks in a round-robin manner so @@ -328,11 +306,11 @@ private[spark] class TaskSchedulerImpl private[scheduler]( // Also track if new executor is added var newExecAvail = false for (o <- offers) { - if (!executorsByHost.contains(o.host)) { - executorsByHost(o.host) = new HashSet[String]() + if (!hostToExecutors.contains(o.host)) { + hostToExecutors(o.host) = new HashSet[String]() } if (!executorIdToTaskCount.contains(o.executorId)) { - executorsByHost(o.host) += o.executorId + hostToExecutors(o.host) += o.executorId executorAdded(o.executorId, o.host) executorIdToHost(o.executorId) = o.host executorIdToTaskCount(o.executorId) = 0 @@ -343,38 +321,21 @@ private[spark] class TaskSchedulerImpl private[scheduler]( } } - // ensure that we periodically check if executors can be removed from the blacklist, without - // requiring a separate thread and added synchronization overhead + // Before making any offers, remove any nodes from the blacklist whose blacklist 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. blacklistTracker.foreach(_.expireExecutorsInBlacklist()) val sortedTaskSets = rootPool.getSortedTaskSetQueue - val filteredOffers: IndexedSeq[WorkerOffer] = (blacklistTracker match { - case Some(bl) => offers.filter { offer => + val filteredOffers = blacklistTracker.map { bl => + offers.filter { offer => !bl.isNodeBlacklisted(offer.host) && !bl.isExecutorBlacklisted(offer.executorId) } - case None => offers - }) match { - // toIndexedSeq always makes an *immutable* IndexedSeq, though we don't care if its mutable - // or immutable. So we do this to avoid making a pointless copy - case is: IndexedSeq[WorkerOffer] => is - case other: Seq[WorkerOffer] => other.toIndexedSeq - } - if (offers.nonEmpty && filteredOffers.isEmpty) { - // Its possible that all the executors are now blacklisted, though we haven't aborted stages - // during the check in resourceOfferSingleTaskSet. If so, fail all existing task sets to - // avoid unschedulability. - if (areAllExecutorsBlacklisted()) { - sortedTaskSets.foreach { tsm => - tsm.abort(s"All executors are blacklisted, so aborting ${tsm.taskSet}") - } - } - return Seq() - } + }.getOrElse(offers) - // Randomly shuffle offers to avoid always placing tasks on the same set of workers. We will - // index into this list by position later, so we want an IndexedSeq so its efficient. - val shuffledOffers: IndexedSeq[WorkerOffer] = Random.shuffle(filteredOffers).toIndexedSeq + // Randomly shuffle offers to avoid always placing tasks on the same set of workers. + val shuffledOffers = Random.shuffle(filteredOffers) // Build a list of tasks to assign to each worker. val tasks = shuffledOffers.map(o => new ArrayBuffer[TaskDescription](o.cores)) val availableCpus = shuffledOffers.map(o => o.cores).toArray @@ -393,7 +354,7 @@ private[spark] class TaskSchedulerImpl private[scheduler]( for (taskSet <- sortedTaskSets; maxLocality <- taskSet.myLocalityLevels) { do { launchedTask = resourceOfferSingleTaskSet( - taskSet, maxLocality, shuffledOffers, availableCpus, tasks) + taskSet, maxLocality, shuffledOffers, availableCpus, tasks) } while (launchedTask) } @@ -487,7 +448,7 @@ private[spark] class TaskSchedulerImpl private[scheduler]( taskSetManager: TaskSetManager, tid: Long, taskState: TaskState, - reason: TaskEndReason): Unit = synchronized { + reason: TaskFailedReason): Unit = synchronized { taskSetManager.handleFailedTask(tid, taskState, reason) if (!taskSetManager.isZombie && taskState != TaskState.KILLED) { // Need to revive offers again now that the task set manager state has been updated to @@ -598,10 +559,10 @@ private[spark] class TaskSchedulerImpl private[scheduler]( executorIdToTaskCount -= executorId val host = executorIdToHost(executorId) - val execs = executorsByHost.getOrElse(host, new HashSet) + val execs = hostToExecutors.getOrElse(host, new HashSet) execs -= executorId if (execs.isEmpty) { - executorsByHost -= host + hostToExecutors -= host for (rack <- getRackForHost(host); hosts <- hostsByRack.get(rack)) { hosts -= host if (hosts.isEmpty) { @@ -614,23 +575,19 @@ private[spark] class TaskSchedulerImpl private[scheduler]( executorIdToHost -= executorId rootPool.executorLost(executorId, host, reason) } - blacklistTracker.foreach(_.removeExecutor(executorId)) + blacklistTracker.foreach(_.handleRemovedExecutor(executorId)) } def executorAdded(execId: String, host: String) { dagScheduler.executorAdded(execId, host) } - def getHostForExecutor(execId: String): String = synchronized { - executorIdToHost(execId) - } - def getExecutorsAliveOnHost(host: String): Option[Set[String]] = synchronized { - executorsByHost.get(host).map(_.toSet) + hostToExecutors.get(host).map(_.toSet) } def hasExecutorsAliveOnHost(host: String): Boolean = synchronized { - executorsByHost.contains(host) + hostToExecutors.contains(host) } def hasHostAliveOnRack(rack: String): Boolean = synchronized { 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 53ad18718614b..72e4988d81284 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -61,10 +61,11 @@ private[spark] class TaskSetManager( taskSet: TaskSet, maxTaskFailures: Int, clock: Clock = new SystemClock()) { - this(sched, sched.blacklistTracker, taskSet, maxTaskFailures, clock) + this(sched, None, taskSet, maxTaskFailures, clock) } - val conf = sched.sc.conf + private val conf = sched.sc.conf + private val MAX_TASK_ATTEMPTS_PER_EXECUTOR = conf.get(config.MAX_TASK_ATTEMPTS_PER_EXECUTOR) private val MAX_TASK_ATTEMPTS_PER_NODE = conf.get(config.MAX_TASK_ATTEMPTS_PER_NODE) private val MAX_FAILURES_PER_EXEC_STAGE = conf.get(config.MAX_FAILURES_PER_EXEC_STAGE) private val MAX_FAILED_EXEC_PER_NODE_STAGE = conf.get(config.MAX_FAILED_EXEC_PER_NODE_STAGE) @@ -85,13 +86,13 @@ private[spark] class TaskSetManager( val copiesRunning = new Array[Int](numTasks) val successful = new Array[Boolean](numTasks) private val numFailures = new Array[Int](numTasks) - val execToFailures: HashMap[String, FailureStatus] = new HashMap() + val execToFailures: HashMap[String, ExecutorFailuresInTaskSet] = new HashMap() /** * Map from node to all executors on it with failures. Needed because we want to know about * executors on a node even after they have died. */ - private val nodesToExecsWithFailures: HashMap[String, HashSet[String]] = new HashMap() - private val nodeBlacklistedTasks: HashMap[String, HashSet[Int]] = new HashMap() + private val nodeToExecsWithFailures: HashMap[String, HashSet[String]] = new HashMap() + private val nodeToBlacklistedTasks: HashMap[String, HashSet[Int]] = new HashMap() private val blacklistedNodes: HashSet[String] = new HashSet() @@ -262,7 +263,7 @@ private[spark] class TaskSetManager( while (indexOffset > 0) { indexOffset -= 1 val index = list(indexOffset) - if (!blacklistedOnExec(execId, host, index)) { + if (!isTaskBlacklistedOnExecOrNode(index, execId, host)) { // This should almost always be list.trimEnd(1) to remove tail list.remove(indexOffset) if (copiesRunning(index) == 0 && !successful(index)) { @@ -278,11 +279,13 @@ private[spark] class TaskSetManager( taskAttempts(taskIndex).exists(_.host == host) } - private def blacklistedOnExec(execId: String, host: String, index: Int): Boolean = { - blacklistTracker.map { bl => + private def isTaskBlacklistedOnExecOrNode(index: Int, execId: String, host: String): Boolean = { + if (blacklistTracker.isDefined) { isNodeBlacklistedForTask(host, index) || isExecutorBlacklistedForTask(execId, index) - }.getOrElse(false) + } else { + false + } } /** @@ -298,7 +301,7 @@ private[spark] class TaskSetManager( def canRunOnHost(index: Int): Boolean = { !hasAttemptOnHost(index, host) && - !blacklistedOnExec(execId, host, index) + !isTaskBlacklistedOnExecOrNode(index, execId, host) } if (!speculatableTasks.isEmpty) { @@ -427,7 +430,11 @@ private[spark] class TaskSetManager( maxLocality: TaskLocality.TaskLocality) : Option[TaskDescription] = { - if (!isZombie) { + val offerBlacklisted = blacklistTracker.map { _ => + isNodeBlacklistedForTaskSet(host) || + isExecutorBlacklistedForTaskSet(execId) + }.getOrElse(false) + if (!isZombie && !offerBlacklisted) { val curTime = clock.getTimeMillis() var allowedLocality = maxLocality @@ -484,7 +491,7 @@ private[spark] class TaskSetManager( // a good proxy to task serialization time. // val timeTaken = clock.getTime() - startTime val taskName = s"task ${info.id} in stage ${taskSet.id}" - logInfo(s"Starting $taskName (TID $taskId, $host, exec ${info.executorId}, " + + logInfo(s"Starting $taskName (TID $taskId, $host, executor ${info.executorId}, " + s"partition ${task.partitionId},$taskLocality, ${serializedTask.limit} bytes)") sched.dagScheduler.taskStarted(task, info) @@ -498,8 +505,10 @@ private[spark] class TaskSetManager( private def maybeFinishTaskSet() { if (isZombie && runningTasks == 0) { - val success = tasksSuccessful == numTasks - sched.taskSetFinished(this, success) + sched.taskSetFinished(this) + if (tasksSuccessful == numTasks) { + blacklistTracker.foreach(_.updateBlacklistForSuccessfulTaskSet(execToFailures)) + } } } @@ -600,51 +609,58 @@ private[spark] class TaskSetManager( * executor until it finds one that the task hasn't failed on already). */ private[scheduler] def abortIfCompletelyBlacklisted( - executorsByHost: HashMap[String, HashSet[String]], - blacklist: BlacklistTracker): Unit = { - - val pendingTask: Option[Int] = { - // usually this will just take the last pending task, but because of the lazy removal - // from each list, we may need to go deeper in the list. We poll from the end because - // failed tasks are put back at the end of allPendingTasks, so we're more likely to find - // an unschedulable task this way. - val indexOffset = allPendingTasks.lastIndexWhere { indexInTaskSet => - copiesRunning(indexInTaskSet) == 0 && !successful(indexInTaskSet) - } - if (indexOffset == -1) { - None - } else { - Some(allPendingTasks(indexOffset)) - } - } - - // If no executors have registered yet, don't abort the stage, just wait. We probably - // got here because a task set was added before the executors registered. - if (executorsByHost.nonEmpty) { - // take any task that needs to be scheduled, and see if we can find some executor it *could* - // run on - pendingTask.foreach { indexInTaskSet => - // try to find some executor this task can run on. Its possible that some *other* - // task isn't schedulable anywhere, but we will discover that in some later call, - // when that unschedulable task is the last task remaining. - val blacklistedEverywhere = executorsByHost.forall { case (host, execs) => - val nodeBlacklisted = blacklist.isNodeBlacklisted(host) || - isNodeBlacklistedForTaskSet(host) || - isNodeBlacklistedForTask(host, indexInTaskSet) - if (nodeBlacklisted) { - true + executorsByHost: HashMap[String, HashSet[String]]): Unit = { + blacklistTracker.foreach { blacklist => + // because this is called in a loop, with multiple resource offers and locality levels, + // we could end up aborting this taskset multiple times without the !isZombie check + if (!isZombie) { + // take any task that needs to be scheduled, and see if we can find some executor it *could* + // run on + val pendingTask: Option[Int] = { + // usually this will just take the last pending task, but because of the lazy removal + // from each list, we may need to go deeper in the list. We poll from the end because + // failed tasks are put back at the end of allPendingTasks, so we're more likely to find + // an unschedulable task this way. + val indexOffset = allPendingTasks.lastIndexWhere { indexInTaskSet => + copiesRunning(indexInTaskSet) == 0 && !successful(indexInTaskSet) + } + if (indexOffset == -1) { + None } else { - execs.forall { exec => - blacklist.isExecutorBlacklisted(exec) || - isExecutorBlacklistedForTaskSet(exec) || - isExecutorBlacklistedForTask(exec, indexInTaskSet) - } + Some(allPendingTasks(indexOffset)) } } - if (blacklistedEverywhere) { - val partition = tasks(indexInTaskSet).partitionId - abort(s"Aborting ${taskSet} because task $indexInTaskSet (partition $partition) cannot " + - s"run anywhere due to node and executor blacklist.") + + // If no executors have registered yet, don't abort the stage, just wait. We probably + // got here because a task set was added before the executors registered. + if (executorsByHost.nonEmpty) { + pendingTask.foreach { indexInTaskSet => + // try to find some executor this task can run on. Its possible that some *other* + // task isn't schedulable anywhere, but we will discover that in some later call, + // when that unschedulable task is the last task remaining. + val blacklistedEverywhere = executorsByHost.forall { case (host, execs) => + // Check if the task can run on the node + val nodeBlacklisted = blacklist.isNodeBlacklisted(host) || + isNodeBlacklistedForTaskSet(host) || + isNodeBlacklistedForTask(host, indexInTaskSet) + if (nodeBlacklisted) { + true + } else { + // Check if the task can run on any of the executors + execs.forall { exec => + blacklist.isExecutorBlacklisted(exec) || + isExecutorBlacklistedForTaskSet(exec) || + isExecutorBlacklistedForTask(exec, indexInTaskSet) + } + } + } + if (blacklistedEverywhere) { + val partition = tasks(indexInTaskSet).partitionId + abort(s"Aborting ${taskSet} because task $indexInTaskSet (partition $partition) " + + s"cannot run anywhere due to node and executor blacklist. Blacklisting behavior " + + s"can be configured via spark.blacklist.*.") + } + } } } } @@ -702,7 +718,7 @@ private[spark] class TaskSetManager( } if (!successful(index)) { tasksSuccessful += 1 - logInfo("Finished task %s in stage %s (TID %d) in %d ms on %s / exec %s (%d/%d)".format( + logInfo("Finished task %s in stage %s (TID %d) in %d ms on %s (executor %s) (%d/%d)".format( info.id, taskSet.id, info.taskId, info.duration, info.host, info.executorId, tasksSuccessful, numTasks)) // Mark successful and stop if all the tasks have succeeded. @@ -721,7 +737,7 @@ private[spark] class TaskSetManager( * Marks the task as failed, re-adds it to the list of pending tasks, and notifies the * DAG Scheduler. */ - def handleFailedTask(tid: Long, state: TaskState, reason: TaskEndReason) { + def handleFailedTask(tid: Long, state: TaskState, reason: TaskFailedReason) { val info = taskInfos(tid) if (info.failed || info.killed) { return @@ -732,7 +748,7 @@ private[spark] class TaskSetManager( copiesRunning(index) -= 1 var accumUpdates: Seq[AccumulatorV2[_, _]] = Seq.empty val failureReason = s"Lost task ${info.id} in stage ${taskSet.id} (TID $tid, ${info.host}," + - s" exec ${info.executorId}): ${reason.asInstanceOf[TaskFailedReason].toErrorString}" + s" executor ${info.executorId}): ${reason.toErrorString}" val failureException: Option[Throwable] = reason match { case fetchFailed: FetchFailed => logWarning(failureReason) @@ -775,8 +791,8 @@ private[spark] class TaskSetManager( logWarning(failureReason) } else { logInfo( - s"Lost task ${info.id} in stage ${taskSet.id} (TID $tid) on executor ${info.host}: " + - s"${ef.className} (${ef.description}) [duplicate $dupCount]") + s"Lost task ${info.id} in stage ${taskSet.id} (TID $tid) on ${info.host}, executor" + + s" ${info.executorId}: ${ef.className} (${ef.description}) [duplicate $dupCount]") } ef.exception @@ -789,21 +805,13 @@ private[spark] class TaskSetManager( case e: TaskFailedReason => // TaskResultLost, TaskKilled, and others logWarning(failureReason) None - - case e: TaskEndReason => - logError("Unknown TaskEndReason: " + e) - None } // we might rack up a bunch of fetch-failures in rapid succession, due to a bad node. But // that bad node will get handled separately by spark's stage-failure handling mechanism. It // shouldn't penalize *this* executor at all, so don't count it as a task-failure as far as // the blacklist is concerned. - val countTowardsTaskFailures = reason match { - case fail: TaskFailedReason => fail.countTowardsTaskFailures - case Success => false - } - if (countTowardsTaskFailures && blacklistTracker.isDefined) { + if (reason.countTowardsTaskFailures && blacklistTracker.isDefined) { updateBlacklistForFailedTask(info.host, info.executorId, index) } @@ -818,7 +826,7 @@ private[spark] class TaskSetManager( addPendingTask(index) } - if (!isZombie && countTowardsTaskFailures) { + if (!isZombie && reason.countTowardsTaskFailures) { assert (null != failureReason) numFailures(index) += 1 if (numFailures(index) >= maxTaskFailures) { @@ -836,30 +844,34 @@ private[spark] class TaskSetManager( host: String, exec: String, index: Int): Unit = { - val failureStatus = execToFailures.getOrElseUpdate(exec, new FailureStatus(host)) - failureStatus.totalFailures += 1 - failureStatus.tasksWithFailures += index + val execFailures = execToFailures.getOrElseUpdate(exec, new ExecutorFailuresInTaskSet(host)) + execFailures.taskToFailureCount(index) = execFailures.taskToFailureCount.getOrElse(index, 0) + 1 - // check if this task has also failed on other executors on the same host, and if so, blacklist - // this task from the host - val execsWithFailuresOnNode = nodesToExecsWithFailures.getOrElseUpdate(host, new HashSet()) + // check if this task has also failed on other executors on the same host -- if its gone + // over the limit, blacklist it from the entire host + val execsWithFailuresOnNode = nodeToExecsWithFailures.getOrElseUpdate(host, new HashSet()) execsWithFailuresOnNode += exec - val failuresOnHost = (for { - exec <- execsWithFailuresOnNode.toIterator - failures <- execToFailures.get(exec) - } yield { - if (failures.tasksWithFailures.contains(index)) 1 else 0 - }).sum + val failuresOnHost = execsWithFailuresOnNode.toIterator.map { exec => + execToFailures.get(exec).map { failures => + // We count task attempts here, not the number of unique executors with failures. This is + // because jobs are aborted based on the number task attempts, and we want to make sure + // it's easy to setup the configs so that you can ensure that you always try another node + // before hitting the max number of task failures. + failures.taskToFailureCount.getOrElse(index, 0) + }.getOrElse(0) + }.sum if (failuresOnHost >= MAX_TASK_ATTEMPTS_PER_NODE) { - nodeBlacklistedTasks.getOrElseUpdate(host, new HashSet()) += index + nodeToBlacklistedTasks.getOrElseUpdate(host, new HashSet()) += index } - if (failureStatus.totalFailures >= MAX_FAILURES_PER_EXEC_STAGE) { + if (execFailures.numUniqueTasksWithFailures >= MAX_FAILURES_PER_EXEC_STAGE) { // This executor has been pushed into the blacklist for this stage. Let's check if it pushes // the whole node into the blacklist - val blacklistedExecutors = - execToFailures.filter(_._2.totalFailures >= MAX_FAILURES_PER_EXEC_STAGE) - if (blacklistedExecutors.size >= MAX_FAILED_EXEC_PER_NODE_STAGE) { + val blacklistedExecutorsOnNode = + execsWithFailuresOnNode. + flatMap(execToFailures.get(_)). + filter(_.numUniqueTasksWithFailures >= MAX_FAILURES_PER_EXEC_STAGE) + if (blacklistedExecutorsOnNode.size >= MAX_FAILED_EXEC_PER_NODE_STAGE) { if (blacklistedNodes.add(host)) { logInfo(s"Blacklisting ${host} for stage $stageId") } @@ -877,14 +889,14 @@ private[spark] class TaskSetManager( executorId: String, index: Int): Boolean = { execToFailures.get(executorId) - .map(_.tasksWithFailures.contains(index)) + .map(_.taskToFailureCount.getOrElse(index, 0) >= MAX_TASK_ATTEMPTS_PER_EXECUTOR) .getOrElse(false) } def isNodeBlacklistedForTask( node: String, index: Int): Boolean = { - nodeBlacklistedTasks.get(node) + nodeToBlacklistedTasks.get(node) .map(_.contains(index)) .getOrElse(false) } @@ -897,7 +909,7 @@ private[spark] class TaskSetManager( */ def isExecutorBlacklistedForTaskSet(executorId: String): Boolean = { execToFailures.get(executorId) - .map(_.totalFailures >= MAX_FAILURES_PER_EXEC_STAGE).getOrElse(false) + .map(_.numUniqueTasksWithFailures >= MAX_FAILURES_PER_EXEC_STAGE).getOrElse(false) } def isNodeBlacklistedForTaskSet(node: String): Boolean = { diff --git a/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala index c703e4f80846d..2ee3aa2fda8d4 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala @@ -95,8 +95,8 @@ class BlacklistIntegrationSuite extends SchedulerIntegrationSuite[MultiExecutorM // to reliably reproduce the failure, 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 -- we never try scheduling tasks on the good host-1, since we - // hit too many failures trying our preferred host-0. + // 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. val rdd = new MockRDDWithLocalityPrefs(sc, 1, Nil, badHost) withBackend(badHostBackend _) { val jobFuture = submit(rdd, (0 until 1).toArray) diff --git a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala index 5c71021388078..7e1074bcffcb2 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala @@ -22,7 +22,7 @@ import org.scalatest.BeforeAndAfterEach import org.scalatest.mock.MockitoSugar import org.apache.spark._ -import org.apache.spark.internal.config.{BLACKLIST_ENABLED, BLACKLIST_EXPIRY_TIMEOUT_CONF, BLACKLIST_LEGACY_TIMEOUT_CONF} +import org.apache.spark.internal.config import org.apache.spark.util.ManualClock class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with MockitoSugar @@ -65,7 +65,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M test("Blacklisting individual tasks") { val conf = new SparkConf().setAppName("test").setMaster("local") - .set(BLACKLIST_ENABLED.key, "true") + .set(config.BLACKLIST_ENABLED.key, "true") val scheduler = mockTaskSchedWithConf(conf) // Task 1 failed on executor 1 blacklistTracker = new BlacklistTracker(conf, clock) @@ -113,12 +113,12 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M // when the stage completes successfully, now there is sufficient evidence we've got // bad executors and node - blacklistTracker.taskSetSucceeded(tsm.execToFailures) + blacklistTracker.updateBlacklistForSuccessfulTaskSet(tsm.execToFailures) assert(blacklistTracker.nodeBlacklist() === Set("hostA")) assertEquivalentToSet(blacklistTracker.isNodeBlacklisted(_), Set("hostA")) assertEquivalentToSet(blacklistTracker.isExecutorBlacklisted(_), Set("1", "2")) - clock.advance(blacklistTracker.EXECUTOR_RECOVERY_MILLIS + 1) + clock.advance(blacklistTracker.BLACKLIST_TIMEOUT_MILLIS + 1) blacklistTracker.expireExecutorsInBlacklist() assert(blacklistTracker.nodeBlacklist() === Set()) assertEquivalentToSet(blacklistTracker.isNodeBlacklisted(_), Set()) @@ -126,8 +126,13 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M } def trackerFixture: (BlacklistTracker, TaskSchedulerImpl) = { - val conf = new SparkConf().setAppName("test").setMaster("local") - .set(BLACKLIST_ENABLED.key, "true") + trackerFixture() + } + + def trackerFixture(confs: (String, String)*): (BlacklistTracker, TaskSchedulerImpl) = { + val conf = new SparkConf().setAppName("test").setMaster("local") + .set(config.BLACKLIST_ENABLED.key, "true") + confs.foreach { case (k, v) => conf.set(k, v) } val scheduler = mockTaskSchedWithConf(conf) clock.setTime(0) @@ -144,7 +149,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M val taskSet = FakeTask.createTaskSet(1) val tsm = new TaskSetManager(scheduler, Some(tracker), taskSet, 4, clock) tsm.updateBlacklistForFailedTask("hostA", "1", 0) - tracker.taskSetSucceeded(tsm.execToFailures) + tracker.updateBlacklistForSuccessfulTaskSet(tsm.execToFailures) } assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set("1")) } @@ -178,7 +183,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M if (succeedTaskSet) { // the task set succeeded elsewhere, so we count those failures against our executor, // and blacklist it across stages - tracker.taskSetSucceeded(tsm.execToFailures) + tracker.updateBlacklistForSuccessfulTaskSet(tsm.execToFailures) assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set("1")) } else { // the task set failed, so we don't count these failures against the executor for other @@ -195,7 +200,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M (0 until 4).foreach { partition => tsm0.updateBlacklistForFailedTask("hostA", "1", partition) } - tracker.taskSetSucceeded(tsm0.execToFailures) + tracker.updateBlacklistForSuccessfulTaskSet(tsm0.execToFailures) assert(tracker.nodeBlacklist() === Set()) assertEquivalentToSet(tracker.isNodeBlacklisted(_), Set()) assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set("1")) @@ -205,12 +210,12 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M (0 until 4).foreach { partition => tsm1.updateBlacklistForFailedTask("hostA", "2", partition) } - tracker.taskSetSucceeded(tsm1.execToFailures) + tracker.updateBlacklistForSuccessfulTaskSet(tsm1.execToFailures) assert(tracker.nodeBlacklist() === Set("hostA")) assertEquivalentToSet(tracker.isNodeBlacklisted(_), Set("hostA")) assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set("1", "2")) - clock.advance(tracker.EXECUTOR_RECOVERY_MILLIS + 1) + clock.advance(tracker.BLACKLIST_TIMEOUT_MILLIS + 1) tracker.expireExecutorsInBlacklist() assert(tracker.nodeBlacklist() === Set()) assertEquivalentToSet(tracker.isNodeBlacklisted(_), Set()) @@ -220,7 +225,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M val taskSet2 = FakeTask.createTaskSet(4, 2, 0) val tsm2 = new TaskSetManager(scheduler, Some(tracker), taskSet2, 4, clock) tsm2.updateBlacklistForFailedTask("hostA", "1", 0) - tracker.taskSetSucceeded(tsm2.execToFailures) + tracker.updateBlacklistForSuccessfulTaskSet(tsm2.execToFailures) assert(tracker.nodeBlacklist() === Set()) assertEquivalentToSet(tracker.isNodeBlacklisted(_), Set()) assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set()) @@ -230,35 +235,31 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M // the blacklist should still work if an executor is killed completely. We should still // be able to blacklist the entire node. val (tracker, scheduler) = trackerFixture - when(scheduler.getExecutorsAliveOnHost("hostA")).thenReturn(Some(Set("1"))) val taskSet0 = FakeTask.createTaskSet(4) val tsm0 = new TaskSetManager(scheduler, Some(tracker), taskSet0, 4, clock) (0 until 3).foreach { partition => tsm0.updateBlacklistForFailedTask("hostA", "1", partition) } // now lets say that executor 1 dies completely - when(scheduler.getExecutorsAliveOnHost("hostA")).thenReturn(Some(Set[String]())) - when(scheduler.getHostForExecutor("1")).thenThrow(new NoSuchElementException("1")) // we get a task failure for the last task tsm0.updateBlacklistForFailedTask("hostA", "1", 3) - tracker.taskSetSucceeded(tsm0.execToFailures) + tracker.updateBlacklistForSuccessfulTaskSet(tsm0.execToFailures) assert(tracker.isExecutorBlacklisted("1")) - clock.advance(tracker.EXECUTOR_RECOVERY_MILLIS / 2) + clock.advance(tracker.BLACKLIST_TIMEOUT_MILLIS / 2) // say another executor gets spun up on that host - when(scheduler.getExecutorsAliveOnHost("hostA")).thenReturn(Some(Set("2"))) val taskSet1 = FakeTask.createTaskSet(4) val tsm1 = new TaskSetManager(scheduler, Some(tracker), taskSet1, 4, clock) (0 until 4).foreach { partition => tsm1.updateBlacklistForFailedTask("hostA", "2", partition) } - tracker.taskSetSucceeded(tsm1.execToFailures) + tracker.updateBlacklistForSuccessfulTaskSet(tsm1.execToFailures) // we've now had two bad executors on the hostA, so we should blacklist the entire node assert(tracker.isExecutorBlacklisted("1")) assert(tracker.isExecutorBlacklisted("2")) assert(tracker.isNodeBlacklisted("hostA")) - clock.advance(tracker.EXECUTOR_RECOVERY_MILLIS / 2 + 1) + clock.advance(tracker.BLACKLIST_TIMEOUT_MILLIS / 2 + 1) tracker.expireExecutorsInBlacklist() // executor 1 is no longer explicitly blacklisted, since we've gone past its recovery time, // but everything else is still blacklisted. @@ -269,15 +270,131 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M assert(!tracker.nodeToFailedExecs("hostA").contains("1")) } + test("task failures expire with time") { + val (tracker, scheduler) = trackerFixture + def failOneTaskInTaskSet(): Unit = { + val taskSet = FakeTask.createTaskSet(1) + val tsm = new TaskSetManager(scheduler, Some(tracker), taskSet, 1, clock) + tsm.updateBlacklistForFailedTask("hostA", "1", 0) + tracker.updateBlacklistForSuccessfulTaskSet(tsm.execToFailures) + } + failOneTaskInTaskSet() + + // now we advance the clock past the expiry time + clock.advance(tracker.BLACKLIST_TIMEOUT_MILLIS + 1) + tracker.expireExecutorsInBlacklist() + failOneTaskInTaskSet() + + // because we went past the expiry time, nothing should have been blacklisted + assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set()) + + // now we add one more failure, within the timeout, and it should be counted + clock.advance(tracker.BLACKLIST_TIMEOUT_MILLIS - 1) + failOneTaskInTaskSet() + assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set("1")) + + } + + 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 b/c 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 (tracker, scheduler) = trackerFixture( + config.MAX_TASK_ATTEMPTS_PER_EXECUTOR.key -> "2", + config.MAX_TASK_ATTEMPTS_PER_NODE.key -> "3", + config.MAX_FAILURES_PER_EXEC_STAGE.key -> "2", + config.MAX_FAILED_EXEC_PER_NODE_STAGE.key -> "3" + ) + val taskSet0 = FakeTask.createTaskSet(5) + val tsm0 = new TaskSetManager(scheduler, Some(tracker), taskSet0, 4, clock) + // fail a task twice on hostA, exec:1 + tsm0.updateBlacklistForFailedTask("hostA", "1", 0) + tsm0.updateBlacklistForFailedTask("hostA", "1", 0) + assert(tsm0.isExecutorBlacklistedForTask("1", 0)) + assert(!tsm0.isNodeBlacklistedForTask("hostA", 0)) + assert(!tsm0.isExecutorBlacklistedForTaskSet("1")) + assert(!tsm0.isNodeBlacklistedForTaskSet("hostA")) + + // fail the same task once more on hostA, exec:2 + tsm0.updateBlacklistForFailedTask("hostA", "2", 0) + assert(tsm0.isNodeBlacklistedForTask("hostA", 0)) + assert(!tsm0.isExecutorBlacklistedForTaskSet("2")) + assert(!tsm0.isNodeBlacklistedForTaskSet("hostA")) + + // fail another task on hostA, exec:1. Now that executor has failures on two different tasks, + // so its blacklisted + tsm0.updateBlacklistForFailedTask("hostA", "1", 1) + tracker.updateBlacklistForSuccessfulTaskSet(tsm0.execToFailures) + assert(tsm0.isExecutorBlacklistedForTaskSet("1")) + assert(!tsm0.isNodeBlacklistedForTaskSet("hostA")) + + // fail a third task on hostA, exec:2, so that exec is blacklisted for the whole task set + tsm0.updateBlacklistForFailedTask("hostA", "2", 2) + tracker.updateBlacklistForSuccessfulTaskSet(tsm0.execToFailures) + assert(tsm0.isExecutorBlacklistedForTaskSet("2")) + assert(!tsm0.isNodeBlacklistedForTaskSet("hostA")) + + // 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. + tsm0.updateBlacklistForFailedTask("hostA", "3", 3) + tsm0.updateBlacklistForFailedTask("hostA", "3", 4) + tracker.updateBlacklistForSuccessfulTaskSet(tsm0.execToFailures) + assert(tsm0.isExecutorBlacklistedForTaskSet("3")) + assert(tsm0.isNodeBlacklistedForTaskSet("hostA")) + } + + test("only blacklist nodes when all the blacklisted executors are all on same host (app level)") { + // we blacklist executors on two different hosts -- make sure that doesn't lead to any + // node blacklisting + val (tracker, scheduler) = trackerFixture + val taskSet0 = FakeTask.createTaskSet(4) + val tsm0 = new TaskSetManager(scheduler, Some(tracker), taskSet0, 1, clock) + tsm0.updateBlacklistForFailedTask("hostA", "1", 0) + tsm0.updateBlacklistForFailedTask("hostA", "1", 1) + tracker.updateBlacklistForSuccessfulTaskSet(tsm0.execToFailures) + assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set("1")) + assertEquivalentToSet(tracker.isNodeBlacklisted(_), Set()) + + val taskSet1 = FakeTask.createTaskSet(4) + val tsm1 = new TaskSetManager(scheduler, Some(tracker), taskSet1, 1, clock) + tsm1.updateBlacklistForFailedTask("hostB", "2", 0) + tsm1.updateBlacklistForFailedTask("hostB", "2", 1) + tracker.updateBlacklistForSuccessfulTaskSet(tsm1.execToFailures) + assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set("1", "2")) + assertEquivalentToSet(tracker.isNodeBlacklisted(_), Set()) + } + + test("only blacklist nodes when all the blacklisted executors are all on same host (tsm level)") { + // we blacklist executors on two different hosts within one taskSet -- make sure that doesn't + // lead to any node blacklisting + val (tracker, scheduler) = trackerFixture + val taskSet = FakeTask.createTaskSet(4) + val tsm = new TaskSetManager(scheduler, Some(tracker), taskSet, 1, clock) + tsm.updateBlacklistForFailedTask("hostA", "1", 0) + tsm.updateBlacklistForFailedTask("hostA", "1", 1) + assertEquivalentToSet(tsm.isExecutorBlacklistedForTaskSet(_), Set("1")) + assertEquivalentToSet(tsm.isNodeBlacklistedForTaskSet(_), Set()) + + tsm.updateBlacklistForFailedTask("hostB", "2", 0) + tsm.updateBlacklistForFailedTask("hostB", "2", 1) + assertEquivalentToSet(tsm.isExecutorBlacklistedForTaskSet(_), Set("1", "2")) + assertEquivalentToSet(tsm.isNodeBlacklistedForTaskSet(_), Set()) + } + + test("blacklist still respects legacy configs") { - val legacyKey = BLACKLIST_LEGACY_TIMEOUT_CONF.key + val legacyKey = config.BLACKLIST_LEGACY_TIMEOUT_CONF.key { val localConf = new SparkConf().setMaster("local") assert(!BlacklistTracker.isBlacklistEnabled(localConf)) localConf.set(legacyKey, "5000") assert(BlacklistTracker.isBlacklistEnabled(localConf)) - assert(5000 === BlacklistTracker.getBlacklistExpiryTime(localConf)) + assert(5000 === BlacklistTracker.getBlacklistTimeout(localConf)) localConf.set(legacyKey, "0") assert(!BlacklistTracker.isBlacklistEnabled(localConf)) @@ -286,11 +403,11 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M { val distConf = new SparkConf().setMaster("yarn-cluster") assert(BlacklistTracker.isBlacklistEnabled(distConf)) - assert(60 * 60 * 1000L === BlacklistTracker.getBlacklistExpiryTime(distConf)) + assert(60 * 60 * 1000L === BlacklistTracker.getBlacklistTimeout(distConf)) distConf.set(legacyKey, "5000") - assert(5000 === BlacklistTracker.getBlacklistExpiryTime(distConf)) - distConf.set(BLACKLIST_EXPIRY_TIMEOUT_CONF.key, "10h") - assert(10 * 60 * 60 * 1000L == BlacklistTracker.getBlacklistExpiryTime(distConf)) + assert(5000 === BlacklistTracker.getBlacklistTimeout(distConf)) + distConf.set(config.BLACKLIST_TIMEOUT_CONF.key, "10h") + assert(10 * 60 * 60 * 1000L == BlacklistTracker.getBlacklistTimeout(distConf)) } } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala index 85e09a0da361f..60cabc14c4d1d 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala @@ -511,9 +511,9 @@ class TestTaskScheduler(sc: SparkContext) extends TaskSchedulerImpl(sc) { super.submitTasks(taskSet) } - override def taskSetFinished(manager: TaskSetManager, success: Boolean): Unit = { + override def taskSetFinished(manager: TaskSetManager): Unit = { runningTaskSets -= manager.taskSet - super.taskSetFinished(manager, success) + super.taskSetFinished(manager) } } 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 32a069933dce0..8e39c456f751a 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -97,8 +97,8 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B val tsmSpy = spy(tsm) stageToMockTsm(taskSet.stageId) = tsmSpy // intentionally bogus, just lets us easily verify - val execToFailures = new HashMap[String, FailureStatus]() - execToFailures(taskSet.stageId.toString) = new FailureStatus("dummy") + val execToFailures = new HashMap[String, ExecutorFailuresInTaskSet]() + execToFailures(taskSet.stageId.toString) = new ExecutorFailuresInTaskSet("dummy") when(tsmSpy.execToFailures).thenReturn(execToFailures) tsmSpy } @@ -280,7 +280,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B // attempt 1 finished (this can happen even if it was marked zombie earlier -- all tasks were // already submitted, and then they finish) - taskScheduler.taskSetFinished(mgr1, true) + taskScheduler.taskSetFinished(mgr1) // now with another resource offer, we should still schedule all the tasks in attempt2 val taskDescriptions3 = taskScheduler.resourceOffers(workerOffers).flatten @@ -350,20 +350,22 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B when(tsm.isNodeBlacklistedForTaskSet(anyString())).thenReturn(false) when(tsm.isExecutorBlacklistedForTaskSet(anyString())).thenReturn(false) when(tsm.isExecutorBlacklistedForTask(anyString(), anyInt())).thenReturn(false) + when(tsm.isNodeBlacklistedForTask(anyString(), anyInt())).thenReturn(false) } when(stageToMockTsm(0).isNodeBlacklistedForTaskSet("host1")).thenReturn(true) when(stageToMockTsm(1).isExecutorBlacklistedForTaskSet("executor3")).thenReturn(true) when(stageToMockTsm(0).isExecutorBlacklistedForTask("executor0", 0)).thenReturn(true) val firstTaskAttempts = taskScheduler.resourceOffers(offers).flatten - // these verifications are tricky b/c we reference them multiple times -- also invoked when we - // check if we need to abort any stages from unschedulability. + // these verifications are tricky b/c (a) we reference them multiple times -- also invoked when + // we check if we need to abort any stages from unschedulability and (b) resources + // are offered to the taskSets until they have no more free cores, so we don't necessarily + // even make an offer for each resource to each taskSet. ('0' until '2').foreach { hostNum => - val host = s"host$hostNum" - verify(blacklist, atLeast(1)).isNodeBlacklisted(host) - (0 to 2).foreach { stageId => - verify(stageToMockTsm(stageId), atLeast(1)).isNodeBlacklistedForTaskSet(host) - } + verify(blacklist, atLeast(1)).isNodeBlacklisted(s"host$hostNum") + } + (0 to 2).foreach { stageId => + verify(stageToMockTsm(stageId), atLeast(1)).isNodeBlacklistedForTaskSet(anyString()) } for { exec <- Seq("executor1", "executor2") @@ -436,8 +438,10 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B } // the tasksSets complete, so the tracker should be notified - verify(blacklist, times(1)).taskSetSucceeded(stageToMockTsm(0).execToFailures) - verify(blacklist, times(1)).taskSetSucceeded(stageToMockTsm(1).execToFailures) + verify(blacklist, times(1)).updateBlacklistForSuccessfulTaskSet( + stageToMockTsm(0).execToFailures) + verify(blacklist, times(1)).updateBlacklistForSuccessfulTaskSet( + stageToMockTsm(1).execToFailures) } test("scheduled tasks obey node and executor blacklists") { @@ -474,11 +478,6 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B ('0' until '2').foreach { hostNum => verify(blacklist, atLeast(1)).isNodeBlacklisted("host" + hostNum) } - stageToMockTsm.values.foreach { tsm => - verify(tsm, never()).isNodeBlacklistedForTaskSet(anyString()) - verify(tsm, never()).isExecutorBlacklistedForTaskSet(anyString()) - verify(tsm, never()).isExecutorBlacklistedForTask(anyString(), anyInt()) - } // we should have aborted the existing stages, since they aren't schedulable (0 to 2).foreach { stageId => @@ -489,7 +488,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B test("abort stage when all executors are blacklisted") { val blacklist = mock[BlacklistTracker] taskScheduler = setupSchedulerWithMockTsm(blacklist) - val taskSet = FakeTask.createTaskSet(numTasks = 2, stageId = 0, stageAttemptId = 0) + val taskSet = FakeTask.createTaskSet(numTasks = 10, stageId = 0, stageAttemptId = 0) taskScheduler.submitTasks(taskSet) val tsm = stageToMockTsm(0) @@ -572,7 +571,8 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B assert(failedTaskSet) val idx = failedTask.index assert(failedTaskSetReason === s"Aborting TaskSet 0.0 because task $idx (partition $idx) " + - s"cannot run anywhere due to node and executor blacklist.") + s"cannot run anywhere due to node and executor blacklist. Blacklisting behavior can be " + + s"configured via spark.blacklist.*.") } test("don't abort if there is an executor available, though it hasn't had scheduled tasks yet") { 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 57435cded6d95..1989be142685d 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -104,7 +104,7 @@ class FakeTaskScheduler(sc: SparkContext, liveExecutors: (String, String)* /* ex val host = executorIdToHost.get(execId) assert(host != None) val hostId = host.get - val executorsOnHost = executorsByHost(hostId) + val executorsOnHost = hostToExecutors(hostId) executorsOnHost -= execId for (rack <- getRackForHost(hostId); hosts <- hostsByRack.get(rack)) { hosts -= hostId @@ -114,7 +114,7 @@ class FakeTaskScheduler(sc: SparkContext, liveExecutors: (String, String)* /* ex } } - override def taskSetFinished(manager: TaskSetManager, success: Boolean): Unit = { + override def taskSetFinished(manager: TaskSetManager): Unit = { finishedManagers += manager } @@ -128,7 +128,7 @@ class FakeTaskScheduler(sc: SparkContext, liveExecutors: (String, String)* /* ex def addExecutor(execId: String, host: String) { executors.put(execId, host) - val executorsOnHost = executorsByHost.getOrElseUpdate(host, new mutable.HashSet[String]) + val executorsOnHost = hostToExecutors.getOrElseUpdate(host, new mutable.HashSet[String]) executorsOnHost += execId executorIdToHost += execId -> host for (rack <- getRackForHost(host)) { @@ -415,7 +415,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val rescheduleDelay = 300L val conf = new SparkConf(). set(config.BLACKLIST_ENABLED.key, "true"). - set(config.BLACKLIST_EXPIRY_TIMEOUT_CONF.key, rescheduleDelay.toString). + set(config.BLACKLIST_TIMEOUT_CONF.key, rescheduleDelay.toString). // don't wait to jump locality levels in this test set("spark.locality.wait", "0") diff --git a/docs/configuration.md b/docs/configuration.md index 5f8a7cb8450f1..fffcdc7023805 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1161,51 +1161,65 @@ Apart from these, the following properties are also available, and may be useful - spark.blacklist.recoveryTime + spark.blacklist.timeout 1h - 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 blacklisted for the entire application, before it + is unconditionally removed from the blacklist to attempt running new tasks. - spark.blacklist.maxTaskAttemptsPerNode + spark.blacklist.task.maxTaskAttemptsPerExecutor 2 - For a given task, how many executors the task can fail on, before the entire node is blacklisted - for the given task + (Experimental) For a given task, how many times it can be retried on one executor before the + executor is blacklisted for that task. - spark.blacklist.maxFailedTasksPerExecutorStage + spark.blacklist.task.maxTaskAttemptsPerNode 2 - How many different tasks must fail on one executor, within one stage, before the executor is - blacklisted for that stage. + (Experimental) For a given task, how many times it can be retried on one node, before the entire + node is blacklisted for that task. - spark.blacklist.maxFailedExecutorsPerNodeStage + spark.blacklist.stage.maxFailedTasksPerExecutor 2 - How many different executors are marked as failed for a given stage, before the entire node - is marked as failed for the stage. + (Experimental) How many different tasks must fail on one executor, within one stage, before the + executor is blacklisted for that stage. - spark.blacklist.maxFailedTasksPerExecutor + spark.blacklist.stage.maxFailedExecutorsPerNode 2 - How many different tasks must fail on one executor, in successful task sets, before the executor - is blacklisted for the entire application. + (Experimental) How many different executors are marked as failed for a given stage, before the + entire node is marked as failed for the stage. - spark.blacklist.maxFailedExecutorsPerNode + spark.blacklist.application.maxFailedTasksPerExecutor 2 - How many different executors must be blacklisted for the entire application, before the node is - blacklisted for the entire application. + (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 + 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 + may get marked as idle and be reclaimed by the cluster manager. + + + + spark.blacklist.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 + 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 + may get marked as idle and be reclaimed by the cluster manager. diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala index 0575c74c6c835..c795c1bcc26cc 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -219,8 +219,9 @@ private[yarn] class YarnAllocator( * @param localityAwareTasks number of locality aware tasks to be used as container placement hint * @param hostToLocalTaskCount a map of preferred hostname to possible task counts to be used as * container placement hint. - * @param nodeBlacklist a set of blacklisted node to avoid allocating new container on them. It - * will be used to update AM blacklist. + * @param nodeBlacklist a set of blacklisted nodes, which is passed in to avoid allocating new + * containers on them. It will be used to update the application master's + * blacklist. * @return Whether the new requested total is different than the old value. */ def requestTotalExecutorsWithPreferredLocalities( @@ -236,7 +237,7 @@ private[yarn] class YarnAllocator( targetNumExecutors = requestedTotal // Update blacklist infomation to YARN ResouceManager for this application, - // in order to avoid allocating new Container on the problematic nodes. + // in order to avoid allocating new Containers on the problematic nodes. val blacklistAdditions = nodeBlacklist -- currentNodeBlacklist val blacklistRemovals = currentNodeBlacklist -- nodeBlacklist if (blacklistAdditions.nonEmpty) { From 515b18a6d15b1385caf6e79a849e3397e37e8602 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 18 Aug 2016 10:16:36 -0500 Subject: [PATCH 22/57] add task timeouts --- .../spark/scheduler/BlacklistTracker.scala | 119 +++++++++++++++--- .../spark/scheduler/TaskSchedulerImpl.scala | 2 +- .../spark/scheduler/TaskSetManager.scala | 19 +-- .../scheduler/BlacklistTrackerSuite.scala | 108 ++++++++-------- .../scheduler/TaskSchedulerImplSuite.scala | 6 +- .../spark/scheduler/TaskSetManagerSuite.scala | 2 +- 6 files changed, 175 insertions(+), 81 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala index 6a79a89e690f0..68c3773e13041 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala @@ -19,7 +19,7 @@ package org.apache.spark.scheduler import java.util.concurrent.atomic.AtomicReference -import scala.collection.mutable.{HashMap, HashSet} +import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import org.apache.spark.SparkConf import org.apache.spark.internal.Logging @@ -33,12 +33,15 @@ import org.apache.spark.util.{Clock, SystemClock, Utils} * concert with the blacklisting here. * * The tracker needs to deal with a variety of workloads, eg.: - * * bad user code, which may lead to many task failures, but that should not count against - * individual executors - * * many small stages, which may prevent a bad executor for having many failures within one stage, - * but still many failures over the entire application - * * "flaky" executors, that don't fail every task, but are still faulty - * * etc. + * + * * 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 @@ -56,9 +59,9 @@ private[scheduler] class BlacklistTracker ( * 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. */ - private val executorIdToFailureCount: HashMap[String, Int] = new HashMap() - private val executorIdToBlacklistStatus: HashMap[String, BlacklistedExecutor] = new HashMap() - private val nodeIdToBlacklistExpiryTime: HashMap[String, Long] = new HashMap() + private val executorIdToFailureList: HashMap[String, ExecutorFailureList] = new HashMap() + val executorIdToBlacklistStatus: HashMap[String, BlacklistedExecutor] = new HashMap() + val nodeIdToBlacklistExpiryTime: HashMap[String, Long] = new HashMap() /** * An immutable copy of the set of nodes that are currently blacklisted. Kept in an * AtomicReference to make [[nodeBlacklist()]] thread-safe. @@ -78,10 +81,26 @@ private[scheduler] class BlacklistTracker ( */ val nodeToFailedExecs: HashMap[String, HashSet[String]] = new HashMap() - def expireExecutorsInBlacklist(): Unit = { + 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 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. + executorIdToFailureList.values.foreach { executorFailureList => + executorFailureList.dropFailuresWithTimeoutBefore(now) + } + + // 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. @@ -113,13 +132,19 @@ private[scheduler] class BlacklistTracker ( } 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. failuresByExec.foreach { case (exec, failuresInTaskSet) => - val prevFailures = executorIdToFailureCount.getOrElse(exec, 0) - val newTotal = prevFailures + failuresInTaskSet.numUniqueTasksWithFailures - executorIdToFailureCount.put(exec, newTotal) + val allExecutorFailures = + executorIdToFailureList.getOrElseUpdate(exec, new ExecutorFailureList) + allExecutorFailures.addFailures(stageId, stageAttemptId, failuresInTaskSet) + val newTotal = allExecutorFailures.numUniqueTaskFailures + if (allExecutorFailures.minExpiryTime < nextExpiryTime) { + nextExpiryTime = allExecutorFailures.minExpiryTime + } if (newTotal >= MAX_FAILURES_PER_EXEC) { logInfo(s"Blacklisting executor id: $exec because it has $newTotal" + @@ -128,7 +153,7 @@ private[scheduler] class BlacklistTracker ( val expiryTime = now + BLACKLIST_TIMEOUT_MILLIS val node = failuresInTaskSet.node executorIdToBlacklistStatus.put(exec, BlacklistedExecutor(node, expiryTime)) - executorIdToFailureCount.remove(exec) + executorIdToFailureList.remove(exec) if (expiryTime < nextExpiryTime) { nextExpiryTime = expiryTime } @@ -170,7 +195,7 @@ private[scheduler] class BlacklistTracker ( // 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. - executorIdToFailureCount -= executorId + executorIdToFailureList -= executorId } } @@ -232,12 +257,68 @@ private[scheduler] final class ExecutorFailuresInTaskSet(val node: String) { * Mapping from index of the tasks in the taskset, to the number of times it has failed on this * executor. */ - val taskToFailureCount = HashMap[Int, Int]() - def numUniqueTasksWithFailures: Int = taskToFailureCount.size + val taskToFailureCountAndExpiryTime = HashMap[Int, (Int, Long)]() + def updateWithFailure(taskIndex: Int, failureExpiryTime: Long): Unit = { + val (prevFailureCount, prevFailureExpiryTime) = + taskToFailureCountAndExpiryTime.getOrElse(taskIndex, (0, -1L)) + assert(failureExpiryTime >= prevFailureExpiryTime) + taskToFailureCountAndExpiryTime(taskIndex) = (prevFailureCount + 1, failureExpiryTime) + } + def numUniqueTasksWithFailures: Int = taskToFailureCountAndExpiryTime.size + override def toString(): String = { s"numUniqueTasksWithFailures= $numUniqueTasksWithFailures; " + - s"tasksToFailureCount = $taskToFailureCount" + s"tasksToFailureCount = $taskToFailureCountAndExpiryTime" + } +} + +/** + * Tracks all failures for one executor (that have not passed the timeout). Designed to efficiently + * remove failures that are older than the timeout, and query for the number of unique failed tasks. + */ +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, sorted by time ascending. + */ + private var failures = ArrayBuffer[(TaskId, Long)]() + + def addFailures( + stage: Int, + stageAttempt: Int, + failuresInTaskSet: ExecutorFailuresInTaskSet): Unit = { + // The new failures may interleave with the old ones, so rebuild the failures in sorted order. + // This shouldn't be expensive because if there were a lot of failures, the executor would + // have been blacklisted. + if (failuresInTaskSet.taskToFailureCountAndExpiryTime.nonEmpty) { + failuresInTaskSet.taskToFailureCountAndExpiryTime.foreach { case (taskIdx, (_, time)) => + failures += ((TaskId(stage, stageAttempt, taskIdx), time)) + } + // sort by failure time, so we can quickly determine if any failure has gone past the timeout + failures = failures.sortBy(_._2) + } + } + + def minExpiryTime: Long = failures.head._2 + + /** + * The number of unique tasks that failed on this executor. Only counts failures within the + * timeout, and in successful tasksets. + */ + def numUniqueTaskFailures: Int = failures.size + + def dropFailuresWithTimeoutBefore(dropBefore: Long): Unit = { + if (minExpiryTime < dropBefore) { + val minIndexToKeep = failures.indexWhere(_._2 >= dropBefore) + if (minIndexToKeep == -1) { + failures.clear() + } else { + failures = failures.drop(minIndexToKeep) + } + } } } 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 07e143069c866..e161d83815395 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -324,7 +324,7 @@ private[spark] class TaskSchedulerImpl private[scheduler]( // Before making any offers, remove any nodes from the blacklist whose blacklist 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. - blacklistTracker.foreach(_.expireExecutorsInBlacklist()) + blacklistTracker.foreach(_.applyBlacklistTimeout()) val sortedTaskSets = rootPool.getSortedTaskSetQueue val filteredOffers = blacklistTracker.map { bl => 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 72e4988d81284..c54707eabcf56 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -507,7 +507,8 @@ private[spark] class TaskSetManager( if (isZombie && runningTasks == 0) { sched.taskSetFinished(this) if (tasksSuccessful == numTasks) { - blacklistTracker.foreach(_.updateBlacklistForSuccessfulTaskSet(execToFailures)) + blacklistTracker.foreach(_.updateBlacklistForSuccessfulTaskSet(taskSet.stageId, + taskSet.stageAttemptId, execToFailures)) } } } @@ -845,7 +846,8 @@ private[spark] class TaskSetManager( exec: String, index: Int): Unit = { val execFailures = execToFailures.getOrElseUpdate(exec, new ExecutorFailuresInTaskSet(host)) - execFailures.taskToFailureCount(index) = execFailures.taskToFailureCount.getOrElse(index, 0) + 1 + execFailures.updateWithFailure(index, clock.getTimeMillis() + + blacklistTracker.get.BLACKLIST_TIMEOUT_MILLIS) // check if this task has also failed on other executors on the same host -- if its gone // over the limit, blacklist it from the entire host @@ -854,10 +856,10 @@ private[spark] class TaskSetManager( val failuresOnHost = execsWithFailuresOnNode.toIterator.map { exec => execToFailures.get(exec).map { failures => // We count task attempts here, not the number of unique executors with failures. This is - // because jobs are aborted based on the number task attempts, and we want to make sure - // it's easy to setup the configs so that you can ensure that you always try another node - // before hitting the max number of task failures. - failures.taskToFailureCount.getOrElse(index, 0) + // because jobs are aborted based on the number task attempts; if we counted unique + // executors, it would be hard to config to ensure that you try another + // node before hitting the max number of task failures. + failures.taskToFailureCountAndExpiryTime.getOrElse(index, (0, 0))._1 }.getOrElse(0) }.sum if (failuresOnHost >= MAX_TASK_ATTEMPTS_PER_NODE) { @@ -889,7 +891,10 @@ private[spark] class TaskSetManager( executorId: String, index: Int): Boolean = { execToFailures.get(executorId) - .map(_.taskToFailureCount.getOrElse(index, 0) >= MAX_TASK_ATTEMPTS_PER_EXECUTOR) + .map { execFailures => + val count = execFailures.taskToFailureCountAndExpiryTime.getOrElse(index, (0, 0))._1 + count >= MAX_TASK_ATTEMPTS_PER_EXECUTOR + } .getOrElse(false) } diff --git a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala index 7e1074bcffcb2..b382533e92f43 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala @@ -113,13 +113,13 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M // when the stage completes successfully, now there is sufficient evidence we've got // bad executors and node - blacklistTracker.updateBlacklistForSuccessfulTaskSet(tsm.execToFailures) + blacklistTracker.updateBlacklistForSuccessfulTaskSet(0, 0, tsm.execToFailures) assert(blacklistTracker.nodeBlacklist() === Set("hostA")) assertEquivalentToSet(blacklistTracker.isNodeBlacklisted(_), Set("hostA")) assertEquivalentToSet(blacklistTracker.isExecutorBlacklisted(_), Set("1", "2")) clock.advance(blacklistTracker.BLACKLIST_TIMEOUT_MILLIS + 1) - blacklistTracker.expireExecutorsInBlacklist() + blacklistTracker.applyBlacklistTimeout() assert(blacklistTracker.nodeBlacklist() === Set()) assertEquivalentToSet(blacklistTracker.isNodeBlacklisted(_), Set()) assertEquivalentToSet(blacklistTracker.isExecutorBlacklisted(_), Set()) @@ -149,7 +149,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M val taskSet = FakeTask.createTaskSet(1) val tsm = new TaskSetManager(scheduler, Some(tracker), taskSet, 4, clock) tsm.updateBlacklistForFailedTask("hostA", "1", 0) - tracker.updateBlacklistForSuccessfulTaskSet(tsm.execToFailures) + tracker.updateBlacklistForSuccessfulTaskSet(stage, 0, tsm.execToFailures) } assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set("1")) } @@ -183,7 +183,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M if (succeedTaskSet) { // the task set succeeded elsewhere, so we count those failures against our executor, // and blacklist it across stages - tracker.updateBlacklistForSuccessfulTaskSet(tsm.execToFailures) + tracker.updateBlacklistForSuccessfulTaskSet(stageId, 0, tsm.execToFailures) assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set("1")) } else { // the task set failed, so we don't count these failures against the executor for other @@ -200,7 +200,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M (0 until 4).foreach { partition => tsm0.updateBlacklistForFailedTask("hostA", "1", partition) } - tracker.updateBlacklistForSuccessfulTaskSet(tsm0.execToFailures) + tracker.updateBlacklistForSuccessfulTaskSet(0, 0, tsm0.execToFailures) assert(tracker.nodeBlacklist() === Set()) assertEquivalentToSet(tracker.isNodeBlacklisted(_), Set()) assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set("1")) @@ -210,13 +210,13 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M (0 until 4).foreach { partition => tsm1.updateBlacklistForFailedTask("hostA", "2", partition) } - tracker.updateBlacklistForSuccessfulTaskSet(tsm1.execToFailures) + tracker.updateBlacklistForSuccessfulTaskSet(0, 0, tsm1.execToFailures) assert(tracker.nodeBlacklist() === Set("hostA")) assertEquivalentToSet(tracker.isNodeBlacklisted(_), Set("hostA")) assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set("1", "2")) clock.advance(tracker.BLACKLIST_TIMEOUT_MILLIS + 1) - tracker.expireExecutorsInBlacklist() + tracker.applyBlacklistTimeout() assert(tracker.nodeBlacklist() === Set()) assertEquivalentToSet(tracker.isNodeBlacklisted(_), Set()) assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set()) @@ -225,64 +225,72 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M val taskSet2 = FakeTask.createTaskSet(4, 2, 0) val tsm2 = new TaskSetManager(scheduler, Some(tracker), taskSet2, 4, clock) tsm2.updateBlacklistForFailedTask("hostA", "1", 0) - tracker.updateBlacklistForSuccessfulTaskSet(tsm2.execToFailures) + tracker.updateBlacklistForSuccessfulTaskSet(2, 0, tsm2.execToFailures) assert(tracker.nodeBlacklist() === Set()) assertEquivalentToSet(tracker.isNodeBlacklisted(_), Set()) assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set()) } test("blacklist can handle lost executors") { - // the blacklist should still work if an executor is killed completely. We should still + // The blacklist should still work if an executor is killed completely. We should still // be able to blacklist the entire node. val (tracker, scheduler) = trackerFixture val taskSet0 = FakeTask.createTaskSet(4) val tsm0 = new TaskSetManager(scheduler, Some(tracker), taskSet0, 4, clock) - (0 until 3).foreach { partition => + // Lets say that executor 1 dies completely. We get a task failure for the last task, but + // the taskset then finishes successfully (elsewhere). + (0 until 4).foreach { partition => tsm0.updateBlacklistForFailedTask("hostA", "1", partition) } - // now lets say that executor 1 dies completely - // we get a task failure for the last task - tsm0.updateBlacklistForFailedTask("hostA", "1", 3) - tracker.updateBlacklistForSuccessfulTaskSet(tsm0.execToFailures) + tracker.handleRemovedExecutor("1") + tracker.updateBlacklistForSuccessfulTaskSet(0, 0, tsm0.execToFailures) assert(tracker.isExecutorBlacklisted("1")) clock.advance(tracker.BLACKLIST_TIMEOUT_MILLIS / 2) - // say another executor gets spun up on that host - val taskSet1 = FakeTask.createTaskSet(4) + // Now another executor gets spun up on that host, but it also dies. + val taskSet1 = FakeTask.createTaskSet(4, 1, 0) val tsm1 = new TaskSetManager(scheduler, Some(tracker), taskSet1, 4, clock) (0 until 4).foreach { partition => tsm1.updateBlacklistForFailedTask("hostA", "2", partition) } - tracker.updateBlacklistForSuccessfulTaskSet(tsm1.execToFailures) - // we've now had two bad executors on the hostA, so we should blacklist the entire node + tracker.handleRemovedExecutor("2") + tracker.updateBlacklistForSuccessfulTaskSet(1, 0, tsm1.execToFailures) + // We've now had two bad executors on the hostA, so we should blacklist the entire node. assert(tracker.isExecutorBlacklisted("1")) assert(tracker.isExecutorBlacklisted("2")) assert(tracker.isNodeBlacklisted("hostA")) clock.advance(tracker.BLACKLIST_TIMEOUT_MILLIS / 2 + 1) - tracker.expireExecutorsInBlacklist() + tracker.applyBlacklistTimeout() // executor 1 is no longer explicitly blacklisted, since we've gone past its recovery time, // but everything else is still blacklisted. assert(!tracker.isExecutorBlacklisted("1")) assert(tracker.isExecutorBlacklisted("2")) assert(tracker.isNodeBlacklisted("hostA")) // make sure we don't leak memory + assert(!tracker.executorIdToBlacklistStatus.contains("1")) assert(!tracker.nodeToFailedExecs("hostA").contains("1")) + clock.advance(tracker.BLACKLIST_TIMEOUT_MILLIS) + tracker.applyBlacklistTimeout() + assert(!tracker.nodeIdToBlacklistExpiryTime.contains("hostA")) } test("task failures expire with time") { val (tracker, scheduler) = trackerFixture + var stageId = 0 def failOneTaskInTaskSet(): Unit = { - val taskSet = FakeTask.createTaskSet(1) + val taskSet = FakeTask.createTaskSet(1, stageId, 0) val tsm = new TaskSetManager(scheduler, Some(tracker), taskSet, 1, clock) tsm.updateBlacklistForFailedTask("hostA", "1", 0) - tracker.updateBlacklistForSuccessfulTaskSet(tsm.execToFailures) + tracker.updateBlacklistForSuccessfulTaskSet(stageId, 0, tsm.execToFailures) + stageId += 1 } failOneTaskInTaskSet() + assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set()) // now we advance the clock past the expiry time clock.advance(tracker.BLACKLIST_TIMEOUT_MILLIS + 1) - tracker.expireExecutorsInBlacklist() + tracker.applyBlacklistTimeout() failOneTaskInTaskSet() // because we went past the expiry time, nothing should have been blacklisted @@ -309,42 +317,42 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M config.MAX_FAILURES_PER_EXEC_STAGE.key -> "2", config.MAX_FAILED_EXEC_PER_NODE_STAGE.key -> "3" ) - val taskSet0 = FakeTask.createTaskSet(5) - val tsm0 = new TaskSetManager(scheduler, Some(tracker), taskSet0, 4, clock) + val taskSet = FakeTask.createTaskSet(5) + val tsm = new TaskSetManager(scheduler, Some(tracker), taskSet, 4, clock) // fail a task twice on hostA, exec:1 - tsm0.updateBlacklistForFailedTask("hostA", "1", 0) - tsm0.updateBlacklistForFailedTask("hostA", "1", 0) - assert(tsm0.isExecutorBlacklistedForTask("1", 0)) - assert(!tsm0.isNodeBlacklistedForTask("hostA", 0)) - assert(!tsm0.isExecutorBlacklistedForTaskSet("1")) - assert(!tsm0.isNodeBlacklistedForTaskSet("hostA")) + tsm.updateBlacklistForFailedTask("hostA", "1", 0) + tsm.updateBlacklistForFailedTask("hostA", "1", 0) + assert(tsm.isExecutorBlacklistedForTask("1", 0)) + assert(!tsm.isNodeBlacklistedForTask("hostA", 0)) + assert(!tsm.isExecutorBlacklistedForTaskSet("1")) + assert(!tsm.isNodeBlacklistedForTaskSet("hostA")) // fail the same task once more on hostA, exec:2 - tsm0.updateBlacklistForFailedTask("hostA", "2", 0) - assert(tsm0.isNodeBlacklistedForTask("hostA", 0)) - assert(!tsm0.isExecutorBlacklistedForTaskSet("2")) - assert(!tsm0.isNodeBlacklistedForTaskSet("hostA")) + tsm.updateBlacklistForFailedTask("hostA", "2", 0) + assert(tsm.isNodeBlacklistedForTask("hostA", 0)) + assert(!tsm.isExecutorBlacklistedForTaskSet("2")) + assert(!tsm.isNodeBlacklistedForTaskSet("hostA")) // fail another task on hostA, exec:1. Now that executor has failures on two different tasks, // so its blacklisted - tsm0.updateBlacklistForFailedTask("hostA", "1", 1) - tracker.updateBlacklistForSuccessfulTaskSet(tsm0.execToFailures) - assert(tsm0.isExecutorBlacklistedForTaskSet("1")) - assert(!tsm0.isNodeBlacklistedForTaskSet("hostA")) + tsm.updateBlacklistForFailedTask("hostA", "1", 1) + tracker.updateBlacklistForSuccessfulTaskSet(0, 0, tsm.execToFailures) + assert(tsm.isExecutorBlacklistedForTaskSet("1")) + assert(!tsm.isNodeBlacklistedForTaskSet("hostA")) // fail a third task on hostA, exec:2, so that exec is blacklisted for the whole task set - tsm0.updateBlacklistForFailedTask("hostA", "2", 2) - tracker.updateBlacklistForSuccessfulTaskSet(tsm0.execToFailures) - assert(tsm0.isExecutorBlacklistedForTaskSet("2")) - assert(!tsm0.isNodeBlacklistedForTaskSet("hostA")) + tsm.updateBlacklistForFailedTask("hostA", "2", 2) + tracker.updateBlacklistForSuccessfulTaskSet(0, 0, tsm.execToFailures) + assert(tsm.isExecutorBlacklistedForTaskSet("2")) + assert(!tsm.isNodeBlacklistedForTaskSet("hostA")) // 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. - tsm0.updateBlacklistForFailedTask("hostA", "3", 3) - tsm0.updateBlacklistForFailedTask("hostA", "3", 4) - tracker.updateBlacklistForSuccessfulTaskSet(tsm0.execToFailures) - assert(tsm0.isExecutorBlacklistedForTaskSet("3")) - assert(tsm0.isNodeBlacklistedForTaskSet("hostA")) + tsm.updateBlacklistForFailedTask("hostA", "3", 3) + tsm.updateBlacklistForFailedTask("hostA", "3", 4) + tracker.updateBlacklistForSuccessfulTaskSet(0, 0, tsm.execToFailures) + assert(tsm.isExecutorBlacklistedForTaskSet("3")) + assert(tsm.isNodeBlacklistedForTaskSet("hostA")) } test("only blacklist nodes when all the blacklisted executors are all on same host (app level)") { @@ -355,15 +363,15 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M val tsm0 = new TaskSetManager(scheduler, Some(tracker), taskSet0, 1, clock) tsm0.updateBlacklistForFailedTask("hostA", "1", 0) tsm0.updateBlacklistForFailedTask("hostA", "1", 1) - tracker.updateBlacklistForSuccessfulTaskSet(tsm0.execToFailures) + tracker.updateBlacklistForSuccessfulTaskSet(0, 0, tsm0.execToFailures) assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set("1")) assertEquivalentToSet(tracker.isNodeBlacklisted(_), Set()) - val taskSet1 = FakeTask.createTaskSet(4) + val taskSet1 = FakeTask.createTaskSet(4, 1, 0) val tsm1 = new TaskSetManager(scheduler, Some(tracker), taskSet1, 1, clock) tsm1.updateBlacklistForFailedTask("hostB", "2", 0) tsm1.updateBlacklistForFailedTask("hostB", "2", 1) - tracker.updateBlacklistForSuccessfulTaskSet(tsm1.execToFailures) + tracker.updateBlacklistForSuccessfulTaskSet(1, 0, tsm1.execToFailures) assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set("1", "2")) assertEquivalentToSet(tracker.isNodeBlacklisted(_), Set()) } 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 8e39c456f751a..3dff79bf9324c 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -439,9 +439,9 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B // the tasksSets complete, so the tracker should be notified verify(blacklist, times(1)).updateBlacklistForSuccessfulTaskSet( - stageToMockTsm(0).execToFailures) + 0, 0, stageToMockTsm(0).execToFailures) verify(blacklist, times(1)).updateBlacklistForSuccessfulTaskSet( - stageToMockTsm(1).execToFailures) + 1, 0, stageToMockTsm(1).execToFailures) } test("scheduled tasks obey node and executor blacklists") { @@ -661,7 +661,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B new WorkerOffer("executor0", "host0", 1) )).flatten - verify(blacklist).expireExecutorsInBlacklist() + verify(blacklist).applyBlacklistTimeout() } test("don't update blacklist for shuffle-fetch failures, preemption, denied commits, " + 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 1989be142685d..d271d4c5b9dce 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -484,7 +484,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg // Despite advancing beyond the time for expiring executors from within the blacklist, // we *never* expire from *within* the stage blacklist clock.advance(rescheduleDelay) - blacklist.expireExecutorsInBlacklist() + blacklist.applyBlacklistTimeout() { val offerResult = manager.resourceOffer("exec1", "host1", PROCESS_LOCAL) From f0428b46e7de9837fa30850be85b7fb482191a48 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 18 Aug 2016 10:24:47 -0500 Subject: [PATCH 23/57] separate datastructure to track blacklisted execs in a tsm, to simplify code --- .../spark/scheduler/TaskSetManager.scala | 23 ++++++++++--------- 1 file changed, 12 insertions(+), 11 deletions(-) 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 c54707eabcf56..eee312789e45d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -93,6 +93,7 @@ private[spark] class TaskSetManager( */ private val nodeToExecsWithFailures: HashMap[String, HashSet[String]] = new HashMap() private val nodeToBlacklistedTasks: HashMap[String, HashSet[Int]] = new HashMap() + private val blacklistedExecs: HashSet[String] = new HashSet() private val blacklistedNodes: HashSet[String] = new HashSet() @@ -867,15 +868,16 @@ private[spark] class TaskSetManager( } if (execFailures.numUniqueTasksWithFailures >= MAX_FAILURES_PER_EXEC_STAGE) { - // 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. - flatMap(execToFailures.get(_)). - filter(_.numUniqueTasksWithFailures >= MAX_FAILURES_PER_EXEC_STAGE) - if (blacklistedExecutorsOnNode.size >= MAX_FAILED_EXEC_PER_NODE_STAGE) { - if (blacklistedNodes.add(host)) { - logInfo(s"Blacklisting ${host} for stage $stageId") + 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 (blacklistedExecutorsOnNode.size >= MAX_FAILED_EXEC_PER_NODE_STAGE) { + if (blacklistedNodes.add(host)) { + logInfo(s"Blacklisting ${host} for stage $stageId") + } } } } @@ -913,8 +915,7 @@ private[spark] class TaskSetManager( * filters will already have been applied. */ def isExecutorBlacklistedForTaskSet(executorId: String): Boolean = { - execToFailures.get(executorId) - .map(_.numUniqueTasksWithFailures >= MAX_FAILURES_PER_EXEC_STAGE).getOrElse(false) + blacklistedExecs.contains(executorId) } def isNodeBlacklistedForTaskSet(node: String): Boolean = { From b582d8e84fc17d908b43b91ad819acfc5b4b79fc Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 18 Aug 2016 10:34:49 -0500 Subject: [PATCH 24/57] fix missing import --- .../main/scala/org/apache/spark/internal/config/package.scala | 2 ++ 1 file changed, 2 insertions(+) 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 0609d05c523d8..3fd0ae9623b29 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 @@ -17,6 +17,8 @@ package org.apache.spark.internal +import java.util.concurrent.TimeUnit + import org.apache.spark.launcher.SparkLauncher import org.apache.spark.network.util.ByteUnit From cec36c93c16d3b3f02a4720649c187595471663d Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 18 Aug 2016 10:47:24 -0500 Subject: [PATCH 25/57] fix line wrapping --- .../org/apache/spark/scheduler/TaskSchedulerImpl.scala | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) 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 3b15e01e4c0df..041c4649b2afb 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -58,12 +58,17 @@ private[spark] class TaskSchedulerImpl private[scheduler]( extends TaskScheduler with Logging { def this(sc: SparkContext) = { - this(sc, sc.conf.getInt("spark.task.maxFailures", 4), + this( + sc, + sc.conf.getInt("spark.task.maxFailures", 4), TaskSchedulerImpl.createBlacklistTracker(sc.conf)) } def this(sc: SparkContext, maxTaskFailures: Int, isLocal: Boolean) = { - this(sc, maxTaskFailures, TaskSchedulerImpl.createBlacklistTracker(sc.conf), + this( + sc, + maxTaskFailures, + TaskSchedulerImpl.createBlacklistTracker(sc.conf), isLocal = isLocal) } From 290b3154d8cc179f6f18742ce53f50861fcc2c9c Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 18 Aug 2016 15:06:16 -0500 Subject: [PATCH 26/57] fix test by turning off blacklist --- core/src/test/scala/org/apache/spark/DistributedSuite.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/DistributedSuite.scala b/core/src/test/scala/org/apache/spark/DistributedSuite.scala index 90eaca58c2a30..72638c59511b6 100644 --- a/core/src/test/scala/org/apache/spark/DistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/DistributedSuite.scala @@ -142,7 +142,9 @@ class DistributedSuite extends SparkFunSuite with Matchers with LocalSparkContex test("repeatedly failing task that crashes JVM with a zero exit code (SPARK-16925)") { // Ensures that if a task which causes the JVM to exit with a zero exit code will cause the // Spark job to eventually fail. - sc = new SparkContext(clusterUrl, "test") + val conf = new SparkConf().setAppName("test").setMaster(clusterUrl) + .set(BLACKLIST_ENABLED, false) + sc = new SparkContext(conf) failAfter(Span(100000, Millis)) { val thrown = intercept[SparkException] { sc.parallelize(1 to 1, 1).foreachPartition { _ => System.exit(0) } From 8c58ad96db948bec95f2e99fb9298db481e60b58 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 18 Aug 2016 17:04:48 -0500 Subject: [PATCH 27/57] unused import --- .../scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 041c4649b2afb..62e703a47b011 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -33,7 +33,7 @@ import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.scheduler.TaskLocality.TaskLocality import org.apache.spark.scheduler.local.LocalSchedulerBackend import org.apache.spark.storage.BlockManagerId -import org.apache.spark.util.{AccumulatorV2, Clock, SystemClock, ThreadUtils, Utils} +import org.apache.spark.util.{AccumulatorV2, SystemClock, ThreadUtils, Utils} /** * Schedules tasks for multiple types of clusters by acting through a SchedulerBackend. From f01278038b5ee5cfdb80de9b254ae9fab0bd5f52 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 22 Aug 2016 10:34:32 -0500 Subject: [PATCH 28/57] review feedback --- core/src/main/scala/org/apache/spark/TaskEndReason.scala | 4 ++-- docs/configuration.md | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala index 20ea7d04d6480..59a25398a039b 100644 --- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala +++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala @@ -213,7 +213,7 @@ case object TaskResultLost extends TaskFailedReason { @DeveloperApi case object TaskKilled extends TaskFailedReason { override def toErrorString: String = "TaskKilled (killed intentionally)" - override val countTowardsTaskFailures: Boolean = false + override def countTowardsTaskFailures: Boolean = false } /** @@ -232,7 +232,7 @@ case class TaskCommitDenied( * towards failing the stage. This is intended to prevent spurious stage failures in cases * where many speculative tasks are launched and denied to commit. */ - override val countTowardsTaskFailures: Boolean = false + override def countTowardsTaskFailures: Boolean = false } /** diff --git a/docs/configuration.md b/docs/configuration.md index 689981f9d8831..6b2d0bcd2bd61 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1199,7 +1199,7 @@ Apart from these, the following properties are also available, and may be useful spark.blacklist.task.maxTaskAttemptsPerExecutor - 2 + 1 (Experimental) For a given task, how many times it can be retried on one executor before the executor is blacklisted for that task. From fc45f5b2e2fc38aff0714f1465f03f5e0ba16e01 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 22 Aug 2016 10:57:34 -0500 Subject: [PATCH 29/57] fix some typos --- core/src/main/scala/org/apache/spark/SparkConf.scala | 2 +- core/src/test/scala/org/apache/spark/DistributedSuite.scala | 2 +- .../apache/spark/scheduler/SchedulerIntegrationSuite.scala | 5 +++-- 3 files changed, 5 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index c648034fb3e90..bb2e692cbdd2e 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -633,7 +633,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 any more."), DeprecatedConfig("spark.scheduler.executorTaskBlacklistTime", "2.1.0", - "Please use the new blacklisting options, spark.scheduler.blacklist.*") + "Please use the new blacklisting options, spark.blacklist.*") ) Map(configs.map { cfg => (cfg.key -> cfg) } : _*) diff --git a/core/src/test/scala/org/apache/spark/DistributedSuite.scala b/core/src/test/scala/org/apache/spark/DistributedSuite.scala index 72638c59511b6..eec7744ae356b 100644 --- a/core/src/test/scala/org/apache/spark/DistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/DistributedSuite.scala @@ -292,7 +292,7 @@ class DistributedSuite extends SparkFunSuite with Matchers with LocalSparkContex import DistributedSuite.{markNodeIfIdentity, failOnMarkedIdentity} DistributedSuite.amMaster = true val conf = new SparkConf().setAppName("test").setMaster(clusterUrl) - .set("spark.scheduler.blacklist.enabled", "false") + .set(BLACKLIST_ENABLED, false) sc = new SparkContext(conf) for (i <- 1 to 3) { val data = sc.parallelize(Seq(true, true), 2) diff --git a/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala index 60cabc14c4d1d..9f7184a75042e 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala @@ -31,6 +31,7 @@ import org.scalatest.Assertions.AssertionsHelper import org.apache.spark._ import org.apache.spark.TaskState._ +import org.apache.spark.internal.config.BLACKLIST_ENABLED import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.util.{CallSite, ThreadUtils, Utils} @@ -642,7 +643,7 @@ class BasicSchedulerIntegrationSuite extends SchedulerIntegrationSuite[SingleCor val jobFuture = submit(new MockRDD(sc, 10, Nil), (0 until 10).toArray) val duration = Duration(1, SECONDS) awaitJobTermination(jobFuture, duration) - failure.getMessage.contains("test task failure") + assert(failure.getMessage.contains("test task failure")) } assertDataStructuresEmpty(noFailure = false) } @@ -651,6 +652,6 @@ class BasicSchedulerIntegrationSuite extends SchedulerIntegrationSuite[SingleCor def testNoBlacklist(name: String)(body: => Unit): Unit = { // in these simple tests, we only have one executor, so it doens't make sense to turn on the // blacklist. Just an artifact of this simple test-framework still kinda acting like local-mode - testScheduler(name, extraConfs = Seq("spark.scheduler.blacklist.enabled" -> "false"))(body) + testScheduler(name, extraConfs = Seq(BLACKLIST_ENABLED.key -> "false"))(body) } } From f8b1bffee588df45809519436983cb95c6a481f3 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 22 Aug 2016 12:02:15 -0500 Subject: [PATCH 30/57] add validation for blacklist confs --- .../spark/scheduler/BlacklistTracker.scala | 60 +++++++++++++++++++ .../scheduler/BlacklistIntegrationSuite.scala | 5 +- .../scheduler/BlacklistTrackerSuite.scala | 42 +++++++++++++ 3 files changed, 106 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala index 68c3773e13041..46d83f0741413 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala @@ -51,6 +51,7 @@ private[scheduler] class BlacklistTracker ( conf: SparkConf, clock: Clock = new SystemClock()) extends Logging { + 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) @@ -249,6 +250,65 @@ private[scheduler] object BlacklistTracker extends Logging { } } } + + /** + * 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.") + } + + // undocumented escape hatch for validation -- just for tests that want to run in an "unsafe" + // configuration. + if (!conf.get("spark.blacklist.testing.skipValidation", "false").toBoolean) { + + 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.getInt("spark.task.maxFailures", 4) + val maxNodeAttempts = conf.get(config.MAX_TASK_ATTEMPTS_PER_NODE) + + if (maxTaskFailures <= maxNodeAttempts) { + throw new IllegalArgumentException(s"${config.MAX_TASK_ATTEMPTS_PER_NODE.key} " + + s"( = ${maxNodeAttempts}) was <= spark.task.maxFailures " + + s"( = ${maxTaskFailures} ). Though blacklisting is enabled, with this configuration, " + + s"Spark will not be robust to one failed disk. Increase " + + s"${config.MAX_TASK_ATTEMPTS_PER_NODE.key} or spark.task.maxFailures, or disable " + + s"blacklisting with ${config.BLACKLIST_ENABLED.key}") + } + } + + } } /** Failures for one executor, within one taskset */ diff --git a/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala index 2ee3aa2fda8d4..f8596a88bc510 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala @@ -65,7 +65,10 @@ class BlacklistIntegrationSuite extends SchedulerIntegrationSuite[MultiExecutorM "spark.task.maxFailures" -> "4", "spark.testing.nHosts" -> "2", "spark.testing.nExecutorsPerHost" -> "5", - "spark.testing.nCoresPerExecutor" -> "10" + "spark.testing.nCoresPerExecutor" -> "10", + // Blacklisting will normally immediately complain that this config is invalid -- the point + // of this test is to expose that the configuration is unsafe, so skip the validation. + "spark.blacklist.testing.skipValidation" -> "true" ) ) { // to reliably reproduce the failure, we have to use 1 task. That way, we ensure this diff --git a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala index b382533e92f43..e0635928db891 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala @@ -418,4 +418,46 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M assert(10 * 60 * 60 * 1000L == BlacklistTracker.getBlacklistTimeout(distConf)) } } + + test("check blacklist configuration invariants") { + val conf = new SparkConf().setMaster("yarn-cluster") + Seq( + (2, 2), + (2, 3), + (3, 3) + ).foreach { case (maxTaskFailures, maxNodeAttempts) => + conf.set("spark.task.maxFailures", maxTaskFailures.toString) + conf.set(config.MAX_TASK_ATTEMPTS_PER_NODE.key, maxNodeAttempts.toString) + val excMsg = intercept[IllegalArgumentException] { + BlacklistTracker.validateBlacklistConfs(conf) + }.getMessage() + assert(excMsg.contains(s"${config.MAX_TASK_ATTEMPTS_PER_NODE.key} " + + s"( = ${maxNodeAttempts}) was <= spark.task.maxFailures " + + s"( = ${maxTaskFailures} ). Though blacklisting is enabled, with this configuration, " + + s"Spark will not be robust to one failed disk. Increase " + + s"${config.MAX_TASK_ATTEMPTS_PER_NODE.key } or spark.task.maxFailures, or disable " + + s"blacklisting with ${config.BLACKLIST_ENABLED.key}")) + } + + conf.remove("spark.task.maxFailures") + 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) + } + } } From e56bb90152b4198556773f5cbc39a192107eeae2 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 22 Aug 2016 17:06:41 -0500 Subject: [PATCH 31/57] update test to turn off blacklist --- .../spark/serializer/KryoSerializerDistributedSuite.scala | 2 ++ 1 file changed, 2 insertions(+) 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 c1484b0afa85f..01e4135c398af 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.serializer import com.esotericsoftware.kryo.Kryo import org.apache.spark._ +import org.apache.spark.internal.config import org.apache.spark.serializer.KryoDistributedTest._ import org.apache.spark.util.Utils @@ -30,6 +31,7 @@ class KryoSerializerDistributedSuite extends SparkFunSuite with LocalSparkContex .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") .set("spark.kryo.registrator", classOf[AppJarRegistrator].getName) .set("spark.task.maxFailures", "1") + .set(config.BLACKLIST_ENABLED, false) val jar = TestUtils.createJarWithClasses(List(AppJarRegistrator.customClassName)) conf.setJars(List(jar.getPath)) From cc3b9683940eebc930e9bf7a5088dc987da23595 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Fri, 26 Aug 2016 14:44:06 -0500 Subject: [PATCH 32/57] fix timeout of individual tasks --- .../spark/scheduler/BlacklistTracker.scala | 97 ++++++++++++++----- .../scheduler/BlacklistTrackerSuite.scala | 66 +++++++++++-- 2 files changed, 129 insertions(+), 34 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala index 46d83f0741413..c70156e1badd2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala @@ -60,7 +60,8 @@ private[scheduler] class BlacklistTracker ( * 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. */ - private val executorIdToFailureList: HashMap[String, ExecutorFailureList] = new HashMap() + private[scheduler] val executorIdToFailureList: HashMap[String, ExecutorFailureList] = + new HashMap() val executorIdToBlacklistStatus: HashMap[String, BlacklistedExecutor] = new HashMap() val nodeIdToBlacklistExpiryTime: HashMap[String, Long] = new HashMap() /** @@ -69,10 +70,16 @@ private[scheduler] class BlacklistTracker ( */ private val _nodeBlacklist: AtomicReference[Set[String]] = new AtomicReference(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. + * The next time we should check if anything has hit the timeout for the blacklists. Used as a + * shortcut to avoid iterating over all entries in the blacklist when none will have expired. + * + * This is never greater than min(executor_blacklist_timeout), so that we can unblacklist + * executors quickly. However, it may be much greater than min(task_blacklist_timeout) -- we + * might have lots of task failures, all slightly spread out in time, and we don't want to do an + * expensive check for each one. But we do need to clear out those blacklisted tasks to avoid + * expensive memory. */ - private var nextExpiryTime: Long = Long.MaxValue + private[scheduler] var nextTimeoutCheck: 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 @@ -85,7 +92,7 @@ private[scheduler] class BlacklistTracker ( 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) { + if (now > nextTimeoutCheck) { // 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 @@ -97,8 +104,11 @@ private[scheduler] class BlacklistTracker ( // --> 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. - executorIdToFailureList.values.foreach { executorFailureList => + executorIdToFailureList.foreach { case (exec, executorFailureList) => executorFailureList.dropFailuresWithTimeoutBefore(now) + if (executorFailureList.isEmpty) { + executorIdToFailureList.remove(exec) + } } // Apply the timeout to blacklisted nodes and executors @@ -116,11 +126,6 @@ private[scheduler] class BlacklistTracker ( } } } - if (executorIdToBlacklistStatus.nonEmpty) { - nextExpiryTime = executorIdToBlacklistStatus.map{_._2.expiryTime}.min - } else { - nextExpiryTime = Long.MaxValue - } val nodesToUnblacklist = nodeIdToBlacklistExpiryTime.filter(_._2 < now).keys if (nodesToUnblacklist.nonEmpty) { // Un-blacklist any nodes that have been blacklisted longer than the blacklist timeout. @@ -129,8 +134,43 @@ private[scheduler] class BlacklistTracker ( nodesToUnblacklist.foreach { node => nodeIdToBlacklistExpiryTime.remove(node) } _nodeBlacklist.set(nodeIdToBlacklistExpiryTime.keySet.toSet) } + updateNextExpiryTime(now) + } + } + + /** + * Determine the new nextExpiryTime. + * Nodes always have the same expiry time as a some + * blacklisted executor. Blacklisted executors also have the same expiry time as some task, + * but we pro-actively purge the task expiry times when we blacklist an executor to keep + * memory under control. + */ + private def updateNextExpiryTime(now: Long): Unit = { + if (executorIdToFailureList.nonEmpty) { + // Optimization: we want to balance doing the work of checking the timeouts very often, + // against having too much memory build up from tracking the task failures in between + // checks. Its not uncommon to have a large number of task failures, *slightly* spread + // out in time. We want to avoid setting a separate timeout for each one of those + // task failures. So a heuristic here is to set the next check that we do for tasks + // at least some time in the future. We don't want to do this for executors, as we want + // to unblacklist them as quickly as we can. The actual padding here doesn't effect + // correctness -- any value in [0, inf) would still lead to correct behavior, it would + // just effect how much memory we use vs. how often we do the work of checking the exact + // times. + nextTimeoutCheck = math.max( + now + BLACKLIST_TIMEOUT_MILLIS, + executorIdToFailureList.values.map(_.minExpiryTime).min) + } else { + nextTimeoutCheck = Long.MaxValue + } + if (executorIdToBlacklistStatus.nonEmpty) { + nextTimeoutCheck = math.min( + nextTimeoutCheck, + executorIdToBlacklistStatus.map{_._2.expiryTime}.min) } - } + + } + def updateBlacklistForSuccessfulTaskSet( stageId: Int, @@ -138,26 +178,21 @@ private[scheduler] class BlacklistTracker ( 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() + val expiryTime = now + BLACKLIST_TIMEOUT_MILLIS failuresByExec.foreach { case (exec, failuresInTaskSet) => - val allExecutorFailures = + val allFailuresOnOneExecutor = executorIdToFailureList.getOrElseUpdate(exec, new ExecutorFailureList) - allExecutorFailures.addFailures(stageId, stageAttemptId, failuresInTaskSet) - val newTotal = allExecutorFailures.numUniqueTaskFailures - if (allExecutorFailures.minExpiryTime < nextExpiryTime) { - nextExpiryTime = allExecutorFailures.minExpiryTime - } + allFailuresOnOneExecutor.addFailures(stageId, stageAttemptId, failuresInTaskSet) + allFailuresOnOneExecutor.dropFailuresWithTimeoutBefore(now) + val newTotal = allFailuresOnOneExecutor.numUniqueTaskFailures if (newTotal >= MAX_FAILURES_PER_EXEC) { logInfo(s"Blacklisting executor id: $exec because it has $newTotal" + s" task failures in successful task sets") - val now = clock.getTimeMillis() - val expiryTime = now + BLACKLIST_TIMEOUT_MILLIS val node = failuresInTaskSet.node executorIdToBlacklistStatus.put(exec, BlacklistedExecutor(node, expiryTime)) executorIdToFailureList.remove(exec) - if (expiryTime < nextExpiryTime) { - nextExpiryTime = expiryTime - } // 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. @@ -171,6 +206,9 @@ private[scheduler] class BlacklistTracker ( } } } + if (failuresByExec.nonEmpty) { + updateNextExpiryTime(now) + } } def isExecutorBlacklisted(executorId: String): Boolean = { @@ -315,7 +353,7 @@ private[scheduler] object BlacklistTracker extends Logging { private[scheduler] final class ExecutorFailuresInTaskSet(val node: String) { /** * Mapping from index of the tasks in the taskset, to the number of times it has failed on this - * executor. + * executor and the last time it failed. */ val taskToFailureCountAndExpiryTime = HashMap[Int, (Int, Long)]() def updateWithFailure(taskIndex: Int, failureExpiryTime: Long): Unit = { @@ -336,6 +374,8 @@ private[scheduler] final class ExecutorFailuresInTaskSet(val node: String) { /** * Tracks all failures for one executor (that have not passed the timeout). Designed to efficiently * remove failures that are older than the timeout, and query for the number of unique failed tasks. + * 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 { @@ -362,7 +402,7 @@ private[scheduler] final class ExecutorFailureList extends Logging { } } - def minExpiryTime: Long = failures.head._2 + def minExpiryTime: Long = failures.headOption.map(_._2).getOrElse(Long.MaxValue) /** * The number of unique tasks that failed on this executor. Only counts failures within the @@ -370,6 +410,8 @@ private[scheduler] final class ExecutorFailureList extends Logging { */ def numUniqueTaskFailures: Int = failures.size + def isEmpty: Boolean = failures.isEmpty + def dropFailuresWithTimeoutBefore(dropBefore: Long): Unit = { if (minExpiryTime < dropBefore) { val minIndexToKeep = failures.indexWhere(_._2 >= dropBefore) @@ -380,6 +422,11 @@ private[scheduler] final class ExecutorFailureList extends Logging { } } } + + override def toString(): String = { + s"failures = $failures" + } + } private final case class BlacklistedExecutor(node: String, expiryTime: Long) diff --git a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala index e0635928db891..50628838ed1fc 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala @@ -278,29 +278,77 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M test("task failures expire with time") { val (tracker, scheduler) = trackerFixture var stageId = 0 - def failOneTaskInTaskSet(): Unit = { + def failOneTaskInTaskSet(exec: String): Unit = { val taskSet = FakeTask.createTaskSet(1, stageId, 0) val tsm = new TaskSetManager(scheduler, Some(tracker), taskSet, 1, clock) - tsm.updateBlacklistForFailedTask("hostA", "1", 0) + tsm.updateBlacklistForFailedTask("host-" + exec, exec, 0) tracker.updateBlacklistForSuccessfulTaskSet(stageId, 0, tsm.execToFailures) stageId += 1 } - failOneTaskInTaskSet() + failOneTaskInTaskSet("1") + // We have one sporadic failure on exec 2 -- it doesn't lead to an exec blacklist. + failOneTaskInTaskSet("2") assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set()) + assert(tracker.nextTimeoutCheck === tracker.BLACKLIST_TIMEOUT_MILLIS) - // now we advance the clock past the expiry time + // We advance the clock past the expiry time. clock.advance(tracker.BLACKLIST_TIMEOUT_MILLIS + 1) + val t0 = clock.getTimeMillis() tracker.applyBlacklistTimeout() - failOneTaskInTaskSet() + assert(tracker.executorIdToFailureList.isEmpty) // make sure we're not leaking memory + assert(tracker.nextTimeoutCheck === Long.MaxValue) + failOneTaskInTaskSet("1") - // because we went past the expiry time, nothing should have been blacklisted + // Because we went past the expiry time, nothing should have been blacklisted. assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set()) - // now we add one more failure, within the timeout, and it should be counted - clock.advance(tracker.BLACKLIST_TIMEOUT_MILLIS - 1) - failOneTaskInTaskSet() + // Now we add one more failure, within the timeout, and it should be counted. + clock.setTime(t0 + tracker.BLACKLIST_TIMEOUT_MILLIS) + val t1 = clock.getTimeMillis() + failOneTaskInTaskSet("1") + tracker.applyBlacklistTimeout() assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set("1")) + // The 2x is because of a heuristic to avoid too many checks from task failures. + assert(tracker.nextTimeoutCheck === t0 + 2 * tracker.BLACKLIST_TIMEOUT_MILLIS) + + // Fail a second executor, and go over its expiry as well. + clock.setTime(t1 + tracker.BLACKLIST_TIMEOUT_MILLIS) + val t2 = clock.getTimeMillis() + failOneTaskInTaskSet("3") + failOneTaskInTaskSet("3") + tracker.applyBlacklistTimeout() + assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set("1", "3")) + assert(tracker.nextTimeoutCheck === t1 + tracker.BLACKLIST_TIMEOUT_MILLIS) + + + clock.setTime(t1 + tracker.BLACKLIST_TIMEOUT_MILLIS + 1) + tracker.applyBlacklistTimeout() + assert(tracker.executorIdToFailureList.isEmpty) + assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set("3")) + assert(tracker.nextTimeoutCheck === t2 + tracker.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 + tracker.BLACKLIST_TIMEOUT_MILLIS) + val t3 = clock.getTimeMillis() + failOneTaskInTaskSet("4") + tracker.applyBlacklistTimeout() + assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set("3")) + assert(tracker.nextTimeoutCheck === t2 + tracker.BLACKLIST_TIMEOUT_MILLIS) + + clock.setTime(t2 + tracker.BLACKLIST_TIMEOUT_MILLIS + 500) + val t4 = clock.getTimeMillis() + tracker.applyBlacklistTimeout() + assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set()) + // The extra time here is b/c we pad the time around task failure timeouts to avoid too many + // checks (otherwise this would be t3 + timeout). + assert(tracker.nextTimeoutCheck === t4 + tracker.BLACKLIST_TIMEOUT_MILLIS) + + clock.setTime(t4 + tracker.BLACKLIST_TIMEOUT_MILLIS + 1) + tracker.applyBlacklistTimeout() + assert(tracker.executorIdToFailureList.isEmpty) + assert(tracker.nextTimeoutCheck === Long.MaxValue) } test("multiple attempts for the same task count once") { From 5fdfe49d16996e6317b291e469656f68d1def3a8 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Fri, 26 Aug 2016 15:20:16 -0500 Subject: [PATCH 33/57] simplify task expiry by doing it lazily --- .../spark/scheduler/BlacklistTracker.scala | 89 ++++++------------- .../scheduler/BlacklistTrackerSuite.scala | 28 +++--- 2 files changed, 36 insertions(+), 81 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala index c70156e1badd2..4c647f1cdfc17 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala @@ -58,7 +58,10 @@ private[scheduler] class BlacklistTracker ( /** * 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. + * 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[scheduler] val executorIdToFailureList: HashMap[String, ExecutorFailureList] = new HashMap() @@ -70,16 +73,10 @@ private[scheduler] class BlacklistTracker ( */ private val _nodeBlacklist: AtomicReference[Set[String]] = new AtomicReference(Set()) /** - * The next time we should check if anything has hit the timeout for the blacklists. Used as a + * 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. - * - * This is never greater than min(executor_blacklist_timeout), so that we can unblacklist - * executors quickly. However, it may be much greater than min(task_blacklist_timeout) -- we - * might have lots of task failures, all slightly spread out in time, and we don't want to do an - * expensive check for each one. But we do need to clear out those blacklisted tasks to avoid - * expensive memory. */ - private[scheduler] var nextTimeoutCheck: Long = Long.MaxValue + private[scheduler] 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 @@ -92,25 +89,7 @@ private[scheduler] class BlacklistTracker ( 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 > nextTimeoutCheck) { - // 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. - executorIdToFailureList.foreach { case (exec, executorFailureList) => - executorFailureList.dropFailuresWithTimeoutBefore(now) - if (executorFailureList.isEmpty) { - executorIdToFailureList.remove(exec) - } - } - + if (now > nextExpiryTime) { // Apply the timeout to blacklisted nodes and executors val execsToUnblacklist = executorIdToBlacklistStatus.filter(_._2.expiryTime < now).keys if (execsToUnblacklist.nonEmpty) { @@ -134,41 +113,16 @@ private[scheduler] class BlacklistTracker ( nodesToUnblacklist.foreach { node => nodeIdToBlacklistExpiryTime.remove(node) } _nodeBlacklist.set(nodeIdToBlacklistExpiryTime.keySet.toSet) } - updateNextExpiryTime(now) - } + updateNextExpiryTime() + } } - /** - * Determine the new nextExpiryTime. - * Nodes always have the same expiry time as a some - * blacklisted executor. Blacklisted executors also have the same expiry time as some task, - * but we pro-actively purge the task expiry times when we blacklist an executor to keep - * memory under control. - */ - private def updateNextExpiryTime(now: Long): Unit = { - if (executorIdToFailureList.nonEmpty) { - // Optimization: we want to balance doing the work of checking the timeouts very often, - // against having too much memory build up from tracking the task failures in between - // checks. Its not uncommon to have a large number of task failures, *slightly* spread - // out in time. We want to avoid setting a separate timeout for each one of those - // task failures. So a heuristic here is to set the next check that we do for tasks - // at least some time in the future. We don't want to do this for executors, as we want - // to unblacklist them as quickly as we can. The actual padding here doesn't effect - // correctness -- any value in [0, inf) would still lead to correct behavior, it would - // just effect how much memory we use vs. how often we do the work of checking the exact - // times. - nextTimeoutCheck = math.max( - now + BLACKLIST_TIMEOUT_MILLIS, - executorIdToFailureList.values.map(_.minExpiryTime).min) - } else { - nextTimeoutCheck = Long.MaxValue - } + private def updateNextExpiryTime(): Unit = { if (executorIdToBlacklistStatus.nonEmpty) { - nextTimeoutCheck = math.min( - nextTimeoutCheck, - executorIdToBlacklistStatus.map{_._2.expiryTime}.min) + nextExpiryTime = executorIdToBlacklistStatus.map{_._2.expiryTime}.min + } else { + nextExpiryTime = Long.MaxValue } - } @@ -183,8 +137,19 @@ private[scheduler] class BlacklistTracker ( failuresByExec.foreach { case (exec, failuresInTaskSet) => val allFailuresOnOneExecutor = executorIdToFailureList.getOrElseUpdate(exec, new ExecutorFailureList) - allFailuresOnOneExecutor.addFailures(stageId, stageAttemptId, failuresInTaskSet) + // 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. allFailuresOnOneExecutor.dropFailuresWithTimeoutBefore(now) + allFailuresOnOneExecutor.addFailures(stageId, stageAttemptId, failuresInTaskSet) val newTotal = allFailuresOnOneExecutor.numUniqueTaskFailures if (newTotal >= MAX_FAILURES_PER_EXEC) { @@ -193,6 +158,7 @@ private[scheduler] class BlacklistTracker ( val node = failuresInTaskSet.node executorIdToBlacklistStatus.put(exec, BlacklistedExecutor(node, expiryTime)) executorIdToFailureList.remove(exec) + updateNextExpiryTime() // 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. @@ -206,9 +172,6 @@ private[scheduler] class BlacklistTracker ( } } } - if (failuresByExec.nonEmpty) { - updateNextExpiryTime(now) - } } def isExecutorBlacklisted(executorId: String): Boolean = { diff --git a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala index 50628838ed1fc..8ba2496d63635 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala @@ -289,14 +289,13 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M // We have one sporadic failure on exec 2 -- it doesn't lead to an exec blacklist. failOneTaskInTaskSet("2") assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set()) - assert(tracker.nextTimeoutCheck === tracker.BLACKLIST_TIMEOUT_MILLIS) + assert(tracker.nextExpiryTime === Long.MaxValue) // We advance the clock past the expiry time. clock.advance(tracker.BLACKLIST_TIMEOUT_MILLIS + 1) val t0 = clock.getTimeMillis() tracker.applyBlacklistTimeout() - assert(tracker.executorIdToFailureList.isEmpty) // make sure we're not leaking memory - assert(tracker.nextTimeoutCheck === Long.MaxValue) + assert(tracker.nextExpiryTime === Long.MaxValue) failOneTaskInTaskSet("1") // Because we went past the expiry time, nothing should have been blacklisted. @@ -308,8 +307,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M failOneTaskInTaskSet("1") tracker.applyBlacklistTimeout() assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set("1")) - // The 2x is because of a heuristic to avoid too many checks from task failures. - assert(tracker.nextTimeoutCheck === t0 + 2 * tracker.BLACKLIST_TIMEOUT_MILLIS) + assert(tracker.nextExpiryTime === t1 + tracker.BLACKLIST_TIMEOUT_MILLIS) // Fail a second executor, and go over its expiry as well. clock.setTime(t1 + tracker.BLACKLIST_TIMEOUT_MILLIS) @@ -318,14 +316,13 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M failOneTaskInTaskSet("3") tracker.applyBlacklistTimeout() assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set("1", "3")) - assert(tracker.nextTimeoutCheck === t1 + tracker.BLACKLIST_TIMEOUT_MILLIS) + assert(tracker.nextExpiryTime === t1 + tracker.BLACKLIST_TIMEOUT_MILLIS) clock.setTime(t1 + tracker.BLACKLIST_TIMEOUT_MILLIS + 1) tracker.applyBlacklistTimeout() - assert(tracker.executorIdToFailureList.isEmpty) assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set("3")) - assert(tracker.nextTimeoutCheck === t2 + tracker.BLACKLIST_TIMEOUT_MILLIS) + assert(tracker.nextExpiryTime === t2 + tracker.BLACKLIST_TIMEOUT_MILLIS) // Make sure that we update correctly when we go from having blacklisted executors to @@ -335,20 +332,15 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M failOneTaskInTaskSet("4") tracker.applyBlacklistTimeout() assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set("3")) - assert(tracker.nextTimeoutCheck === t2 + tracker.BLACKLIST_TIMEOUT_MILLIS) + assert(tracker.nextExpiryTime === t2 + tracker.BLACKLIST_TIMEOUT_MILLIS) - clock.setTime(t2 + tracker.BLACKLIST_TIMEOUT_MILLIS + 500) + clock.setTime(t2 + tracker.BLACKLIST_TIMEOUT_MILLIS + 1) val t4 = clock.getTimeMillis() tracker.applyBlacklistTimeout() assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set()) - // The extra time here is b/c we pad the time around task failure timeouts to avoid too many - // checks (otherwise this would be t3 + timeout). - assert(tracker.nextTimeoutCheck === t4 + tracker.BLACKLIST_TIMEOUT_MILLIS) - - clock.setTime(t4 + tracker.BLACKLIST_TIMEOUT_MILLIS + 1) - tracker.applyBlacklistTimeout() - assert(tracker.executorIdToFailureList.isEmpty) - assert(tracker.nextTimeoutCheck === Long.MaxValue) + // 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(tracker.nextExpiryTime === Long.MaxValue) } test("multiple attempts for the same task count once") { From e10fa109cbc15b0c6fcfb59974585b2ed00e0506 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 31 Aug 2016 10:09:35 -0500 Subject: [PATCH 34/57] review feedback --- .../org/apache/spark/scheduler/BlacklistTracker.scala | 8 +++----- .../apache/spark/scheduler/BlacklistTrackerSuite.scala | 4 ++-- 2 files changed, 5 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala index 4c647f1cdfc17..9a29a013561b8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala @@ -299,11 +299,11 @@ private[scheduler] object BlacklistTracker extends Logging { val maxTaskFailures = conf.getInt("spark.task.maxFailures", 4) val maxNodeAttempts = conf.get(config.MAX_TASK_ATTEMPTS_PER_NODE) - if (maxTaskFailures <= maxNodeAttempts) { + if (maxNodeAttempts >= maxTaskFailures) { throw new IllegalArgumentException(s"${config.MAX_TASK_ATTEMPTS_PER_NODE.key} " + - s"( = ${maxNodeAttempts}) was <= spark.task.maxFailures " + + s"( = ${maxNodeAttempts}) was >= spark.task.maxFailures " + s"( = ${maxTaskFailures} ). Though blacklisting is enabled, with this configuration, " + - s"Spark will not be robust to one failed disk. Increase " + + s"Spark will not be robust to one bad node. Increase " + s"${config.MAX_TASK_ATTEMPTS_PER_NODE.key} or spark.task.maxFailures, or disable " + s"blacklisting with ${config.BLACKLIST_ENABLED.key}") } @@ -327,7 +327,6 @@ private[scheduler] final class ExecutorFailuresInTaskSet(val node: String) { } def numUniqueTasksWithFailures: Int = taskToFailureCountAndExpiryTime.size - override def toString(): String = { s"numUniqueTasksWithFailures= $numUniqueTasksWithFailures; " + s"tasksToFailureCount = $taskToFailureCountAndExpiryTime" @@ -389,7 +388,6 @@ private[scheduler] final class ExecutorFailureList extends Logging { override def toString(): String = { s"failures = $failures" } - } private final case class BlacklistedExecutor(node: String, expiryTime: Long) diff --git a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala index 8ba2496d63635..f7314b109fffa 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala @@ -472,9 +472,9 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M BlacklistTracker.validateBlacklistConfs(conf) }.getMessage() assert(excMsg.contains(s"${config.MAX_TASK_ATTEMPTS_PER_NODE.key} " + - s"( = ${maxNodeAttempts}) was <= spark.task.maxFailures " + + s"( = ${maxNodeAttempts}) was >= spark.task.maxFailures " + s"( = ${maxTaskFailures} ). Though blacklisting is enabled, with this configuration, " + - s"Spark will not be robust to one failed disk. Increase " + + s"Spark will not be robust to one bad node. Increase " + s"${config.MAX_TASK_ATTEMPTS_PER_NODE.key } or spark.task.maxFailures, or disable " + s"blacklisting with ${config.BLACKLIST_ENABLED.key}")) } From c78964fb4dad0662f1670634126d538aa71b1cf5 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 31 Aug 2016 10:40:55 -0500 Subject: [PATCH 35/57] fix bad merge --- .../scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala | 3 --- 1 file changed, 3 deletions(-) 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 8f4b615070b13..00bb7289d3880 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -294,9 +294,6 @@ private[spark] class TaskSchedulerImpl private[scheduler]( } } } - if (!launchedTask && blacklistTracker.isDefined) { - taskSet.abortIfCompletelyBlacklisted(hostToExecutors) - } return launchedTask } From 463b83773742932f7b75d9cd47154c08d7700d22 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 21 Sep 2016 13:26:37 -0500 Subject: [PATCH 36/57] more cleanup of TaskEndReason -> TaskFailedReason --- .../org/apache/spark/executor/CommitDeniedException.scala | 4 ++-- core/src/main/scala/org/apache/spark/executor/Executor.scala | 4 ++-- .../scala/org/apache/spark/shuffle/FetchFailedException.scala | 4 ++-- .../test/scala/org/apache/spark/util/JsonProtocolSuite.scala | 2 +- 4 files changed, 7 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/CommitDeniedException.scala b/core/src/main/scala/org/apache/spark/executor/CommitDeniedException.scala index 7d84889a2def0..326e042419774 100644 --- a/core/src/main/scala/org/apache/spark/executor/CommitDeniedException.scala +++ b/core/src/main/scala/org/apache/spark/executor/CommitDeniedException.scala @@ -17,7 +17,7 @@ package org.apache.spark.executor -import org.apache.spark.{TaskCommitDenied, TaskEndReason} +import org.apache.spark.{TaskCommitDenied, TaskFailedReason} /** * Exception thrown when a task attempts to commit output to HDFS but is denied by the driver. @@ -29,5 +29,5 @@ private[spark] class CommitDeniedException( attemptNumber: Int) extends Exception(msg) { - def toTaskEndReason: TaskEndReason = TaskCommitDenied(jobID, splitID, attemptNumber) + def toTaskFailedReason: TaskFailedReason = TaskCommitDenied(jobID, splitID, attemptNumber) } diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index fbf2b86db1a2e..668ec41153086 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -355,7 +355,7 @@ private[spark] class Executor( } catch { case ffe: FetchFailedException => - val reason = ffe.toTaskEndReason + val reason = ffe.toTaskFailedReason setTaskFinishedAndClearInterruptStatus() execBackend.statusUpdate(taskId, TaskState.FAILED, ser.serialize(reason)) @@ -370,7 +370,7 @@ private[spark] class Executor( execBackend.statusUpdate(taskId, TaskState.KILLED, ser.serialize(TaskKilled)) case CausedBy(cDE: CommitDeniedException) => - val reason = cDE.toTaskEndReason + val reason = cDE.toTaskFailedReason setTaskFinishedAndClearInterruptStatus() execBackend.statusUpdate(taskId, TaskState.FAILED, ser.serialize(reason)) diff --git a/core/src/main/scala/org/apache/spark/shuffle/FetchFailedException.scala b/core/src/main/scala/org/apache/spark/shuffle/FetchFailedException.scala index b2d050b218f53..498c12e196ce0 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/FetchFailedException.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/FetchFailedException.scala @@ -17,7 +17,7 @@ package org.apache.spark.shuffle -import org.apache.spark.{FetchFailed, TaskEndReason} +import org.apache.spark.{FetchFailed, TaskFailedReason} import org.apache.spark.storage.BlockManagerId import org.apache.spark.util.Utils @@ -45,7 +45,7 @@ private[spark] class FetchFailedException( this(bmAddress, shuffleId, mapId, reduceId, cause.getMessage, cause) } - def toTaskEndReason: TaskEndReason = FetchFailed(bmAddress, shuffleId, mapId, reduceId, + def toTaskFailedReason: TaskFailedReason = FetchFailed(bmAddress, shuffleId, mapId, reduceId, Utils.exceptionString(this)) } 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 c89be22a34c9d..00314abf49fd4 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -146,7 +146,7 @@ class JsonProtocolSuite extends SparkFunSuite { val fetchFailed = FetchFailed(BlockManagerId("With or", "without you", 15), 17, 18, 19, "Some exception") val fetchMetadataFailed = new MetadataFetchFailedException(17, - 19, "metadata Fetch failed exception").toTaskEndReason + 19, "metadata Fetch failed exception").toTaskFailedReason val exceptionFailure = new ExceptionFailure(exception, Seq.empty[AccumulableInfo]) testTaskEndReason(Success) testTaskEndReason(Resubmitted) From 9a2cf846ee8a791c95d391a708799356a7aced1a Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 21 Sep 2016 15:04:37 -0500 Subject: [PATCH 37/57] review feedback --- .../org/apache/spark/TaskEndReason.scala | 3 +- .../spark/scheduler/BlacklistTracker.scala | 127 +++++++++--------- .../spark/scheduler/TaskSchedulerImpl.scala | 2 +- .../spark/scheduler/TaskSetManager.scala | 79 +++++------ .../scheduler/BlacklistTrackerSuite.scala | 32 ++--- .../spark/scheduler/TaskSetManagerSuite.scala | 38 +++--- 6 files changed, 133 insertions(+), 148 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala index 59a25398a039b..7ca3c103dbf5b 100644 --- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala +++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala @@ -98,7 +98,8 @@ case class FetchFailed( * 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. + * stored the data. This is especially important because we 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/scheduler/BlacklistTracker.scala b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala index 9a29a013561b8..1abddd9ee46e0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala @@ -63,8 +63,7 @@ private[scheduler] class BlacklistTracker ( * 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[scheduler] val executorIdToFailureList: HashMap[String, ExecutorFailureList] = - new HashMap() + private val executorIdToFailureList: HashMap[String, ExecutorFailureList] = new HashMap() val executorIdToBlacklistStatus: HashMap[String, BlacklistedExecutor] = new HashMap() val nodeIdToBlacklistExpiryTime: HashMap[String, Long] = new HashMap() /** @@ -76,7 +75,7 @@ private[scheduler] class BlacklistTracker ( * 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. */ - private[scheduler] var nextExpiryTime: Long = Long.MaxValue + 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 @@ -199,6 +198,65 @@ private[scheduler] class BlacklistTracker ( // will clear it up periodically in any case. executorIdToFailureList -= executorId } + + + /** + * Tracks all failures for one executor (that have not passed the timeout). Designed to + * efficiently remove failures that are older than the timeout, and query for the number of unique + * failed tasks. + * 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, sorted by time ascending. + */ + private var failures = ArrayBuffer[(TaskId, Long)]() + + def addFailures( + stage: Int, + stageAttempt: Int, + failuresInTaskSet: ExecutorFailuresInTaskSet): Unit = { + // The new failures may interleave with the old ones, so rebuild the failures in sorted order. + // This shouldn't be expensive because if there were a lot of failures, the executor would + // have been blacklisted. + if (failuresInTaskSet.taskToFailureCountAndExpiryTime.nonEmpty) { + failuresInTaskSet.taskToFailureCountAndExpiryTime.foreach { case (taskIdx, (_, time)) => + failures += ((TaskId(stage, stageAttempt, taskIdx), time)) + } + // sort by failure time, so we can quickly determine if a failure has gone past the timeout + failures = failures.sortBy(_._2) + } + } + + /** + * The number of unique tasks that failed on this executor. Only counts failures within the + * timeout, and in successful tasksets. + */ + def numUniqueTaskFailures: Int = failures.size + + def isEmpty: Boolean = failures.isEmpty + + def dropFailuresWithTimeoutBefore(dropBefore: Long): Unit = { + val minExpiryTime = failures.headOption.map(_._2).getOrElse(Long.MaxValue) + if (minExpiryTime < dropBefore) { + val minIndexToKeep = failures.indexWhere(_._2 >= dropBefore) + if (minIndexToKeep == -1) { + failures.clear() + } else { + failures = failures.drop(minIndexToKeep) + } + } + } + + override def toString(): String = { + s"failures = $failures" + } + } + } @@ -308,10 +366,11 @@ private[scheduler] object BlacklistTracker extends Logging { s"blacklisting with ${config.BLACKLIST_ENABLED.key}") } } - } } +private final case class BlacklistedExecutor(node: String, expiryTime: Long) + /** Failures for one executor, within one taskset */ private[scheduler] final class ExecutorFailuresInTaskSet(val node: String) { /** @@ -328,66 +387,8 @@ private[scheduler] final class ExecutorFailuresInTaskSet(val node: String) { def numUniqueTasksWithFailures: Int = taskToFailureCountAndExpiryTime.size override def toString(): String = { - s"numUniqueTasksWithFailures= $numUniqueTasksWithFailures; " + + s"numUniqueTasksWithFailures = $numUniqueTasksWithFailures; " + s"tasksToFailureCount = $taskToFailureCountAndExpiryTime" } } -/** - * Tracks all failures for one executor (that have not passed the timeout). Designed to efficiently - * remove failures that are older than the timeout, and query for the number of unique failed tasks. - * 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, sorted by time ascending. - */ - private var failures = ArrayBuffer[(TaskId, Long)]() - - def addFailures( - stage: Int, - stageAttempt: Int, - failuresInTaskSet: ExecutorFailuresInTaskSet): Unit = { - // The new failures may interleave with the old ones, so rebuild the failures in sorted order. - // This shouldn't be expensive because if there were a lot of failures, the executor would - // have been blacklisted. - if (failuresInTaskSet.taskToFailureCountAndExpiryTime.nonEmpty) { - failuresInTaskSet.taskToFailureCountAndExpiryTime.foreach { case (taskIdx, (_, time)) => - failures += ((TaskId(stage, stageAttempt, taskIdx), time)) - } - // sort by failure time, so we can quickly determine if any failure has gone past the timeout - failures = failures.sortBy(_._2) - } - } - - def minExpiryTime: Long = failures.headOption.map(_._2).getOrElse(Long.MaxValue) - - /** - * The number of unique tasks that failed on this executor. Only counts failures within the - * timeout, and in successful tasksets. - */ - def numUniqueTaskFailures: Int = failures.size - - def isEmpty: Boolean = failures.isEmpty - - def dropFailuresWithTimeoutBefore(dropBefore: Long): Unit = { - if (minExpiryTime < dropBefore) { - val minIndexToKeep = failures.indexWhere(_._2 >= dropBefore) - if (minIndexToKeep == -1) { - failures.clear() - } else { - failures = failures.drop(minIndexToKeep) - } - } - } - - override def toString(): String = { - s"failures = $failures" - } -} - -private final case class BlacklistedExecutor(node: String, expiryTime: Long) 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 ec1624c2de1d8..3be687778fab4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -220,7 +220,7 @@ private[spark] class TaskSchedulerImpl private[scheduler]( private[scheduler] def createTaskSetManager( taskSet: TaskSet, maxTaskFailures: Int): TaskSetManager = { - new TaskSetManager(this, blacklistTracker, taskSet, maxTaskFailures, new SystemClock) + new TaskSetManager(this, taskSet, maxTaskFailures, blacklistTracker, new SystemClock) } override def cancelTasks(stageId: Int, interruptThread: Boolean): Unit = synchronized { 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 7f796aefcf1f2..e84dff9007388 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -50,19 +50,10 @@ import org.apache.spark.util.{AccumulatorV2, Clock, SystemClock, Utils} */ private[spark] class TaskSetManager( val sched: TaskSchedulerImpl, - val blacklistTracker: Option[BlacklistTracker], val taskSet: TaskSet, val maxTaskFailures: Int, - val clock: Clock) - extends Schedulable with Logging { - - def this( - sched: TaskSchedulerImpl, - taskSet: TaskSet, - maxTaskFailures: Int, - clock: Clock = new SystemClock()) { - this(sched, None, taskSet, maxTaskFailures, clock) - } + val blacklistTracker: Option[BlacklistTracker] = None, + val clock: Clock = new SystemClock()) extends Schedulable with Logging { private val conf = sched.sc.conf private val MAX_TASK_ATTEMPTS_PER_EXECUTOR = conf.get(config.MAX_TASK_ATTEMPTS_PER_EXECUTOR) @@ -613,9 +604,9 @@ private[spark] class TaskSetManager( private[scheduler] def abortIfCompletelyBlacklisted( hostToExecutors: HashMap[String, HashSet[String]]): Unit = { blacklistTracker.foreach { blacklist => - // because this is called in a loop, with multiple resource offers and locality levels, - // we could end up aborting this taskset multiple times without the !isZombie check - if (!isZombie) { + // If no executors have registered yet, don't abort the stage, just wait. We probably + // got here because a task set was added before the executors registered. + if (hostToExecutors.nonEmpty) { // take any task that needs to be scheduled, and see if we can find some executor it *could* // run on val pendingTask: Option[Int] = { @@ -633,35 +624,31 @@ private[spark] class TaskSetManager( } } - // If no executors have registered yet, don't abort the stage, just wait. We probably - // got here because a task set was added before the executors registered. - if (hostToExecutors.nonEmpty) { - pendingTask.foreach { indexInTaskSet => - // try to find some executor this task can run on. Its possible that some *other* - // task isn't schedulable anywhere, but we will discover that in some later call, - // when that unschedulable task is the last task remaining. - val blacklistedEverywhere = hostToExecutors.forall { case (host, execs) => - // Check if the task can run on the node - val nodeBlacklisted = blacklist.isNodeBlacklisted(host) || - isNodeBlacklistedForTaskSet(host) || - isNodeBlacklistedForTask(host, indexInTaskSet) - if (nodeBlacklisted) { - true - } else { - // Check if the task can run on any of the executors - execs.forall { exec => - blacklist.isExecutorBlacklisted(exec) || - isExecutorBlacklistedForTaskSet(exec) || - isExecutorBlacklistedForTask(exec, indexInTaskSet) - } + pendingTask.foreach { indexInTaskSet => + // try to find some executor this task can run on. Its possible that some *other* + // task isn't schedulable anywhere, but we will discover that in some later call, + // when that unschedulable task is the last task remaining. + val blacklistedEverywhere = hostToExecutors.forall { case (host, execs) => + // Check if the task can run on the node + val nodeBlacklisted = blacklist.isNodeBlacklisted(host) || + isNodeBlacklistedForTaskSet(host) || + isNodeBlacklistedForTask(host, indexInTaskSet) + if (nodeBlacklisted) { + true + } else { + // Check if the task can run on any of the executors + execs.forall { exec => + blacklist.isExecutorBlacklisted(exec) || + isExecutorBlacklistedForTaskSet(exec) || + isExecutorBlacklistedForTask(exec, indexInTaskSet) } } - if (blacklistedEverywhere) { - val partition = tasks(indexInTaskSet).partitionId - abort(s"Aborting ${taskSet} because task $indexInTaskSet (partition $partition) " + - s"cannot run anywhere due to node and executor blacklist. Blacklisting behavior " + - s"can be configured via spark.blacklist.*.") - } + } + if (blacklistedEverywhere) { + val partition = tasks(indexInTaskSet).partitionId + abort(s"Aborting ${taskSet} because task $indexInTaskSet (partition $partition) " + + s"cannot run anywhere due to node and executor blacklist. Blacklisting behavior " + + s"can be configured via spark.blacklist.*.") } } } @@ -809,10 +796,6 @@ private[spark] class TaskSetManager( None } - // we might rack up a bunch of fetch-failures in rapid succession, due to a bad node. But - // that bad node will get handled separately by spark's stage-failure handling mechanism. It - // shouldn't penalize *this* executor at all, so don't count it as a task-failure as far as - // the blacklist is concerned. if (reason.countTowardsTaskFailures && blacklistTracker.isDefined) { updateBlacklistForFailedTask(info.host, info.executorId, index) } @@ -854,14 +837,14 @@ private[spark] class TaskSetManager( // over the limit, blacklist it from the entire host val execsWithFailuresOnNode = nodeToExecsWithFailures.getOrElseUpdate(host, new HashSet()) execsWithFailuresOnNode += exec - val failuresOnHost = execsWithFailuresOnNode.toIterator.map { exec => + val failuresOnHost = execsWithFailuresOnNode.toIterator.flatMap { exec => execToFailures.get(exec).map { failures => // We count task attempts here, not the number of unique executors with failures. This is // because jobs are aborted based on the number task attempts; if we counted unique // executors, it would be hard to config to ensure that you try another // node before hitting the max number of task failures. failures.taskToFailureCountAndExpiryTime.getOrElse(index, (0, 0))._1 - }.getOrElse(0) + } }.sum if (failuresOnHost >= MAX_TASK_ATTEMPTS_PER_NODE) { nodeToBlacklistedTasks.getOrElseUpdate(host, new HashSet()) += index @@ -894,7 +877,7 @@ private[spark] class TaskSetManager( index: Int): Boolean = { execToFailures.get(executorId) .map { execFailures => - val count = execFailures.taskToFailureCountAndExpiryTime.getOrElse(index, (0, 0))._1 + val count = execFailures.taskToFailureCountAndExpiryTime.get(index).map(_._1).getOrElse(0) count >= MAX_TASK_ATTEMPTS_PER_EXECUTOR } .getOrElse(false) diff --git a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala index f7314b109fffa..dd65ba845a4b4 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala @@ -39,7 +39,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M super.afterEach() } - val allOptions = (('A' to 'Z').map("host" + _) ++ (1 to 100).map{_.toString}).toSet + val allExecutorAndHostIds= (('A' to 'Z').map("host" + _) ++ (1 to 100).map{_.toString}).toSet /** * Its easier to write our tests as if we could directly look at the sets of nodes & executors in @@ -48,7 +48,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M * sets. */ def assertEquivalentToSet(f: String => Boolean, expected: Set[String]): Unit = { - allOptions.foreach { opt => + allExecutorAndHostIds.foreach { opt => val actual = f(opt) val exp = expected.contains(opt) assert(actual === exp, raw"""for string "$opt" """) @@ -70,7 +70,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M // Task 1 failed on executor 1 blacklistTracker = new BlacklistTracker(conf, clock) val taskSet = FakeTask.createTaskSet(10) - val tsm = new TaskSetManager(scheduler, Some(blacklistTracker), taskSet, 4, clock) + val tsm = new TaskSetManager(scheduler, taskSet, 4, Some(blacklistTracker), clock) tsm.updateBlacklistForFailedTask("hostA", "1", 0) for { executor <- (1 to 4).map(_.toString) @@ -147,7 +147,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M // any particular taskset, but we still blacklist the executor overall eventually (0 until 4).foreach { stage => val taskSet = FakeTask.createTaskSet(1) - val tsm = new TaskSetManager(scheduler, Some(tracker), taskSet, 4, clock) + val tsm = new TaskSetManager(scheduler, taskSet, 4, Some(tracker), clock) tsm.updateBlacklistForFailedTask("hostA", "1", 0) tracker.updateBlacklistForSuccessfulTaskSet(stage, 0, tsm.execToFailures) } @@ -161,7 +161,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M // for 4 different stages, executor 1 fails a task, and then the taskSet fails. (0 until 4).foreach { stage => val taskSet = FakeTask.createTaskSet(1) - val tsm = new TaskSetManager(scheduler, Some(tracker), taskSet, 4, clock) + val tsm = new TaskSetManager(scheduler, taskSet, 4, Some(tracker), clock) tsm.updateBlacklistForFailedTask("hostA", "1", 0) } assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set()) @@ -174,7 +174,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M val (tracker, scheduler) = trackerFixture val stageId = 1 + (if (succeedTaskSet) 1 else 0) val taskSet = FakeTask.createTaskSet(4, stageId, 0) - val tsm = new TaskSetManager(scheduler, Some(tracker), taskSet, 4, clock) + val tsm = new TaskSetManager(scheduler, taskSet, 4, Some(tracker), clock) (0 until 4).foreach { partition => tsm.updateBlacklistForFailedTask("hostA", "1", partition) } @@ -196,7 +196,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M test("blacklisted executors and nodes get recovered with time") { val (tracker, scheduler) = trackerFixture val taskSet0 = FakeTask.createTaskSet(4) - val tsm0 = new TaskSetManager(scheduler, Some(tracker), taskSet0, 4, clock) + val tsm0 = new TaskSetManager(scheduler, taskSet0, 4, Some(tracker), clock) (0 until 4).foreach { partition => tsm0.updateBlacklistForFailedTask("hostA", "1", partition) } @@ -206,7 +206,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set("1")) val taskSet1 = FakeTask.createTaskSet(4, 1, 0) - val tsm1 = new TaskSetManager(scheduler, Some(tracker), taskSet1, 4, clock) + val tsm1 = new TaskSetManager(scheduler, taskSet1, 4, Some(tracker), clock) (0 until 4).foreach { partition => tsm1.updateBlacklistForFailedTask("hostA", "2", partition) } @@ -223,7 +223,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M // fail one more task, but executor isn't put back into blacklist since count reset to 0 val taskSet2 = FakeTask.createTaskSet(4, 2, 0) - val tsm2 = new TaskSetManager(scheduler, Some(tracker), taskSet2, 4, clock) + val tsm2 = new TaskSetManager(scheduler, taskSet2, 4, Some(tracker), clock) tsm2.updateBlacklistForFailedTask("hostA", "1", 0) tracker.updateBlacklistForSuccessfulTaskSet(2, 0, tsm2.execToFailures) assert(tracker.nodeBlacklist() === Set()) @@ -236,7 +236,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M // be able to blacklist the entire node. val (tracker, scheduler) = trackerFixture val taskSet0 = FakeTask.createTaskSet(4) - val tsm0 = new TaskSetManager(scheduler, Some(tracker), taskSet0, 4, clock) + val tsm0 = new TaskSetManager(scheduler, taskSet0, 4, Some(tracker), clock) // Lets say that executor 1 dies completely. We get a task failure for the last task, but // the taskset then finishes successfully (elsewhere). (0 until 4).foreach { partition => @@ -249,7 +249,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M // Now another executor gets spun up on that host, but it also dies. val taskSet1 = FakeTask.createTaskSet(4, 1, 0) - val tsm1 = new TaskSetManager(scheduler, Some(tracker), taskSet1, 4, clock) + val tsm1 = new TaskSetManager(scheduler, taskSet1, 4, Some(tracker), clock) (0 until 4).foreach { partition => tsm1.updateBlacklistForFailedTask("hostA", "2", partition) } @@ -280,7 +280,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M var stageId = 0 def failOneTaskInTaskSet(exec: String): Unit = { val taskSet = FakeTask.createTaskSet(1, stageId, 0) - val tsm = new TaskSetManager(scheduler, Some(tracker), taskSet, 1, clock) + val tsm = new TaskSetManager(scheduler, taskSet, 1, Some(tracker), clock) tsm.updateBlacklistForFailedTask("host-" + exec, exec, 0) tracker.updateBlacklistForSuccessfulTaskSet(stageId, 0, tsm.execToFailures) stageId += 1 @@ -358,7 +358,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M config.MAX_FAILED_EXEC_PER_NODE_STAGE.key -> "3" ) val taskSet = FakeTask.createTaskSet(5) - val tsm = new TaskSetManager(scheduler, Some(tracker), taskSet, 4, clock) + val tsm = new TaskSetManager(scheduler, taskSet, 4, Some(tracker), clock) // fail a task twice on hostA, exec:1 tsm.updateBlacklistForFailedTask("hostA", "1", 0) tsm.updateBlacklistForFailedTask("hostA", "1", 0) @@ -400,7 +400,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M // node blacklisting val (tracker, scheduler) = trackerFixture val taskSet0 = FakeTask.createTaskSet(4) - val tsm0 = new TaskSetManager(scheduler, Some(tracker), taskSet0, 1, clock) + val tsm0 = new TaskSetManager(scheduler, taskSet0, 1, Some(tracker), clock) tsm0.updateBlacklistForFailedTask("hostA", "1", 0) tsm0.updateBlacklistForFailedTask("hostA", "1", 1) tracker.updateBlacklistForSuccessfulTaskSet(0, 0, tsm0.execToFailures) @@ -408,7 +408,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M assertEquivalentToSet(tracker.isNodeBlacklisted(_), Set()) val taskSet1 = FakeTask.createTaskSet(4, 1, 0) - val tsm1 = new TaskSetManager(scheduler, Some(tracker), taskSet1, 1, clock) + val tsm1 = new TaskSetManager(scheduler, taskSet1, 1, Some(tracker), clock) tsm1.updateBlacklistForFailedTask("hostB", "2", 0) tsm1.updateBlacklistForFailedTask("hostB", "2", 1) tracker.updateBlacklistForSuccessfulTaskSet(1, 0, tsm1.execToFailures) @@ -421,7 +421,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M // lead to any node blacklisting val (tracker, scheduler) = trackerFixture val taskSet = FakeTask.createTaskSet(4) - val tsm = new TaskSetManager(scheduler, Some(tracker), taskSet, 1, clock) + val tsm = new TaskSetManager(scheduler, taskSet, 1, Some(tracker), clock) tsm.updateBlacklistForFailedTask("hostA", "1", 0) tsm.updateBlacklistForFailedTask("hostA", "1", 1) assertEquivalentToSet(tsm.isExecutorBlacklistedForTaskSet(_), Set("1")) 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 5db3af77b815d..040bd7e1dcb66 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -183,7 +183,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg sched = new FakeTaskScheduler(sc, ("exec1", "host1")) val taskSet = FakeTask.createTaskSet(1) val clock = new ManualClock - val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) val accumUpdates = taskSet.tasks.head.metrics.internalAccums // Offer a host with NO_PREF as the constraint, @@ -236,7 +236,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg sched = new FakeTaskScheduler(sc, ("execA", "host1"), ("execC", "host2")) val taskSet = FakeTask.createTaskSet(1, Seq(TaskLocation("host1", "execB"))) val clock = new ManualClock - val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) // An executor that is not NODE_LOCAL should be rejected. assert(manager.resourceOffer("execC", "host2", ANY) === None) @@ -257,7 +257,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg Seq() // Last task has no locality prefs ) val clock = new ManualClock - val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) // First offer host1, exec1: first task should be chosen assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0) assert(manager.resourceOffer("exec1", "host1", PROCESS_LOCAL) == None) @@ -286,7 +286,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg Seq() // Last task has no locality prefs ) val clock = new ManualClock - val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) // First offer host1, exec1: first task should be chosen assert(manager.resourceOffer("exec1", "host1", PROCESS_LOCAL).get.index === 0) assert(manager.resourceOffer("exec3", "host2", PROCESS_LOCAL).get.index === 1) @@ -306,7 +306,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg Seq(TaskLocation("host2")) ) val clock = new ManualClock - val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) // First offer host1: first task should be chosen assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0) @@ -344,7 +344,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg Seq(TaskLocation("host3")) ) val clock = new ManualClock - val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) // First offer host1: first task should be chosen assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0) @@ -376,7 +376,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg sched = new FakeTaskScheduler(sc, ("exec1", "host1")) val taskSet = FakeTask.createTaskSet(1) val clock = new ManualClock - val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0) @@ -393,7 +393,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg sched = new FakeTaskScheduler(sc, ("exec1", "host1")) val taskSet = FakeTask.createTaskSet(1) val clock = new ManualClock - val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) // Fail the task MAX_TASK_FAILURES times, and check that the task set is aborted // after the last failure. @@ -428,7 +428,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val clock = new ManualClock val blacklist = new BlacklistTracker(conf, clock) - val manager = new TaskSetManager(sched, Some(blacklist), taskSet, 4, clock) + val manager = new TaskSetManager(sched, taskSet, 4, Some(blacklist), clock = clock) { val offerResult = manager.resourceOffer("exec1", "host1", PROCESS_LOCAL) @@ -518,7 +518,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg Seq(TaskLocation("host2", "execC")), Seq()) val clock = new ManualClock - val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) // Only ANY is valid assert(manager.myLocalityLevels.sameElements(Array(NO_PREF, ANY))) // Add a new executor @@ -549,7 +549,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg Seq(TaskLocation("host1", "execB")), Seq(TaskLocation("host2", "execC")), Seq()) - val manager = new TaskSetManager(sched, taskSet, 1, new ManualClock) + val manager = new TaskSetManager(sched, taskSet, 1, clock = new ManualClock) sched.addExecutor("execA", "host1") manager.executorAdded() sched.addExecutor("execC", "host2") @@ -582,7 +582,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg Seq(TaskLocation("host1", "execA")), Seq(TaskLocation("host1", "execA"))) val clock = new ManualClock - val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) assert(manager.myLocalityLevels.sameElements(Array(PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL, ANY))) // Set allowed locality to ANY @@ -673,7 +673,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg Seq(), Seq(TaskLocation("host3", "execC"))) val clock = new ManualClock - val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) assert(manager.resourceOffer("execA", "host1", PROCESS_LOCAL).get.index === 0) assert(manager.resourceOffer("execA", "host1", NODE_LOCAL) == None) @@ -701,7 +701,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg Seq(), Seq(TaskLocation("host3"))) val clock = new ManualClock - val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) // node-local tasks are scheduled without delay assert(manager.resourceOffer("execA", "host1", NODE_LOCAL).get.index === 0) @@ -723,7 +723,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg Seq(ExecutorCacheTaskLocation("host1", "execA")), Seq(ExecutorCacheTaskLocation("host2", "execB"))) val clock = new ManualClock - val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) // process-local tasks are scheduled first assert(manager.resourceOffer("execA", "host1", NODE_LOCAL).get.index === 2) @@ -743,7 +743,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg Seq(ExecutorCacheTaskLocation("host1", "execA")), Seq(ExecutorCacheTaskLocation("host2", "execB"))) val clock = new ManualClock - val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) // process-local tasks are scheduled first assert(manager.resourceOffer("execA", "host1", PROCESS_LOCAL).get.index === 1) @@ -763,7 +763,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg Seq(TaskLocation("host1", "execA")), Seq(TaskLocation("host2", "execB.1"))) val clock = new ManualClock - val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) // Only ANY is valid assert(manager.myLocalityLevels.sameElements(Array(ANY))) // Add a new executor @@ -797,7 +797,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg Seq(TaskLocation("host2")), Seq(TaskLocation("hdfs_cache_host3"))) val clock = new ManualClock - val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) assert(manager.myLocalityLevels.sameElements(Array(PROCESS_LOCAL, NODE_LOCAL, ANY))) sched.removeExecutor("execA") manager.executorAdded() @@ -826,7 +826,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg sc.conf.set("spark.speculation.multiplier", "0.0") sc.conf.set("spark.speculation.quantile", "0.6") val clock = new ManualClock() - val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) val accumUpdatesByTask: Array[Seq[AccumulatorV2[_, _]]] = taskSet.tasks.map { task => task.metrics.internalAccums } From d0f43c7eda8079a453edc722b664404add56d10f Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 21 Sep 2016 15:05:37 -0500 Subject: [PATCH 38/57] review feedback --- .../spark/scheduler/BlacklistTracker.scala | 6 +- .../scheduler/BlacklistIntegrationSuite.scala | 3 +- .../scheduler/BlacklistTrackerSuite.scala | 420 +++++++++--------- docs/configuration.md | 4 +- 4 files changed, 219 insertions(+), 214 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala index 1abddd9ee46e0..db15fc6b9d641 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala @@ -361,9 +361,9 @@ private[scheduler] object BlacklistTracker extends Logging { throw new IllegalArgumentException(s"${config.MAX_TASK_ATTEMPTS_PER_NODE.key} " + s"( = ${maxNodeAttempts}) was >= spark.task.maxFailures " + s"( = ${maxTaskFailures} ). Though blacklisting is enabled, with this configuration, " + - s"Spark will not be robust to one bad node. Increase " + - s"${config.MAX_TASK_ATTEMPTS_PER_NODE.key} or spark.task.maxFailures, or disable " + - s"blacklisting with ${config.BLACKLIST_ENABLED.key}") + s"Spark will not be robust to one bad node. Decrease " + + s"${config.MAX_TASK_ATTEMPTS_PER_NODE.key}, increase spark.task.maxFailures, or " + + s"disable blacklisting with ${config.BLACKLIST_ENABLED.key}") } } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala index f8596a88bc510..0573200de172c 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala @@ -56,7 +56,7 @@ class BlacklistIntegrationSuite extends SchedulerIntegrationSuite[MultiExecutorM // even with the blacklist turned on, bad configs can lead to job failure. To survive one // bad node, you need to make sure that - // maxTaskFailures > min(spark.blacklist.maxTaskFailuresPerNode, nExecutorsPerHost) + // maxTaskFailures > min(spark.blacklist.task.maxTaskAttemptsPerNode, nExecutorsPerHost) testScheduler( "With blacklist on, job will still fail if there are too many bad executors on bad host", extraConfs = Seq( @@ -84,7 +84,6 @@ class BlacklistIntegrationSuite extends SchedulerIntegrationSuite[MultiExecutorM assertDataStructuresEmpty(noFailure = false) } - testScheduler( "With default settings, job can succeed despite multiple bad executors on node", extraConfs = Seq( diff --git a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala index dd65ba845a4b4..9c25e316e7e8a 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala @@ -30,16 +30,22 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M private val clock = new ManualClock(0) - private var blacklistTracker: BlacklistTracker = _ + private var blacklist: BlacklistTracker = _ + private var scheduler: TaskSchedulerImpl = _ + private var conf: SparkConf = _ override def afterEach(): Unit = { - if (blacklistTracker != null) { - blacklistTracker = null + if (blacklist != null) { + blacklist = null + } + if (scheduler != null) { + scheduler.stop() + scheduler = null } super.afterEach() } - val allExecutorAndHostIds= (('A' to 'Z').map("host" + _) ++ (1 to 100).map{_.toString}).toSet + val allExecutorAndHostIds = (('A' to 'Z').map("host" + _) ++ (1 to 100).map{_.toString}).toSet /** * Its easier to write our tests as if we could directly look at the sets of nodes & executors in @@ -63,15 +69,19 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M scheduler } + def createTaskSetManager(numTasks: Int, stageId: Int = 0): TaskSetManager = { + val taskSet = FakeTask.createTaskSet(numTasks, stageId = stageId, stageAttemptId = 0) + new TaskSetManager(scheduler, taskSet, 4, Some(blacklist), clock) + } + test("Blacklisting individual tasks") { - val conf = new SparkConf().setAppName("test").setMaster("local") + conf = new SparkConf().setAppName("test").setMaster("local") .set(config.BLACKLIST_ENABLED.key, "true") - val scheduler = mockTaskSchedWithConf(conf) + scheduler = mockTaskSchedWithConf(conf) // Task 1 failed on executor 1 - blacklistTracker = new BlacklistTracker(conf, clock) - val taskSet = FakeTask.createTaskSet(10) - val tsm = new TaskSetManager(scheduler, taskSet, 4, Some(blacklistTracker), clock) - tsm.updateBlacklistForFailedTask("hostA", "1", 0) + blacklist = new BlacklistTracker(conf, clock) + val tsm = createTaskSetManager(numTasks = 10) + tsm.updateBlacklistForFailedTask("hostA", exec = "1", index = 0) for { executor <- (1 to 4).map(_.toString) index <- 0 until 10 @@ -79,17 +89,19 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M val exp = (executor == "1" && index == 0) assert(tsm.isExecutorBlacklistedForTask(executor, index) === exp) } - assert(blacklistTracker.nodeBlacklist() === Set()) - assertEquivalentToSet(blacklistTracker.isNodeBlacklisted(_), Set()) + assert(blacklist.nodeBlacklist() === Set()) + assertEquivalentToSet(blacklist.isNodeBlacklisted(_), Set()) assertEquivalentToSet(tsm.isNodeBlacklistedForTaskSet, Set()) assertEquivalentToSet(tsm.isExecutorBlacklistedForTaskSet, Set()) - // Task 1 & 2 failed on both executor 1 & 2, so we blacklist all executors on that host, + // Task 1 & 2 failed on both executor 1 & 2, so we should blacklist all executors on that host, // for all tasks for the stage. Note the api expects multiple checks for each type of - // blacklist -- this actually fits naturally with its use in the scheduler - tsm.updateBlacklistForFailedTask("hostA", "1", 1) - tsm.updateBlacklistForFailedTask("hostA", "2", 0) - tsm.updateBlacklistForFailedTask("hostA", "2", 1) + // blacklist -- this actually fits naturally with its use in the scheduler. + tsm.updateBlacklistForFailedTask("hostA", exec = "1", index = 1) + assertEquivalentToSet(tsm.isNodeBlacklistedForTaskSet, Set()) + tsm.updateBlacklistForFailedTask("hostA", exec = "2", index = 0) + assertEquivalentToSet(tsm.isNodeBlacklistedForTaskSet, Set()) + tsm.updateBlacklistForFailedTask("hostA", exec = "2", index = 1) // we don't explicitly return the executors in hostA here, but that is OK for { executor <- (1 to 4).map(_.toString) @@ -97,250 +109,248 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M } { withClue(s"exec = $executor; index = $index") { val badExec = (executor == "1" || executor == "2") - val badPart = (index == 0 || index == 1) - val taskExp = (badExec && badPart) + val badIndex = (index == 0 || index == 1) assert( - tsm.isExecutorBlacklistedForTask(executor, index) === taskExp) - val executorExp = badExec - assert(tsm.isExecutorBlacklistedForTaskSet(executor) === executorExp) + tsm.isExecutorBlacklistedForTask(executor, index) === (badExec && badIndex)) + assert(tsm.isExecutorBlacklistedForTaskSet(executor) === badExec) } } assertEquivalentToSet(tsm.isNodeBlacklistedForTaskSet, Set("hostA")) // we dont' blacklist the nodes or executors till the stages complete - assert(blacklistTracker.nodeBlacklist() === Set()) - assertEquivalentToSet(blacklistTracker.isNodeBlacklisted(_), Set()) - assertEquivalentToSet(blacklistTracker.isExecutorBlacklisted(_), Set()) + assert(blacklist.nodeBlacklist() === Set()) + assertEquivalentToSet(blacklist.isNodeBlacklisted(_), Set()) + assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set()) // when the stage completes successfully, now there is sufficient evidence we've got // bad executors and node - blacklistTracker.updateBlacklistForSuccessfulTaskSet(0, 0, tsm.execToFailures) - assert(blacklistTracker.nodeBlacklist() === Set("hostA")) - assertEquivalentToSet(blacklistTracker.isNodeBlacklisted(_), Set("hostA")) - assertEquivalentToSet(blacklistTracker.isExecutorBlacklisted(_), Set("1", "2")) - - clock.advance(blacklistTracker.BLACKLIST_TIMEOUT_MILLIS + 1) - blacklistTracker.applyBlacklistTimeout() - assert(blacklistTracker.nodeBlacklist() === Set()) - assertEquivalentToSet(blacklistTracker.isNodeBlacklisted(_), Set()) - assertEquivalentToSet(blacklistTracker.isExecutorBlacklisted(_), Set()) - } - - def trackerFixture: (BlacklistTracker, TaskSchedulerImpl) = { - trackerFixture() + blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, tsm.execToFailures) + assert(blacklist.nodeBlacklist() === Set("hostA")) + assertEquivalentToSet(blacklist.isNodeBlacklisted(_), Set("hostA")) + assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1", "2")) + + clock.advance(blacklist.BLACKLIST_TIMEOUT_MILLIS + 1) + blacklist.applyBlacklistTimeout() + assert(blacklist.nodeBlacklist() === Set()) + assertEquivalentToSet(blacklist.isNodeBlacklisted(_), Set()) + assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set()) } - def trackerFixture(confs: (String, String)*): (BlacklistTracker, TaskSchedulerImpl) = { - val conf = new SparkConf().setAppName("test").setMaster("local") + def configureBlacklistAndScheduler(confs: (String, String)*): Unit = { + conf = new SparkConf().setAppName("test").setMaster("local") .set(config.BLACKLIST_ENABLED.key, "true") confs.foreach { case (k, v) => conf.set(k, v) } - val scheduler = mockTaskSchedWithConf(conf) + scheduler = mockTaskSchedWithConf(conf) clock.setTime(0) - blacklistTracker = new BlacklistTracker(conf, clock) - (blacklistTracker, scheduler) + blacklist = new BlacklistTracker(conf, clock) } test("executors can be blacklisted with only a few failures per stage") { - val (tracker, scheduler) = trackerFixture + configureBlacklistAndScheduler() // for 4 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 - (0 until 4).foreach { stage => - val taskSet = FakeTask.createTaskSet(1) - val tsm = new TaskSetManager(scheduler, taskSet, 4, Some(tracker), clock) - tsm.updateBlacklistForFailedTask("hostA", "1", 0) - tracker.updateBlacklistForSuccessfulTaskSet(stage, 0, tsm.execToFailures) + // 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 do not unblacklist the + // executor. + val failuresTillBlacklisted = conf.get(config.MAX_FAILURES_PER_EXEC) + var failuresSoFar = 0 + (0 until failuresTillBlacklisted * 10).foreach { stage => + val tsm = createTaskSetManager(numTasks = 1, stageId = stage) + if (stage % 2 == 0) { + // fail every other task + tsm.updateBlacklistForFailedTask("hostA", exec = "1", index = 0) + failuresSoFar += 1 + } + blacklist.updateBlacklistForSuccessfulTaskSet(stage, 0, tsm.execToFailures) + assert(failuresSoFar == stage / 2 + 1) + if (failuresSoFar < failuresTillBlacklisted) { + assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set()) + } else { + assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1")) + } } - assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set("1")) } // if an executor has many task failures, but the task set ends up failing, don't count it // against the executor test("executors aren't blacklisted if task sets fail") { - val (tracker, scheduler) = trackerFixture + configureBlacklistAndScheduler() // for 4 different stages, executor 1 fails a task, and then the taskSet fails. (0 until 4).foreach { stage => - val taskSet = FakeTask.createTaskSet(1) - val tsm = new TaskSetManager(scheduler, taskSet, 4, Some(tracker), clock) - tsm.updateBlacklistForFailedTask("hostA", "1", 0) + val tsm = createTaskSetManager(numTasks = 1) + tsm.updateBlacklistForFailedTask("hostA", exec = "1", index = 0) } - assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set()) + assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set()) } Seq(true, false).foreach { succeedTaskSet => test(s"stage blacklist updates correctly on stage completion ($succeedTaskSet)") { // within one taskset, an executor fails a few times, so its blacklisted for the taskset. // but if the taskset fails, we don't blacklist the executor after the stage. - val (tracker, scheduler) = trackerFixture + configureBlacklistAndScheduler() val stageId = 1 + (if (succeedTaskSet) 1 else 0) - val taskSet = FakeTask.createTaskSet(4, stageId, 0) - val tsm = new TaskSetManager(scheduler, taskSet, 4, Some(tracker), clock) - (0 until 4).foreach { partition => - tsm.updateBlacklistForFailedTask("hostA", "1", partition) + val tsm = createTaskSetManager(numTasks = 4, stageId = stageId) + (0 until 4).foreach { index => + tsm.updateBlacklistForFailedTask("hostA", exec = "1", index = index) } assert(tsm.isExecutorBlacklistedForTaskSet("1")) - assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set()) + assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set()) if (succeedTaskSet) { // the task set succeeded elsewhere, so we count those failures against our executor, // and blacklist it across stages - tracker.updateBlacklistForSuccessfulTaskSet(stageId, 0, tsm.execToFailures) - assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set("1")) + blacklist.updateBlacklistForSuccessfulTaskSet(stageId, 0, tsm.execToFailures) + assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1")) } else { // the task set failed, so we don't count these failures against the executor for other // stages - assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set()) + assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set()) } } } test("blacklisted executors and nodes get recovered with time") { - val (tracker, scheduler) = trackerFixture - val taskSet0 = FakeTask.createTaskSet(4) - val tsm0 = new TaskSetManager(scheduler, taskSet0, 4, Some(tracker), clock) + configureBlacklistAndScheduler() + val tsm0 = createTaskSetManager(numTasks = 4, stageId = 0) (0 until 4).foreach { partition => - tsm0.updateBlacklistForFailedTask("hostA", "1", partition) + tsm0.updateBlacklistForFailedTask("hostA", exec = "1", index = partition) } - tracker.updateBlacklistForSuccessfulTaskSet(0, 0, tsm0.execToFailures) - assert(tracker.nodeBlacklist() === Set()) - assertEquivalentToSet(tracker.isNodeBlacklisted(_), Set()) - assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set("1")) + blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, tsm0.execToFailures) + assert(blacklist.nodeBlacklist() === Set()) + assertEquivalentToSet(blacklist.isNodeBlacklisted(_), Set()) + assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1")) - val taskSet1 = FakeTask.createTaskSet(4, 1, 0) - val tsm1 = new TaskSetManager(scheduler, taskSet1, 4, Some(tracker), clock) + val tsm1 = createTaskSetManager(numTasks = 4, stageId = 1) (0 until 4).foreach { partition => - tsm1.updateBlacklistForFailedTask("hostA", "2", partition) + tsm1.updateBlacklistForFailedTask("hostA", exec = "2", index = partition) } - tracker.updateBlacklistForSuccessfulTaskSet(0, 0, tsm1.execToFailures) - assert(tracker.nodeBlacklist() === Set("hostA")) - assertEquivalentToSet(tracker.isNodeBlacklisted(_), Set("hostA")) - assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set("1", "2")) + blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, tsm1.execToFailures) + assert(blacklist.nodeBlacklist() === Set("hostA")) + assertEquivalentToSet(blacklist.isNodeBlacklisted(_), Set("hostA")) + assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1", "2")) - clock.advance(tracker.BLACKLIST_TIMEOUT_MILLIS + 1) - tracker.applyBlacklistTimeout() - assert(tracker.nodeBlacklist() === Set()) - assertEquivalentToSet(tracker.isNodeBlacklisted(_), Set()) - assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set()) + clock.advance(blacklist.BLACKLIST_TIMEOUT_MILLIS + 1) + blacklist.applyBlacklistTimeout() + assert(blacklist.nodeBlacklist() === Set()) + assertEquivalentToSet(blacklist.isNodeBlacklisted(_), Set()) + assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set()) // fail one more task, but executor isn't put back into blacklist since count reset to 0 - val taskSet2 = FakeTask.createTaskSet(4, 2, 0) - val tsm2 = new TaskSetManager(scheduler, taskSet2, 4, Some(tracker), clock) - tsm2.updateBlacklistForFailedTask("hostA", "1", 0) - tracker.updateBlacklistForSuccessfulTaskSet(2, 0, tsm2.execToFailures) - assert(tracker.nodeBlacklist() === Set()) - assertEquivalentToSet(tracker.isNodeBlacklisted(_), Set()) - assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set()) + val tsm2 = createTaskSetManager(numTasks = 4, stageId = 2) + tsm2.updateBlacklistForFailedTask("hostA", exec = "1", index = 0) + blacklist.updateBlacklistForSuccessfulTaskSet(2, 0, tsm2.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 (tracker, scheduler) = trackerFixture - val taskSet0 = FakeTask.createTaskSet(4) - val tsm0 = new TaskSetManager(scheduler, taskSet0, 4, Some(tracker), clock) + configureBlacklistAndScheduler() + val tsm0 = createTaskSetManager(numTasks = 4, stageId = 0) // Lets say that executor 1 dies completely. We get a task failure for the last task, but // the taskset then finishes successfully (elsewhere). (0 until 4).foreach { partition => - tsm0.updateBlacklistForFailedTask("hostA", "1", partition) + tsm0.updateBlacklistForFailedTask("hostA", exec = "1", index = partition) } - tracker.handleRemovedExecutor("1") - tracker.updateBlacklistForSuccessfulTaskSet(0, 0, tsm0.execToFailures) - assert(tracker.isExecutorBlacklisted("1")) - clock.advance(tracker.BLACKLIST_TIMEOUT_MILLIS / 2) + blacklist.handleRemovedExecutor("1") + blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, tsm0.execToFailures) + assert(blacklist.isExecutorBlacklisted("1")) + clock.advance(blacklist.BLACKLIST_TIMEOUT_MILLIS / 2) // Now another executor gets spun up on that host, but it also dies. - val taskSet1 = FakeTask.createTaskSet(4, 1, 0) - val tsm1 = new TaskSetManager(scheduler, taskSet1, 4, Some(tracker), clock) + val tsm1 = createTaskSetManager(numTasks = 4, stageId = 1) (0 until 4).foreach { partition => - tsm1.updateBlacklistForFailedTask("hostA", "2", partition) + tsm1.updateBlacklistForFailedTask("hostA", exec = "2", index = partition) } - tracker.handleRemovedExecutor("2") - tracker.updateBlacklistForSuccessfulTaskSet(1, 0, tsm1.execToFailures) + blacklist.handleRemovedExecutor("2") + blacklist.updateBlacklistForSuccessfulTaskSet(1, 0, tsm1.execToFailures) // We've now had two bad executors on the hostA, so we should blacklist the entire node. - assert(tracker.isExecutorBlacklisted("1")) - assert(tracker.isExecutorBlacklisted("2")) - assert(tracker.isNodeBlacklisted("hostA")) + assert(blacklist.isExecutorBlacklisted("1")) + assert(blacklist.isExecutorBlacklisted("2")) + assert(blacklist.isNodeBlacklisted("hostA")) - clock.advance(tracker.BLACKLIST_TIMEOUT_MILLIS / 2 + 1) - tracker.applyBlacklistTimeout() + clock.advance(blacklist.BLACKLIST_TIMEOUT_MILLIS / 2 + 1) + blacklist.applyBlacklistTimeout() // executor 1 is no longer explicitly blacklisted, since we've gone past its recovery time, // but everything else is still blacklisted. - assert(!tracker.isExecutorBlacklisted("1")) - assert(tracker.isExecutorBlacklisted("2")) - assert(tracker.isNodeBlacklisted("hostA")) + assert(!blacklist.isExecutorBlacklisted("1")) + assert(blacklist.isExecutorBlacklisted("2")) + assert(blacklist.isNodeBlacklisted("hostA")) // make sure we don't leak memory - assert(!tracker.executorIdToBlacklistStatus.contains("1")) - assert(!tracker.nodeToFailedExecs("hostA").contains("1")) - clock.advance(tracker.BLACKLIST_TIMEOUT_MILLIS) - tracker.applyBlacklistTimeout() - assert(!tracker.nodeIdToBlacklistExpiryTime.contains("hostA")) + assert(!blacklist.executorIdToBlacklistStatus.contains("1")) + assert(!blacklist.nodeToFailedExecs("hostA").contains("1")) + clock.advance(blacklist.BLACKLIST_TIMEOUT_MILLIS) + blacklist.applyBlacklistTimeout() + assert(!blacklist.nodeIdToBlacklistExpiryTime.contains("hostA")) } test("task failures expire with time") { - val (tracker, scheduler) = trackerFixture + configureBlacklistAndScheduler() var stageId = 0 def failOneTaskInTaskSet(exec: String): Unit = { - val taskSet = FakeTask.createTaskSet(1, stageId, 0) - val tsm = new TaskSetManager(scheduler, taskSet, 1, Some(tracker), clock) + val tsm = createTaskSetManager(numTasks = 1, stageId = stageId) tsm.updateBlacklistForFailedTask("host-" + exec, exec, 0) - tracker.updateBlacklistForSuccessfulTaskSet(stageId, 0, tsm.execToFailures) + blacklist.updateBlacklistForSuccessfulTaskSet(stageId, 0, tsm.execToFailures) stageId += 1 } - failOneTaskInTaskSet("1") - // We have one sporadic failure on exec 2 -- it doesn't lead to an exec blacklist. - failOneTaskInTaskSet("2") - assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set()) - assert(tracker.nextExpiryTime === Long.MaxValue) + 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(tracker.BLACKLIST_TIMEOUT_MILLIS + 1) + clock.advance(blacklist.BLACKLIST_TIMEOUT_MILLIS + 1) val t0 = clock.getTimeMillis() - tracker.applyBlacklistTimeout() - assert(tracker.nextExpiryTime === Long.MaxValue) - failOneTaskInTaskSet("1") + blacklist.applyBlacklistTimeout() + assert(blacklist.nextExpiryTime === Long.MaxValue) + failOneTaskInTaskSet(exec = "1") - // Because we went past the expiry time, nothing should have been blacklisted. - assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set()) + // 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 + tracker.BLACKLIST_TIMEOUT_MILLIS) + clock.setTime(t0 + blacklist.BLACKLIST_TIMEOUT_MILLIS - 1) val t1 = clock.getTimeMillis() - failOneTaskInTaskSet("1") - tracker.applyBlacklistTimeout() - assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set("1")) - assert(tracker.nextExpiryTime === t1 + tracker.BLACKLIST_TIMEOUT_MILLIS) + failOneTaskInTaskSet(exec = "1") + blacklist.applyBlacklistTimeout() + assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1")) + assert(blacklist.nextExpiryTime === t1 + blacklist.BLACKLIST_TIMEOUT_MILLIS) // Fail a second executor, and go over its expiry as well. - clock.setTime(t1 + tracker.BLACKLIST_TIMEOUT_MILLIS) + clock.setTime(t1 + blacklist.BLACKLIST_TIMEOUT_MILLIS - 1) val t2 = clock.getTimeMillis() - failOneTaskInTaskSet("3") - failOneTaskInTaskSet("3") - tracker.applyBlacklistTimeout() - assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set("1", "3")) - assert(tracker.nextExpiryTime === t1 + tracker.BLACKLIST_TIMEOUT_MILLIS) - - - clock.setTime(t1 + tracker.BLACKLIST_TIMEOUT_MILLIS + 1) - tracker.applyBlacklistTimeout() - assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set("3")) - assert(tracker.nextExpiryTime === t2 + tracker.BLACKLIST_TIMEOUT_MILLIS) + failOneTaskInTaskSet(exec = "3") + failOneTaskInTaskSet(exec = "3") + blacklist.applyBlacklistTimeout() + assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1", "3")) + assert(blacklist.nextExpiryTime === t1 + blacklist.BLACKLIST_TIMEOUT_MILLIS) + clock.setTime(t1 + blacklist.BLACKLIST_TIMEOUT_MILLIS + 1) + blacklist.applyBlacklistTimeout() + assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("3")) + 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 + tracker.BLACKLIST_TIMEOUT_MILLIS) - val t3 = clock.getTimeMillis() - failOneTaskInTaskSet("4") - tracker.applyBlacklistTimeout() - assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set("3")) - assert(tracker.nextExpiryTime === t2 + tracker.BLACKLIST_TIMEOUT_MILLIS) - - clock.setTime(t2 + tracker.BLACKLIST_TIMEOUT_MILLIS + 1) - val t4 = clock.getTimeMillis() - tracker.applyBlacklistTimeout() - assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set()) + 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()) // 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(tracker.nextExpiryTime === Long.MaxValue) + assert(blacklist.nextExpiryTime === Long.MaxValue) } test("multiple attempts for the same task count once") { @@ -351,89 +361,86 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M // But with stage-blacklisting, we want to make sure we're not just counting one bad task // that has failed many times. - val (tracker, scheduler) = trackerFixture( + configureBlacklistAndScheduler( config.MAX_TASK_ATTEMPTS_PER_EXECUTOR.key -> "2", config.MAX_TASK_ATTEMPTS_PER_NODE.key -> "3", config.MAX_FAILURES_PER_EXEC_STAGE.key -> "2", config.MAX_FAILED_EXEC_PER_NODE_STAGE.key -> "3" ) - val taskSet = FakeTask.createTaskSet(5) - val tsm = new TaskSetManager(scheduler, taskSet, 4, Some(tracker), clock) + val tsm = createTaskSetManager(numTasks = 5, stageId = 0) // fail a task twice on hostA, exec:1 - tsm.updateBlacklistForFailedTask("hostA", "1", 0) - tsm.updateBlacklistForFailedTask("hostA", "1", 0) + tsm.updateBlacklistForFailedTask("hostA", exec = "1", index = 0) + tsm.updateBlacklistForFailedTask("hostA", exec = "1", index = 0) assert(tsm.isExecutorBlacklistedForTask("1", 0)) assert(!tsm.isNodeBlacklistedForTask("hostA", 0)) assert(!tsm.isExecutorBlacklistedForTaskSet("1")) assert(!tsm.isNodeBlacklistedForTaskSet("hostA")) // fail the same task once more on hostA, exec:2 - tsm.updateBlacklistForFailedTask("hostA", "2", 0) + tsm.updateBlacklistForFailedTask("hostA", exec = "2", index = 0) assert(tsm.isNodeBlacklistedForTask("hostA", 0)) assert(!tsm.isExecutorBlacklistedForTaskSet("2")) assert(!tsm.isNodeBlacklistedForTaskSet("hostA")) // fail another task on hostA, exec:1. Now that executor has failures on two different tasks, // so its blacklisted - tsm.updateBlacklistForFailedTask("hostA", "1", 1) - tracker.updateBlacklistForSuccessfulTaskSet(0, 0, tsm.execToFailures) + tsm.updateBlacklistForFailedTask("hostA", exec = "1", index = 1) + blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, tsm.execToFailures) assert(tsm.isExecutorBlacklistedForTaskSet("1")) assert(!tsm.isNodeBlacklistedForTaskSet("hostA")) // fail a third task on hostA, exec:2, so that exec is blacklisted for the whole task set - tsm.updateBlacklistForFailedTask("hostA", "2", 2) - tracker.updateBlacklistForSuccessfulTaskSet(0, 0, tsm.execToFailures) + tsm.updateBlacklistForFailedTask("hostA", exec = "2", index = 2) + blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, tsm.execToFailures) assert(tsm.isExecutorBlacklistedForTaskSet("2")) assert(!tsm.isNodeBlacklistedForTaskSet("hostA")) // 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. - tsm.updateBlacklistForFailedTask("hostA", "3", 3) - tsm.updateBlacklistForFailedTask("hostA", "3", 4) - tracker.updateBlacklistForSuccessfulTaskSet(0, 0, tsm.execToFailures) + tsm.updateBlacklistForFailedTask("hostA", exec = "3", index = 3) + tsm.updateBlacklistForFailedTask("hostA", exec = "3", index = 4) + blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, tsm.execToFailures) assert(tsm.isExecutorBlacklistedForTaskSet("3")) assert(tsm.isNodeBlacklistedForTaskSet("hostA")) } - test("only blacklist nodes when all the blacklisted executors are all on same host (app level)") { + test("only blacklist nodes for the application when all the blacklisted executors are all on " + + "same host") { // we blacklist executors on two different hosts -- make sure that doesn't lead to any // node blacklisting - val (tracker, scheduler) = trackerFixture - val taskSet0 = FakeTask.createTaskSet(4) - val tsm0 = new TaskSetManager(scheduler, taskSet0, 1, Some(tracker), clock) - tsm0.updateBlacklistForFailedTask("hostA", "1", 0) - tsm0.updateBlacklistForFailedTask("hostA", "1", 1) - tracker.updateBlacklistForSuccessfulTaskSet(0, 0, tsm0.execToFailures) - assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set("1")) - assertEquivalentToSet(tracker.isNodeBlacklisted(_), Set()) - - val taskSet1 = FakeTask.createTaskSet(4, 1, 0) - val tsm1 = new TaskSetManager(scheduler, taskSet1, 1, Some(tracker), clock) - tsm1.updateBlacklistForFailedTask("hostB", "2", 0) - tsm1.updateBlacklistForFailedTask("hostB", "2", 1) - tracker.updateBlacklistForSuccessfulTaskSet(1, 0, tsm1.execToFailures) - assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set("1", "2")) - assertEquivalentToSet(tracker.isNodeBlacklisted(_), Set()) + configureBlacklistAndScheduler() + val tsm0 = createTaskSetManager(numTasks = 4, stageId = 0) + tsm0.updateBlacklistForFailedTask("hostA", exec = "1", index = 0) + tsm0.updateBlacklistForFailedTask("hostA", exec = "1", index = 1) + blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, tsm0.execToFailures) + assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1")) + assertEquivalentToSet(blacklist.isNodeBlacklisted(_), Set()) + + val tsm1 = createTaskSetManager(numTasks = 4, stageId = 1) + tsm1.updateBlacklistForFailedTask("hostB", exec = "2", index = 0) + tsm1.updateBlacklistForFailedTask("hostB", exec = "2", index = 1) + blacklist.updateBlacklistForSuccessfulTaskSet(1, 0, tsm1.execToFailures) + assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1", "2")) + assertEquivalentToSet(blacklist.isNodeBlacklisted(_), Set()) } - test("only blacklist nodes when all the blacklisted executors are all on same host (tsm level)") { + 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 (tracker, scheduler) = trackerFixture - val taskSet = FakeTask.createTaskSet(4) - val tsm = new TaskSetManager(scheduler, taskSet, 1, Some(tracker), clock) - tsm.updateBlacklistForFailedTask("hostA", "1", 0) - tsm.updateBlacklistForFailedTask("hostA", "1", 1) + configureBlacklistAndScheduler() + val tsm = createTaskSetManager(numTasks = 4, stageId = 0) + tsm.updateBlacklistForFailedTask("hostA", exec = "1", index = 0) + tsm.updateBlacklistForFailedTask("hostA", exec = "1", index = 1) assertEquivalentToSet(tsm.isExecutorBlacklistedForTaskSet(_), Set("1")) assertEquivalentToSet(tsm.isNodeBlacklistedForTaskSet(_), Set()) - tsm.updateBlacklistForFailedTask("hostB", "2", 0) - tsm.updateBlacklistForFailedTask("hostB", "2", 1) + tsm.updateBlacklistForFailedTask("hostB", exec = "2", index = 0) + tsm.updateBlacklistForFailedTask("hostB", exec = "2", index = 1) assertEquivalentToSet(tsm.isExecutorBlacklistedForTaskSet(_), Set("1", "2")) assertEquivalentToSet(tsm.isNodeBlacklistedForTaskSet(_), Set()) } - test("blacklist still respects legacy configs") { val legacyKey = config.BLACKLIST_LEGACY_TIMEOUT_CONF.key @@ -471,18 +478,17 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M val excMsg = intercept[IllegalArgumentException] { BlacklistTracker.validateBlacklistConfs(conf) }.getMessage() - assert(excMsg.contains(s"${config.MAX_TASK_ATTEMPTS_PER_NODE.key} " + + assert(excMsg === s"${config.MAX_TASK_ATTEMPTS_PER_NODE.key} " + s"( = ${maxNodeAttempts}) was >= spark.task.maxFailures " + s"( = ${maxTaskFailures} ). Though blacklisting is enabled, with this configuration, " + - s"Spark will not be robust to one bad node. Increase " + - s"${config.MAX_TASK_ATTEMPTS_PER_NODE.key } or spark.task.maxFailures, or disable " + - s"blacklisting with ${config.BLACKLIST_ENABLED.key}")) + s"Spark will not be robust to one bad node. Decrease " + + s"${config.MAX_TASK_ATTEMPTS_PER_NODE.key }, increase spark.task.maxFailures, or disable " + + s"blacklisting with ${config.BLACKLIST_ENABLED.key}") } conf.remove("spark.task.maxFailures") conf.remove(config.MAX_TASK_ATTEMPTS_PER_NODE) - Seq( config.MAX_TASK_ATTEMPTS_PER_EXECUTOR, config.MAX_TASK_ATTEMPTS_PER_NODE, diff --git a/docs/configuration.md b/docs/configuration.md index 5816fade27f2e..5caa7464d908c 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1272,8 +1272,8 @@ Apart from these, the following properties are also available, and may be useful spark.blacklist.stage.maxFailedExecutorsPerNode 2 - (Experimental) How many different executors are marked as failed for a given stage, before the - entire node is marked as failed for the stage. + (Experimental) How many different executors are marked as blacklisted for a given stage, before + the entire node is marked as failed for the stage. From 18ef5c6874f5f8fb20ffce238dd2d6e7d7849ca9 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 22 Sep 2016 13:59:43 -0500 Subject: [PATCH 39/57] review feedback --- .../scheduler/BlacklistTrackerSuite.scala | 61 ----------------- .../scheduler/ExecutorFailuresInTaskSet.scala | 43 ++++++++++++ .../spark/scheduler/TaskSetManagerSuite.scala | 66 ++++++++++++++++++- 3 files changed, 108 insertions(+), 62 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/scheduler/ExecutorFailuresInTaskSet.scala diff --git a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala index 9c25e316e7e8a..92c92f69527bd 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala @@ -74,67 +74,6 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M new TaskSetManager(scheduler, taskSet, 4, Some(blacklist), clock) } - test("Blacklisting individual tasks") { - conf = new SparkConf().setAppName("test").setMaster("local") - .set(config.BLACKLIST_ENABLED.key, "true") - scheduler = mockTaskSchedWithConf(conf) - // Task 1 failed on executor 1 - blacklist = new BlacklistTracker(conf, clock) - val tsm = createTaskSetManager(numTasks = 10) - tsm.updateBlacklistForFailedTask("hostA", exec = "1", index = 0) - for { - executor <- (1 to 4).map(_.toString) - index <- 0 until 10 - } { - val exp = (executor == "1" && index == 0) - assert(tsm.isExecutorBlacklistedForTask(executor, index) === exp) - } - assert(blacklist.nodeBlacklist() === Set()) - assertEquivalentToSet(blacklist.isNodeBlacklisted(_), Set()) - assertEquivalentToSet(tsm.isNodeBlacklistedForTaskSet, Set()) - assertEquivalentToSet(tsm.isExecutorBlacklistedForTaskSet, Set()) - - // Task 1 & 2 failed on both executor 1 & 2, so we should blacklist all executors on that host, - // for all tasks for the stage. Note the api expects multiple checks for each type of - // blacklist -- this actually fits naturally with its use in the scheduler. - tsm.updateBlacklistForFailedTask("hostA", exec = "1", index = 1) - assertEquivalentToSet(tsm.isNodeBlacklistedForTaskSet, Set()) - tsm.updateBlacklistForFailedTask("hostA", exec = "2", index = 0) - assertEquivalentToSet(tsm.isNodeBlacklistedForTaskSet, Set()) - tsm.updateBlacklistForFailedTask("hostA", exec = "2", index = 1) - // we don't explicitly return the executors in hostA here, but that is OK - for { - executor <- (1 to 4).map(_.toString) - index <- 0 until 10 - } { - withClue(s"exec = $executor; index = $index") { - val badExec = (executor == "1" || executor == "2") - val badIndex = (index == 0 || index == 1) - assert( - tsm.isExecutorBlacklistedForTask(executor, index) === (badExec && badIndex)) - assert(tsm.isExecutorBlacklistedForTaskSet(executor) === badExec) - } - } - assertEquivalentToSet(tsm.isNodeBlacklistedForTaskSet, Set("hostA")) - // we dont' blacklist the nodes or executors till the stages complete - assert(blacklist.nodeBlacklist() === Set()) - assertEquivalentToSet(blacklist.isNodeBlacklisted(_), Set()) - assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set()) - - // when the stage completes successfully, now there is sufficient evidence we've got - // bad executors and node - blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, tsm.execToFailures) - assert(blacklist.nodeBlacklist() === Set("hostA")) - assertEquivalentToSet(blacklist.isNodeBlacklisted(_), Set("hostA")) - assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1", "2")) - - clock.advance(blacklist.BLACKLIST_TIMEOUT_MILLIS + 1) - blacklist.applyBlacklistTimeout() - assert(blacklist.nodeBlacklist() === Set()) - assertEquivalentToSet(blacklist.isNodeBlacklisted(_), Set()) - assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set()) - } - def configureBlacklistAndScheduler(confs: (String, String)*): Unit = { conf = new SparkConf().setAppName("test").setMaster("local") .set(config.BLACKLIST_ENABLED.key, "true") diff --git a/core/src/test/scala/org/apache/spark/scheduler/ExecutorFailuresInTaskSet.scala b/core/src/test/scala/org/apache/spark/scheduler/ExecutorFailuresInTaskSet.scala new file mode 100644 index 0000000000000..0ea38c666d65a --- /dev/null +++ b/core/src/test/scala/org/apache/spark/scheduler/ExecutorFailuresInTaskSet.scala @@ -0,0 +1,43 @@ +/* + * 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 scala.collection.mutable.HashMap + +/** + * Small helper for tracking failed tasks for blacklisting purposes. Info on all failures for one + * task set, within one task set. + */ +private[scheduler] final class ExecutorFailuresInTaskSet(val node: String) { + /** + * Mapping from index of the tasks in the taskset, to the number of times it has failed on this + * executor and the last time it failed. + */ + val taskToFailureCountAndExpiryTime = HashMap[Int, (Int, Long)]() + def updateWithFailure(taskIndex: Int, failureExpiryTime: Long): Unit = { + val (prevFailureCount, prevFailureExpiryTime) = + taskToFailureCountAndExpiryTime.getOrElse(taskIndex, (0, -1L)) + assert(failureExpiryTime >= prevFailureExpiryTime) + taskToFailureCountAndExpiryTime(taskIndex) = (prevFailureCount + 1, failureExpiryTime) + } + def numUniqueTasksWithFailures: Int = taskToFailureCountAndExpiryTime.size + + override def toString(): String = { + s"numUniqueTasksWithFailures = $numUniqueTasksWithFailures; " + + s"tasksToFailureCount = $taskToFailureCountAndExpiryTime" + } +} 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 040bd7e1dcb66..7f05f30d8e347 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -22,7 +22,7 @@ import java.util.Random import scala.collection.mutable import scala.collection.mutable.ArrayBuffer -import org.mockito.Mockito.{mock, verify} +import org.mockito.Mockito.{mock, verify, when} import org.apache.spark._ import org.apache.spark.internal.config @@ -411,6 +411,70 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg } } + test("Blacklisting individual tasks") { + val conf = new SparkConf().setAppName("test").setMaster("local") + .set(config.BLACKLIST_ENABLED.key, "true") + sc = new SparkContext("local", "test") + sched = new FakeTaskScheduler(sc, ("exec1", "hostA"), ("exec2", "hostA")) + val clock = new ManualClock + val blacklist = mock(classOf[BlacklistTracker]) + when(blacklist.BLACKLIST_TIMEOUT_MILLIS).thenReturn(BlacklistTracker.getBlacklistTimeout(conf)) + + val taskSet = FakeTask.createTaskSet(numTasks = 10) + val tsm = new TaskSetManager(sched, taskSet, 4, Some(blacklist), clock) + clock.setTime(0) + tsm.updateBlacklistForFailedTask("hostA", exec = "exec1", index = 0) + for { + executor <- (1 to 4).map(_.toString) + index <- 0 until 10 + } { + val exp = (executor == "exec1" && index == 0) + assert(tsm.isExecutorBlacklistedForTask(executor, index) === exp) + } + assert(!tsm.isExecutorBlacklistedForTaskSet("exec1")) + assert(!tsm.isNodeBlacklistedForTaskSet("hostA")) + + // Task 1 & 2 failed on both executor 1 & 2, so we should blacklist all executors on that host, + // for all tasks for the stage. Note the api expects multiple checks for each type of + // blacklist -- this actually fits naturally with its use in the scheduler. + tsm.updateBlacklistForFailedTask("hostA", exec = "exec1", index = 1) + assert(tsm.isExecutorBlacklistedForTaskSet("exec1")) + assert(!tsm.isNodeBlacklistedForTaskSet("hostA")) + tsm.updateBlacklistForFailedTask("hostA", exec = "exec2", index = 0) + assert(tsm.isExecutorBlacklistedForTaskSet("exec1")) + assert(!tsm.isExecutorBlacklistedForTaskSet("exec2")) + assert(!tsm.isNodeBlacklistedForTaskSet("hostA")) + tsm.updateBlacklistForFailedTask("hostA", exec = "exec2", index = 1) + assert(tsm.isExecutorBlacklistedForTaskSet("exec1")) + assert(tsm.isExecutorBlacklistedForTaskSet("exec2")) + assert(tsm.isNodeBlacklistedForTaskSet("hostA")) + 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( + tsm.isExecutorBlacklistedForTask(executor, index) === (badExec && badIndex)) + assert(tsm.isExecutorBlacklistedForTaskSet(executor) === badExec) + } + } + assert(tsm.isNodeBlacklistedForTaskSet("hostA")) + val execToFailures = tsm.execToFailures + assert(execToFailures.keySet === Set("exec1", "exec2")) + + val expectedExpiryTime = BlacklistTracker.getBlacklistTimeout(conf) + Seq("exec1", "exec2").foreach { exec => + assert( + execToFailures(exec).taskToFailureCountAndExpiryTime === Map( + 0 -> (1, expectedExpiryTime), + 1 -> (1, expectedExpiryTime) + ) + ) + } + } + test("executors should be blacklisted after task failure, in spite of locality preferences") { val rescheduleDelay = 300L val conf = new SparkConf(). From 0c3cebac02eeb194f1792db6235cffd50833a783 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 26 Sep 2016 11:21:44 -0500 Subject: [PATCH 40/57] pull out TaskSetBlacklist helper --- .../spark/scheduler/TaskSetBlacklist.scala | 144 ++++++++++++++++++ .../spark/scheduler/TaskSetManager.scala | 136 +++-------------- .../scheduler/BlacklistTrackerSuite.scala | 143 +++++++++-------- .../scheduler/TaskSchedulerImplSuite.scala | 49 +++--- .../scheduler/TaskSetBlacklistSuite.scala | 84 ++++++++++ .../spark/scheduler/TaskSetManagerSuite.scala | 64 -------- 6 files changed, 350 insertions(+), 270 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/scheduler/TaskSetBlacklist.scala create mode 100644 core/src/test/scala/org/apache/spark/scheduler/TaskSetBlacklistSuite.scala diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetBlacklist.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetBlacklist.scala new file mode 100644 index 0000000000000..a70e4ae52ca8e --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetBlacklist.scala @@ -0,0 +1,144 @@ +/* + * 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 scala.collection.mutable.{HashMap, HashSet} + +import org.apache.spark.SparkConf +import org.apache.spark.internal.config +import org.apache.spark.internal.Logging +import org.apache.spark.util.Clock + +/** + * Handles balcklisting executors and nodes within a taskset. This includes blacklisting specific + * (task, executor) / (task, nodes) pairs, and also completely blacklisting 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 + * about task failures until a taskset completes successfully. + * + * THREADING: As a helper to [[TaskSetManager]], this class is designed to only 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(val conf: SparkConf, val stageId: Int, val clock: Clock) + extends Logging { + + private val MAX_TASK_ATTEMPTS_PER_EXECUTOR = conf.get(config.MAX_TASK_ATTEMPTS_PER_EXECUTOR) + private val MAX_TASK_ATTEMPTS_PER_NODE = conf.get(config.MAX_TASK_ATTEMPTS_PER_NODE) + private val MAX_FAILURES_PER_EXEC_STAGE = conf.get(config.MAX_FAILURES_PER_EXEC_STAGE) + private val MAX_FAILED_EXEC_PER_NODE_STAGE = conf.get(config.MAX_FAILED_EXEC_PER_NODE_STAGE) + private val TIMEOUT_MILLIS = BlacklistTracker.getBlacklistTimeout(conf) + + /** + * 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. + */ + val execToFailures: HashMap[String, ExecutorFailuresInTaskSet] = new HashMap() + + /** + * Map from node to all executors on it with failures. Needed because we want to know about + * executors on a node even after they have died. + */ + private val nodeToExecsWithFailures: HashMap[String, HashSet[String]] = new HashMap() + private val nodeToBlacklistedTasks: HashMap[String, HashSet[Int]] = new HashMap() + private val blacklistedExecs: HashSet[String] = new HashSet() + private val blacklistedNodes: HashSet[String] = new HashSet() + + /** + * 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 + * altogether. 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 = { + execToFailures.get(executorId) + .map { execFailures => + val count = execFailures.taskToFailureCountAndExpiryTime.get(index).map(_._1).getOrElse(0) + count >= MAX_TASK_ATTEMPTS_PER_EXECUTOR + } + .getOrElse(false) + } + + def isNodeBlacklistedForTask( + node: String, + index: Int): Boolean = { + nodeToBlacklistedTasks.get(node) + .map(_.contains(index)) + .getOrElse(false) + } + + /** + * Return true if this executor is blacklisted for the given stage. Completely ignores whether + * the executor is blacklisted overall (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 isNodeBlacklistedForTaskSet(node: String): Boolean = { + blacklistedNodes.contains(node) + } + + private[scheduler] def updateBlacklistForFailedTask( + host: String, + exec: String, + index: Int): Unit = { + val execFailures = execToFailures.getOrElseUpdate(exec, new ExecutorFailuresInTaskSet(host)) + execFailures.updateWithFailure(index, clock.getTimeMillis() + TIMEOUT_MILLIS) + + // check if this task has also failed on other executors on the same host -- if its gone + // over the limit, blacklist it from the entire host + val execsWithFailuresOnNode = nodeToExecsWithFailures.getOrElseUpdate(host, new HashSet()) + execsWithFailuresOnNode += exec + val failuresOnHost = execsWithFailuresOnNode.toIterator.flatMap { exec => + execToFailures.get(exec).map { failures => + // We count task attempts here, not the number of unique executors with failures. This is + // because jobs are aborted based on the number task attempts; if we counted unique + // executors, it would be hard to config to ensure that you try another + // node before hitting the max number of task failures. + failures.taskToFailureCountAndExpiryTime.getOrElse(index, (0, 0))._1 + } + }.sum + if (failuresOnHost >= MAX_TASK_ATTEMPTS_PER_NODE) { + nodeToBlacklistedTasks.getOrElseUpdate(host, new HashSet()) += index + } + + if (execFailures.numUniqueTasksWithFailures >= 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 (blacklistedExecutorsOnNode.size >= MAX_FAILED_EXEC_PER_NODE_STAGE) { + if (blacklistedNodes.add(host)) { + logInfo(s"Blacklisting ${host} for stage $stageId") + } + } + } + } + } + + +} 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 e84dff9007388..4ab62e375906b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -28,7 +28,6 @@ import scala.util.control.NonFatal import org.apache.spark._ import org.apache.spark.internal.Logging -import org.apache.spark.internal.config import org.apache.spark.scheduler.SchedulingMode._ import org.apache.spark.TaskState.TaskState import org.apache.spark.util.{AccumulatorV2, Clock, SystemClock, Utils} @@ -56,10 +55,6 @@ private[spark] class TaskSetManager( val clock: Clock = new SystemClock()) extends Schedulable with Logging { private val conf = sched.sc.conf - private val MAX_TASK_ATTEMPTS_PER_EXECUTOR = conf.get(config.MAX_TASK_ATTEMPTS_PER_EXECUTOR) - private val MAX_TASK_ATTEMPTS_PER_NODE = conf.get(config.MAX_TASK_ATTEMPTS_PER_NODE) - private val MAX_FAILURES_PER_EXEC_STAGE = conf.get(config.MAX_FAILURES_PER_EXEC_STAGE) - private val MAX_FAILED_EXEC_PER_NODE_STAGE = conf.get(config.MAX_FAILED_EXEC_PER_NODE_STAGE) // Quantile of tasks at which to start speculation val SPECULATION_QUANTILE = conf.getDouble("spark.speculation.quantile", 0.75) @@ -77,16 +72,6 @@ private[spark] class TaskSetManager( val copiesRunning = new Array[Int](numTasks) val successful = new Array[Boolean](numTasks) private val numFailures = new Array[Int](numTasks) - val execToFailures: HashMap[String, ExecutorFailuresInTaskSet] = new HashMap() - /** - * Map from node to all executors on it with failures. Needed because we want to know about - * executors on a node even after they have died. - */ - private val nodeToExecsWithFailures: HashMap[String, HashSet[String]] = new HashMap() - private val nodeToBlacklistedTasks: HashMap[String, HashSet[Int]] = new HashMap() - private val blacklistedExecs: HashSet[String] = new HashSet() - private val blacklistedNodes: HashSet[String] = new HashSet() - val taskAttempts = Array.fill[List[TaskInfo]](numTasks)(Nil) var tasksSuccessful = 0 @@ -100,6 +85,10 @@ private[spark] class TaskSetManager( var totalResultSize = 0L var calculatedTasks = 0 + val taskSetBlacklistOpt: Option[TaskSetBlacklist] = { + blacklistTracker.map { _ => new TaskSetBlacklist(conf, stageId, clock) } + } + val runningTasksSet = new HashSet[Long] override def runningTasks: Int = runningTasksSet.size @@ -272,12 +261,10 @@ private[spark] class TaskSetManager( } private def isTaskBlacklistedOnExecOrNode(index: Int, execId: String, host: String): Boolean = { - if (blacklistTracker.isDefined) { - isNodeBlacklistedForTask(host, index) || - isExecutorBlacklistedForTask(execId, index) - } else { - false - } + taskSetBlacklistOpt.map { blacklist => + blacklist.isNodeBlacklistedForTask(host, index) || + blacklist.isExecutorBlacklistedForTask(execId, index) + }.getOrElse(false) } /** @@ -422,9 +409,9 @@ private[spark] class TaskSetManager( maxLocality: TaskLocality.TaskLocality) : Option[TaskDescription] = { - val offerBlacklisted = blacklistTracker.map { _ => - isNodeBlacklistedForTaskSet(host) || - isExecutorBlacklistedForTaskSet(execId) + val offerBlacklisted = taskSetBlacklistOpt.map { blacklist => + blacklist.isNodeBlacklistedForTaskSet(host) || + blacklist.isExecutorBlacklistedForTaskSet(execId) }.getOrElse(false) if (!isZombie && !offerBlacklisted) { val curTime = clock.getTimeMillis() @@ -500,7 +487,7 @@ private[spark] class TaskSetManager( sched.taskSetFinished(this) if (tasksSuccessful == numTasks) { blacklistTracker.foreach(_.updateBlacklistForSuccessfulTaskSet(taskSet.stageId, - taskSet.stageAttemptId, execToFailures)) + taskSet.stageAttemptId, taskSetBlacklistOpt.get.execToFailures)) } } } @@ -603,7 +590,8 @@ private[spark] class TaskSetManager( */ private[scheduler] def abortIfCompletelyBlacklisted( hostToExecutors: HashMap[String, HashSet[String]]): Unit = { - blacklistTracker.foreach { blacklist => + blacklistTracker.foreach { appBlacklist => + val taskSetBlacklist = taskSetBlacklistOpt.get // If no executors have registered yet, don't abort the stage, just wait. We probably // got here because a task set was added before the executors registered. if (hostToExecutors.nonEmpty) { @@ -630,17 +618,17 @@ private[spark] class TaskSetManager( // when that unschedulable task is the last task remaining. val blacklistedEverywhere = hostToExecutors.forall { case (host, execs) => // Check if the task can run on the node - val nodeBlacklisted = blacklist.isNodeBlacklisted(host) || - isNodeBlacklistedForTaskSet(host) || - isNodeBlacklistedForTask(host, indexInTaskSet) + val nodeBlacklisted = appBlacklist.isNodeBlacklisted(host) || + taskSetBlacklist.isNodeBlacklistedForTaskSet(host) || + taskSetBlacklist.isNodeBlacklistedForTask(host, indexInTaskSet) if (nodeBlacklisted) { true } else { // Check if the task can run on any of the executors execs.forall { exec => - blacklist.isExecutorBlacklisted(exec) || - isExecutorBlacklistedForTaskSet(exec) || - isExecutorBlacklistedForTask(exec, indexInTaskSet) + appBlacklist.isExecutorBlacklisted(exec) || + taskSetBlacklist.isExecutorBlacklistedForTaskSet(exec) || + taskSetBlacklist.isExecutorBlacklistedForTask(exec, indexInTaskSet) } } } @@ -796,8 +784,8 @@ private[spark] class TaskSetManager( None } - if (reason.countTowardsTaskFailures && blacklistTracker.isDefined) { - updateBlacklistForFailedTask(info.host, info.executorId, index) + if (reason.countTowardsTaskFailures) { + taskSetBlacklistOpt.foreach(_.updateBlacklistForFailedTask(info.host, info.executorId, index)) } sched.dagScheduler.taskEnded(tasks(index), reason, null, accumUpdates, info) @@ -825,86 +813,6 @@ private[spark] class TaskSetManager( maybeFinishTaskSet() } - private[scheduler] def updateBlacklistForFailedTask( - host: String, - exec: String, - index: Int): Unit = { - val execFailures = execToFailures.getOrElseUpdate(exec, new ExecutorFailuresInTaskSet(host)) - execFailures.updateWithFailure(index, clock.getTimeMillis() + - blacklistTracker.get.BLACKLIST_TIMEOUT_MILLIS) - - // check if this task has also failed on other executors on the same host -- if its gone - // over the limit, blacklist it from the entire host - val execsWithFailuresOnNode = nodeToExecsWithFailures.getOrElseUpdate(host, new HashSet()) - execsWithFailuresOnNode += exec - val failuresOnHost = execsWithFailuresOnNode.toIterator.flatMap { exec => - execToFailures.get(exec).map { failures => - // We count task attempts here, not the number of unique executors with failures. This is - // because jobs are aborted based on the number task attempts; if we counted unique - // executors, it would be hard to config to ensure that you try another - // node before hitting the max number of task failures. - failures.taskToFailureCountAndExpiryTime.getOrElse(index, (0, 0))._1 - } - }.sum - if (failuresOnHost >= MAX_TASK_ATTEMPTS_PER_NODE) { - nodeToBlacklistedTasks.getOrElseUpdate(host, new HashSet()) += index - } - - if (execFailures.numUniqueTasksWithFailures >= 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 (blacklistedExecutorsOnNode.size >= MAX_FAILED_EXEC_PER_NODE_STAGE) { - if (blacklistedNodes.add(host)) { - logInfo(s"Blacklisting ${host} for stage $stageId") - } - } - } - } - } - - /** - * 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 - * altogether. 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 = { - execToFailures.get(executorId) - .map { execFailures => - val count = execFailures.taskToFailureCountAndExpiryTime.get(index).map(_._1).getOrElse(0) - count >= MAX_TASK_ATTEMPTS_PER_EXECUTOR - } - .getOrElse(false) - } - - def isNodeBlacklistedForTask( - node: String, - index: Int): Boolean = { - nodeToBlacklistedTasks.get(node) - .map(_.contains(index)) - .getOrElse(false) - } - - /** - * Return true if this executor is blacklisted for the given stage. Completely ignores whether - * the executor is blacklisted overall (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 isNodeBlacklistedForTaskSet(node: String): Boolean = { - blacklistedNodes.contains(node) - } - def abort(message: String, exception: Option[Throwable] = None): Unit = sched.synchronized { // TODO: Kill running tasks if we were not terminated due to a Mesos error sched.dagScheduler.taskSetFailed(taskSet, message, exception) diff --git a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala index 92c92f69527bd..72e50f2abb50d 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala @@ -69,9 +69,8 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M scheduler } - def createTaskSetManager(numTasks: Int, stageId: Int = 0): TaskSetManager = { - val taskSet = FakeTask.createTaskSet(numTasks, stageId = stageId, stageAttemptId = 0) - new TaskSetManager(scheduler, taskSet, 4, Some(blacklist), clock) + def createTaskSetBlacklist(stageId: Int = 0): TaskSetBlacklist = { + new TaskSetBlacklist(conf, stageId, clock) } def configureBlacklistAndScheduler(confs: (String, String)*): Unit = { @@ -97,13 +96,13 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M val failuresTillBlacklisted = conf.get(config.MAX_FAILURES_PER_EXEC) var failuresSoFar = 0 (0 until failuresTillBlacklisted * 10).foreach { stage => - val tsm = createTaskSetManager(numTasks = 1, stageId = stage) + val taskSetBlacklist = createTaskSetBlacklist(stage) if (stage % 2 == 0) { // fail every other task - tsm.updateBlacklistForFailedTask("hostA", exec = "1", index = 0) + taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "1", index = 0) failuresSoFar += 1 } - blacklist.updateBlacklistForSuccessfulTaskSet(stage, 0, tsm.execToFailures) + blacklist.updateBlacklistForSuccessfulTaskSet(stage, 0, taskSetBlacklist.execToFailures) assert(failuresSoFar == stage / 2 + 1) if (failuresSoFar < failuresTillBlacklisted) { assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set()) @@ -119,8 +118,8 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M configureBlacklistAndScheduler() // for 4 different stages, executor 1 fails a task, and then the taskSet fails. (0 until 4).foreach { stage => - val tsm = createTaskSetManager(numTasks = 1) - tsm.updateBlacklistForFailedTask("hostA", exec = "1", index = 0) + val taskSetBlacklist = createTaskSetBlacklist(stage) + taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "1", index = 0) } assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set()) } @@ -131,16 +130,16 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M // but if the taskset fails, we don't blacklist the executor after the stage. configureBlacklistAndScheduler() val stageId = 1 + (if (succeedTaskSet) 1 else 0) - val tsm = createTaskSetManager(numTasks = 4, stageId = stageId) + val taskSetBlacklist = createTaskSetBlacklist(stageId) (0 until 4).foreach { index => - tsm.updateBlacklistForFailedTask("hostA", exec = "1", index = index) + taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "1", index = index) } - assert(tsm.isExecutorBlacklistedForTaskSet("1")) + assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("1")) assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set()) if (succeedTaskSet) { // the task set succeeded elsewhere, so we count those failures against our executor, // and blacklist it across stages - blacklist.updateBlacklistForSuccessfulTaskSet(stageId, 0, tsm.execToFailures) + blacklist.updateBlacklistForSuccessfulTaskSet(stageId, 0, taskSetBlacklist.execToFailures) assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1")) } else { // the task set failed, so we don't count these failures against the executor for other @@ -152,20 +151,20 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M test("blacklisted executors and nodes get recovered with time") { configureBlacklistAndScheduler() - val tsm0 = createTaskSetManager(numTasks = 4, stageId = 0) + val taskSetBlacklist0 = createTaskSetBlacklist(stageId = 0) (0 until 4).foreach { partition => - tsm0.updateBlacklistForFailedTask("hostA", exec = "1", index = partition) + taskSetBlacklist0.updateBlacklistForFailedTask("hostA", exec = "1", index = partition) } - blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, tsm0.execToFailures) + blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, taskSetBlacklist0.execToFailures) assert(blacklist.nodeBlacklist() === Set()) assertEquivalentToSet(blacklist.isNodeBlacklisted(_), Set()) assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1")) - val tsm1 = createTaskSetManager(numTasks = 4, stageId = 1) + val taskSetBlacklist1 = createTaskSetBlacklist(stageId = 1) (0 until 4).foreach { partition => - tsm1.updateBlacklistForFailedTask("hostA", exec = "2", index = partition) + taskSetBlacklist1.updateBlacklistForFailedTask("hostA", exec = "2", index = partition) } - blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, tsm1.execToFailures) + blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, taskSetBlacklist1.execToFailures) assert(blacklist.nodeBlacklist() === Set("hostA")) assertEquivalentToSet(blacklist.isNodeBlacklisted(_), Set("hostA")) assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1", "2")) @@ -177,9 +176,9 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set()) // fail one more task, but executor isn't put back into blacklist since count reset to 0 - val tsm2 = createTaskSetManager(numTasks = 4, stageId = 2) - tsm2.updateBlacklistForFailedTask("hostA", exec = "1", index = 0) - blacklist.updateBlacklistForSuccessfulTaskSet(2, 0, tsm2.execToFailures) + val taskSetBlacklist2 = createTaskSetBlacklist(stageId = 2) + taskSetBlacklist2.updateBlacklistForFailedTask("hostA", exec = "1", index = 0) + blacklist.updateBlacklistForSuccessfulTaskSet(2, 0, taskSetBlacklist2.execToFailures) assert(blacklist.nodeBlacklist() === Set()) assertEquivalentToSet(blacklist.isNodeBlacklisted(_), Set()) assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set()) @@ -189,24 +188,24 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M // The blacklist should still work if an executor is killed completely. We should still // be able to blacklist the entire node. configureBlacklistAndScheduler() - val tsm0 = createTaskSetManager(numTasks = 4, stageId = 0) + val taskSetBlacklist0 = createTaskSetBlacklist(stageId = 0) // Lets say that executor 1 dies completely. We get a task failure for the last task, but // the taskset then finishes successfully (elsewhere). (0 until 4).foreach { partition => - tsm0.updateBlacklistForFailedTask("hostA", exec = "1", index = partition) + taskSetBlacklist0.updateBlacklistForFailedTask("hostA", exec = "1", index = partition) } blacklist.handleRemovedExecutor("1") - blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, tsm0.execToFailures) + blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, taskSetBlacklist0.execToFailures) assert(blacklist.isExecutorBlacklisted("1")) clock.advance(blacklist.BLACKLIST_TIMEOUT_MILLIS / 2) // Now another executor gets spun up on that host, but it also dies. - val tsm1 = createTaskSetManager(numTasks = 4, stageId = 1) + val taskSetBlacklist1 = createTaskSetBlacklist(stageId = 1) (0 until 4).foreach { partition => - tsm1.updateBlacklistForFailedTask("hostA", exec = "2", index = partition) + taskSetBlacklist1.updateBlacklistForFailedTask("hostA", exec = "2", index = partition) } blacklist.handleRemovedExecutor("2") - blacklist.updateBlacklistForSuccessfulTaskSet(1, 0, tsm1.execToFailures) + blacklist.updateBlacklistForSuccessfulTaskSet(1, 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")) @@ -231,9 +230,9 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M configureBlacklistAndScheduler() var stageId = 0 def failOneTaskInTaskSet(exec: String): Unit = { - val tsm = createTaskSetManager(numTasks = 1, stageId = stageId) - tsm.updateBlacklistForFailedTask("host-" + exec, exec, 0) - blacklist.updateBlacklistForSuccessfulTaskSet(stageId, 0, tsm.execToFailures) + val taskSetBlacklist = createTaskSetBlacklist(stageId = stageId) + taskSetBlacklist.updateBlacklistForFailedTask("host-" + exec, exec, 0) + blacklist.updateBlacklistForSuccessfulTaskSet(stageId, 0, taskSetBlacklist.execToFailures) stageId += 1 } failOneTaskInTaskSet(exec = "1") @@ -306,41 +305,41 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M config.MAX_FAILURES_PER_EXEC_STAGE.key -> "2", config.MAX_FAILED_EXEC_PER_NODE_STAGE.key -> "3" ) - val tsm = createTaskSetManager(numTasks = 5, stageId = 0) + val taskSetBlacklist = createTaskSetBlacklist(stageId = 0) // fail a task twice on hostA, exec:1 - tsm.updateBlacklistForFailedTask("hostA", exec = "1", index = 0) - tsm.updateBlacklistForFailedTask("hostA", exec = "1", index = 0) - assert(tsm.isExecutorBlacklistedForTask("1", 0)) - assert(!tsm.isNodeBlacklistedForTask("hostA", 0)) - assert(!tsm.isExecutorBlacklistedForTaskSet("1")) - assert(!tsm.isNodeBlacklistedForTaskSet("hostA")) + taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "1", index = 0) + taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "1", index = 0) + assert(taskSetBlacklist.isExecutorBlacklistedForTask("1", 0)) + assert(!taskSetBlacklist.isNodeBlacklistedForTask("hostA", 0)) + assert(!taskSetBlacklist.isExecutorBlacklistedForTaskSet("1")) + assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) // fail the same task once more on hostA, exec:2 - tsm.updateBlacklistForFailedTask("hostA", exec = "2", index = 0) - assert(tsm.isNodeBlacklistedForTask("hostA", 0)) - assert(!tsm.isExecutorBlacklistedForTaskSet("2")) - assert(!tsm.isNodeBlacklistedForTaskSet("hostA")) + taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "2", index = 0) + assert(taskSetBlacklist.isNodeBlacklistedForTask("hostA", 0)) + assert(!taskSetBlacklist.isExecutorBlacklistedForTaskSet("2")) + assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) // fail another task on hostA, exec:1. Now that executor has failures on two different tasks, // so its blacklisted - tsm.updateBlacklistForFailedTask("hostA", exec = "1", index = 1) - blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, tsm.execToFailures) - assert(tsm.isExecutorBlacklistedForTaskSet("1")) - assert(!tsm.isNodeBlacklistedForTaskSet("hostA")) + taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "1", index = 1) + blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, taskSetBlacklist.execToFailures) + assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("1")) + assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) // fail a third task on hostA, exec:2, so that exec is blacklisted for the whole task set - tsm.updateBlacklistForFailedTask("hostA", exec = "2", index = 2) - blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, tsm.execToFailures) - assert(tsm.isExecutorBlacklistedForTaskSet("2")) - assert(!tsm.isNodeBlacklistedForTaskSet("hostA")) + taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "2", index = 2) + blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, taskSetBlacklist.execToFailures) + assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("2")) + assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) // 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. - tsm.updateBlacklistForFailedTask("hostA", exec = "3", index = 3) - tsm.updateBlacklistForFailedTask("hostA", exec = "3", index = 4) - blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, tsm.execToFailures) - assert(tsm.isExecutorBlacklistedForTaskSet("3")) - assert(tsm.isNodeBlacklistedForTaskSet("hostA")) + taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "3", index = 3) + taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "3", index = 4) + blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, taskSetBlacklist.execToFailures) + assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("3")) + assert(taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) } test("only blacklist nodes for the application when all the blacklisted executors are all on " + @@ -348,17 +347,17 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M // we blacklist executors on two different hosts -- make sure that doesn't lead to any // node blacklisting configureBlacklistAndScheduler() - val tsm0 = createTaskSetManager(numTasks = 4, stageId = 0) - tsm0.updateBlacklistForFailedTask("hostA", exec = "1", index = 0) - tsm0.updateBlacklistForFailedTask("hostA", exec = "1", index = 1) - blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, tsm0.execToFailures) + val taskSetBlacklist0 = createTaskSetBlacklist(stageId = 0) + taskSetBlacklist0.updateBlacklistForFailedTask("hostA", exec = "1", index = 0) + taskSetBlacklist0.updateBlacklistForFailedTask("hostA", exec = "1", index = 1) + blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, taskSetBlacklist0.execToFailures) assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1")) assertEquivalentToSet(blacklist.isNodeBlacklisted(_), Set()) - val tsm1 = createTaskSetManager(numTasks = 4, stageId = 1) - tsm1.updateBlacklistForFailedTask("hostB", exec = "2", index = 0) - tsm1.updateBlacklistForFailedTask("hostB", exec = "2", index = 1) - blacklist.updateBlacklistForSuccessfulTaskSet(1, 0, tsm1.execToFailures) + val taskSetBlacklist1 = createTaskSetBlacklist(stageId = 1) + taskSetBlacklist1.updateBlacklistForFailedTask("hostB", exec = "2", index = 0) + taskSetBlacklist1.updateBlacklistForFailedTask("hostB", exec = "2", index = 1) + blacklist.updateBlacklistForSuccessfulTaskSet(1, 0, taskSetBlacklist1.execToFailures) assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1", "2")) assertEquivalentToSet(blacklist.isNodeBlacklisted(_), Set()) } @@ -368,16 +367,16 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M // we blacklist executors on two different hosts within one taskSet -- make sure that doesn't // lead to any node blacklisting configureBlacklistAndScheduler() - val tsm = createTaskSetManager(numTasks = 4, stageId = 0) - tsm.updateBlacklistForFailedTask("hostA", exec = "1", index = 0) - tsm.updateBlacklistForFailedTask("hostA", exec = "1", index = 1) - assertEquivalentToSet(tsm.isExecutorBlacklistedForTaskSet(_), Set("1")) - assertEquivalentToSet(tsm.isNodeBlacklistedForTaskSet(_), Set()) - - tsm.updateBlacklistForFailedTask("hostB", exec = "2", index = 0) - tsm.updateBlacklistForFailedTask("hostB", exec = "2", index = 1) - assertEquivalentToSet(tsm.isExecutorBlacklistedForTaskSet(_), Set("1", "2")) - assertEquivalentToSet(tsm.isNodeBlacklistedForTaskSet(_), Set()) + val taskSetBlacklist = createTaskSetBlacklist(stageId = 0) + taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "1", index = 0) + taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "1", index = 1) + assertEquivalentToSet(taskSetBlacklist.isExecutorBlacklistedForTaskSet(_), Set("1")) + assertEquivalentToSet(taskSetBlacklist.isNodeBlacklistedForTaskSet(_), Set()) + + taskSetBlacklist.updateBlacklistForFailedTask("hostB", exec = "2", index = 0) + taskSetBlacklist.updateBlacklistForFailedTask("hostB", exec = "2", index = 1) + assertEquivalentToSet(taskSetBlacklist.isExecutorBlacklistedForTaskSet(_), Set("1", "2")) + assertEquivalentToSet(taskSetBlacklist.isNodeBlacklistedForTaskSet(_), Set()) } test("blacklist still respects legacy configs") { 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 3dff79bf9324c..62ce6d1e22841 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -46,14 +46,16 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B var taskScheduler: TaskSchedulerImpl = null var dagScheduler: DAGScheduler = null - val stageToMockTsm = new HashMap[Int, TaskSetManager]() + val stageToMockTaskSetBlacklist = new HashMap[Int, TaskSetBlacklist]() + val stageToMockTaskSetManager = new HashMap[Int, TaskSetManager]() override def beforeEach(): Unit = { super.beforeEach() failedTaskSet = false failedTaskSetException = None failedTaskSetReason = null - stageToMockTsm.clear() + stageToMockTaskSetBlacklist.clear() + stageToMockTaskSetManager.clear() } override def afterEach(): Unit = { @@ -95,11 +97,14 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B override def createTaskSetManager(taskSet: TaskSet, maxFailures: Int): TaskSetManager = { val tsm = super.createTaskSetManager(taskSet, maxFailures) val tsmSpy = spy(tsm) - stageToMockTsm(taskSet.stageId) = tsmSpy + val taskSetBlacklist = mock[TaskSetBlacklist] + when(tsmSpy.taskSetBlacklistOpt).thenReturn(Some(taskSetBlacklist)) + stageToMockTaskSetManager(taskSet.stageId) = tsmSpy + stageToMockTaskSetBlacklist(taskSet.stageId) = taskSetBlacklist // intentionally bogus, just lets us easily verify val execToFailures = new HashMap[String, ExecutorFailuresInTaskSet]() execToFailures(taskSet.stageId.toString) = new ExecutorFailuresInTaskSet("dummy") - when(tsmSpy.execToFailures).thenReturn(execToFailures) + when(taskSetBlacklist.execToFailures).thenReturn(execToFailures) tsmSpy } } @@ -346,15 +351,17 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B when(blacklist.isNodeBlacklisted(anyString())).thenReturn(false) when(blacklist.isExecutorBlacklisted(anyString())).thenReturn(false) // setup some defaults, then override them with particulars - stageToMockTsm.values.foreach { tsm => - when(tsm.isNodeBlacklistedForTaskSet(anyString())).thenReturn(false) - when(tsm.isExecutorBlacklistedForTaskSet(anyString())).thenReturn(false) - when(tsm.isExecutorBlacklistedForTask(anyString(), anyInt())).thenReturn(false) - when(tsm.isNodeBlacklistedForTask(anyString(), anyInt())).thenReturn(false) + stageToMockTaskSetBlacklist.values.foreach { taskSetBlacklist => + when(taskSetBlacklist.isNodeBlacklistedForTaskSet(anyString())).thenReturn(false) + when(taskSetBlacklist.isExecutorBlacklistedForTaskSet(anyString())).thenReturn(false) + when(taskSetBlacklist.isExecutorBlacklistedForTask(anyString(), anyInt())).thenReturn(false) + when(taskSetBlacklist.isNodeBlacklistedForTask(anyString(), anyInt())).thenReturn(false) } - when(stageToMockTsm(0).isNodeBlacklistedForTaskSet("host1")).thenReturn(true) - when(stageToMockTsm(1).isExecutorBlacklistedForTaskSet("executor3")).thenReturn(true) - when(stageToMockTsm(0).isExecutorBlacklistedForTask("executor0", 0)).thenReturn(true) + when(stageToMockTaskSetBlacklist(0).isNodeBlacklistedForTaskSet("host1")).thenReturn(true) + when(stageToMockTaskSetBlacklist(1).isExecutorBlacklistedForTaskSet("executor3")) + .thenReturn(true) + when(stageToMockTaskSetBlacklist(0).isExecutorBlacklistedForTask("executor0", 0)) + .thenReturn(true) val firstTaskAttempts = taskScheduler.resourceOffers(offers).flatten // these verifications are tricky b/c (a) we reference them multiple times -- also invoked when @@ -365,7 +372,8 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B verify(blacklist, atLeast(1)).isNodeBlacklisted(s"host$hostNum") } (0 to 2).foreach { stageId => - verify(stageToMockTsm(stageId), atLeast(1)).isNodeBlacklistedForTaskSet(anyString()) + verify(stageToMockTaskSetBlacklist(stageId), atLeast(1)) + .isNodeBlacklistedForTaskSet(anyString()) } for { exec <- Seq("executor1", "executor2") @@ -374,13 +382,13 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B // the node blacklist should ensure we never check the task blacklist. This is important // for performance, otherwise we end up changing an O(1) operation into a // O(numPendingTasks) one - verify(stageToMockTsm(0), never).isExecutorBlacklistedForTask(exec, part) + verify(stageToMockTaskSetBlacklist(0), never).isExecutorBlacklistedForTask(exec, part) } // similarly, the executor blacklist for an entire stage should prevent us from ever checking // the blacklist for specific parts in a stage. (0 to 1).foreach { part => - verify(stageToMockTsm(1), never).isExecutorBlacklistedForTask("executor3", part) + verify(stageToMockTaskSetBlacklist(1), never).isExecutorBlacklistedForTask("executor3", part) } // we should schedule all tasks. @@ -439,9 +447,9 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B // the tasksSets complete, so the tracker should be notified verify(blacklist, times(1)).updateBlacklistForSuccessfulTaskSet( - 0, 0, stageToMockTsm(0).execToFailures) + 0, 0, stageToMockTaskSetBlacklist(0).execToFailures) verify(blacklist, times(1)).updateBlacklistForSuccessfulTaskSet( - 1, 0, stageToMockTsm(1).execToFailures) + 1, 0, stageToMockTaskSetBlacklist(1).execToFailures) } test("scheduled tasks obey node and executor blacklists") { @@ -490,7 +498,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B taskScheduler = setupSchedulerWithMockTsm(blacklist) val taskSet = FakeTask.createTaskSet(numTasks = 10, stageId = 0, stageAttemptId = 0) taskScheduler.submitTasks(taskSet) - val tsm = stageToMockTsm(0) + val tsm = stageToMockTaskSetManager(0) // first just submit some offers so the scheduler knows about all the executors taskScheduler.resourceOffers(Seq( @@ -674,7 +682,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B Seq(new WorkerOffer("executor0", "host0", 10))).flatten assert(taskDescs.size === 4) - val tsm = stageToMockTsm(0) + val tsm = stageToMockTaskSetManager(0) taskScheduler.handleFailedTask(tsm, taskDescs(0).taskId, TaskState.FAILED, FetchFailed(BlockManagerId("executor1", "host1", 12345), 0, 0, 0, "ignored")) taskScheduler.handleFailedTask(tsm, taskDescs(1).taskId, TaskState.FAILED, @@ -683,6 +691,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B TaskCommitDenied(0, 2, 0)) taskScheduler.handleFailedTask(tsm, taskDescs(3).taskId, TaskState.KILLED, TaskKilled) - verify(tsm, never()).updateBlacklistForFailedTask(anyString(), anyString(), anyInt()) + verify(stageToMockTaskSetBlacklist(0), never()) + .updateBlacklistForFailedTask(anyString(), anyString(), anyInt()) } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetBlacklistSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetBlacklistSuite.scala new file mode 100644 index 0000000000000..dc909b1d1761e --- /dev/null +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetBlacklistSuite.scala @@ -0,0 +1,84 @@ +/* + * 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.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.internal.config +import org.apache.spark.util.ManualClock + +class TaskSetBlacklistSuite extends SparkFunSuite { + + test("Blacklisting individual tasks") { + val conf = new SparkConf().setAppName("test").setMaster("local") + .set(config.BLACKLIST_ENABLED.key, "true") + val clock = new ManualClock + + val taskSetBlacklist = new TaskSetBlacklist(conf, stageId = 0, clock = clock) + clock.setTime(0) + taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "exec1", index = 0) + for { + executor <- (1 to 4).map(_.toString) + index <- 0 until 10 + } { + val exp = (executor == "exec1" && index == 0) + assert(taskSetBlacklist.isExecutorBlacklistedForTask(executor, index) === exp) + } + assert(!taskSetBlacklist.isExecutorBlacklistedForTaskSet("exec1")) + assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) + + // Task 1 & 2 failed on both executor 1 & 2, so we should blacklist all executors on that host, + // for all tasks for the stage. Note the api expects multiple checks for each type of + // blacklist -- this actually fits naturally with its use in the scheduler. + taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "exec1", index = 1) + assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("exec1")) + assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) + taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "exec2", index = 0) + assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("exec1")) + assert(!taskSetBlacklist.isExecutorBlacklistedForTaskSet("exec2")) + assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) + taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "exec2", index = 1) + assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("exec1")) + assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("exec2")) + assert(taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) + 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( + taskSetBlacklist.isExecutorBlacklistedForTask(executor, index) === (badExec && badIndex)) + assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet(executor) === badExec) + } + } + assert(taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) + val execToFailures = taskSetBlacklist.execToFailures + assert(execToFailures.keySet === Set("exec1", "exec2")) + + val expectedExpiryTime = BlacklistTracker.getBlacklistTimeout(conf) + Seq("exec1", "exec2").foreach { exec => + assert( + execToFailures(exec).taskToFailureCountAndExpiryTime === Map( + 0 -> (1, expectedExpiryTime), + 1 -> (1, expectedExpiryTime) + ) + ) + } + } + +} 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 7f05f30d8e347..b8d28b407ce64 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -411,70 +411,6 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg } } - test("Blacklisting individual tasks") { - val conf = new SparkConf().setAppName("test").setMaster("local") - .set(config.BLACKLIST_ENABLED.key, "true") - sc = new SparkContext("local", "test") - sched = new FakeTaskScheduler(sc, ("exec1", "hostA"), ("exec2", "hostA")) - val clock = new ManualClock - val blacklist = mock(classOf[BlacklistTracker]) - when(blacklist.BLACKLIST_TIMEOUT_MILLIS).thenReturn(BlacklistTracker.getBlacklistTimeout(conf)) - - val taskSet = FakeTask.createTaskSet(numTasks = 10) - val tsm = new TaskSetManager(sched, taskSet, 4, Some(blacklist), clock) - clock.setTime(0) - tsm.updateBlacklistForFailedTask("hostA", exec = "exec1", index = 0) - for { - executor <- (1 to 4).map(_.toString) - index <- 0 until 10 - } { - val exp = (executor == "exec1" && index == 0) - assert(tsm.isExecutorBlacklistedForTask(executor, index) === exp) - } - assert(!tsm.isExecutorBlacklistedForTaskSet("exec1")) - assert(!tsm.isNodeBlacklistedForTaskSet("hostA")) - - // Task 1 & 2 failed on both executor 1 & 2, so we should blacklist all executors on that host, - // for all tasks for the stage. Note the api expects multiple checks for each type of - // blacklist -- this actually fits naturally with its use in the scheduler. - tsm.updateBlacklistForFailedTask("hostA", exec = "exec1", index = 1) - assert(tsm.isExecutorBlacklistedForTaskSet("exec1")) - assert(!tsm.isNodeBlacklistedForTaskSet("hostA")) - tsm.updateBlacklistForFailedTask("hostA", exec = "exec2", index = 0) - assert(tsm.isExecutorBlacklistedForTaskSet("exec1")) - assert(!tsm.isExecutorBlacklistedForTaskSet("exec2")) - assert(!tsm.isNodeBlacklistedForTaskSet("hostA")) - tsm.updateBlacklistForFailedTask("hostA", exec = "exec2", index = 1) - assert(tsm.isExecutorBlacklistedForTaskSet("exec1")) - assert(tsm.isExecutorBlacklistedForTaskSet("exec2")) - assert(tsm.isNodeBlacklistedForTaskSet("hostA")) - 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( - tsm.isExecutorBlacklistedForTask(executor, index) === (badExec && badIndex)) - assert(tsm.isExecutorBlacklistedForTaskSet(executor) === badExec) - } - } - assert(tsm.isNodeBlacklistedForTaskSet("hostA")) - val execToFailures = tsm.execToFailures - assert(execToFailures.keySet === Set("exec1", "exec2")) - - val expectedExpiryTime = BlacklistTracker.getBlacklistTimeout(conf) - Seq("exec1", "exec2").foreach { exec => - assert( - execToFailures(exec).taskToFailureCountAndExpiryTime === Map( - 0 -> (1, expectedExpiryTime), - 1 -> (1, expectedExpiryTime) - ) - ) - } - } - test("executors should be blacklisted after task failure, in spite of locality preferences") { val rescheduleDelay = 300L val conf = new SparkConf(). From 2381b2526cc2654258a85d496d9416ded06eeb20 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 26 Sep 2016 11:34:28 -0500 Subject: [PATCH 41/57] oops, put class in the right place --- .../spark/scheduler/BlacklistTracker.scala | 22 ------------------- .../scheduler/ExecutorFailuresInTaskSet.scala | 4 +++- 2 files changed, 3 insertions(+), 23 deletions(-) rename core/src/{test => main}/scala/org/apache/spark/scheduler/ExecutorFailuresInTaskSet.scala (95%) diff --git a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala index db15fc6b9d641..0661c161616d5 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala @@ -370,25 +370,3 @@ private[scheduler] object BlacklistTracker extends Logging { } private final case class BlacklistedExecutor(node: String, expiryTime: Long) - -/** Failures for one executor, within one taskset */ -private[scheduler] final class ExecutorFailuresInTaskSet(val node: String) { - /** - * Mapping from index of the tasks in the taskset, to the number of times it has failed on this - * executor and the last time it failed. - */ - val taskToFailureCountAndExpiryTime = HashMap[Int, (Int, Long)]() - def updateWithFailure(taskIndex: Int, failureExpiryTime: Long): Unit = { - val (prevFailureCount, prevFailureExpiryTime) = - taskToFailureCountAndExpiryTime.getOrElse(taskIndex, (0, -1L)) - assert(failureExpiryTime >= prevFailureExpiryTime) - taskToFailureCountAndExpiryTime(taskIndex) = (prevFailureCount + 1, failureExpiryTime) - } - def numUniqueTasksWithFailures: Int = taskToFailureCountAndExpiryTime.size - - override def toString(): String = { - s"numUniqueTasksWithFailures = $numUniqueTasksWithFailures; " + - s"tasksToFailureCount = $taskToFailureCountAndExpiryTime" - } -} - diff --git a/core/src/test/scala/org/apache/spark/scheduler/ExecutorFailuresInTaskSet.scala b/core/src/main/scala/org/apache/spark/scheduler/ExecutorFailuresInTaskSet.scala similarity index 95% rename from core/src/test/scala/org/apache/spark/scheduler/ExecutorFailuresInTaskSet.scala rename to core/src/main/scala/org/apache/spark/scheduler/ExecutorFailuresInTaskSet.scala index 0ea38c666d65a..3659d028cc3bf 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ExecutorFailuresInTaskSet.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ExecutorFailuresInTaskSet.scala @@ -22,18 +22,20 @@ import scala.collection.mutable.HashMap * Small helper for tracking failed tasks for blacklisting purposes. Info on all failures for one * task set, within one task set. */ -private[scheduler] final class ExecutorFailuresInTaskSet(val node: String) { +class ExecutorFailuresInTaskSet(val node: String) { /** * Mapping from index of the tasks in the taskset, to the number of times it has failed on this * executor and the last time it failed. */ val taskToFailureCountAndExpiryTime = HashMap[Int, (Int, Long)]() + def updateWithFailure(taskIndex: Int, failureExpiryTime: Long): Unit = { val (prevFailureCount, prevFailureExpiryTime) = taskToFailureCountAndExpiryTime.getOrElse(taskIndex, (0, -1L)) assert(failureExpiryTime >= prevFailureExpiryTime) taskToFailureCountAndExpiryTime(taskIndex) = (prevFailureCount + 1, failureExpiryTime) } + def numUniqueTasksWithFailures: Int = taskToFailureCountAndExpiryTime.size override def toString(): String = { From 3ca2f799049284963926a73577be3cb4fe67fb94 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 26 Sep 2016 12:35:12 -0500 Subject: [PATCH 42/57] more refactor for TaskSetBlacklist --- .../scheduler/BlacklistTrackerSuite.scala | 68 ------------------ .../scheduler/TaskSetBlacklistSuite.scala | 69 ++++++++++++++++++- 2 files changed, 68 insertions(+), 69 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala index 72e50f2abb50d..745bffa1b252a 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala @@ -291,57 +291,6 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M assert(blacklist.nextExpiryTime === Long.MaxValue) } - 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 b/c 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. - - configureBlacklistAndScheduler( - config.MAX_TASK_ATTEMPTS_PER_EXECUTOR.key -> "2", - config.MAX_TASK_ATTEMPTS_PER_NODE.key -> "3", - config.MAX_FAILURES_PER_EXEC_STAGE.key -> "2", - config.MAX_FAILED_EXEC_PER_NODE_STAGE.key -> "3" - ) - val taskSetBlacklist = createTaskSetBlacklist(stageId = 0) - // fail a task twice on hostA, exec:1 - taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "1", index = 0) - taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "1", index = 0) - assert(taskSetBlacklist.isExecutorBlacklistedForTask("1", 0)) - assert(!taskSetBlacklist.isNodeBlacklistedForTask("hostA", 0)) - assert(!taskSetBlacklist.isExecutorBlacklistedForTaskSet("1")) - assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) - - // fail the same task once more on hostA, exec:2 - taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "2", index = 0) - assert(taskSetBlacklist.isNodeBlacklistedForTask("hostA", 0)) - assert(!taskSetBlacklist.isExecutorBlacklistedForTaskSet("2")) - assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) - - // fail another task on hostA, exec:1. Now that executor has failures on two different tasks, - // so its blacklisted - taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "1", index = 1) - blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, taskSetBlacklist.execToFailures) - assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("1")) - assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) - - // fail a third task on hostA, exec:2, so that exec is blacklisted for the whole task set - taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "2", index = 2) - blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, taskSetBlacklist.execToFailures) - assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("2")) - assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) - - // 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. - taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "3", index = 3) - taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "3", index = 4) - blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, taskSetBlacklist.execToFailures) - assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("3")) - assert(taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) - } - test("only blacklist nodes for the application when all the blacklisted executors are all on " + "same host") { // we blacklist executors on two different hosts -- make sure that doesn't lead to any @@ -362,23 +311,6 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M assertEquivalentToSet(blacklist.isNodeBlacklisted(_), Set()) } - 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 - configureBlacklistAndScheduler() - val taskSetBlacklist = createTaskSetBlacklist(stageId = 0) - taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "1", index = 0) - taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "1", index = 1) - assertEquivalentToSet(taskSetBlacklist.isExecutorBlacklistedForTaskSet(_), Set("1")) - assertEquivalentToSet(taskSetBlacklist.isNodeBlacklistedForTaskSet(_), Set()) - - taskSetBlacklist.updateBlacklistForFailedTask("hostB", exec = "2", index = 0) - taskSetBlacklist.updateBlacklistForFailedTask("hostB", exec = "2", index = 1) - assertEquivalentToSet(taskSetBlacklist.isExecutorBlacklistedForTaskSet(_), Set("1", "2")) - assertEquivalentToSet(taskSetBlacklist.isNodeBlacklistedForTaskSet(_), Set()) - } - test("blacklist still respects legacy configs") { val legacyKey = config.BLACKLIST_LEGACY_TIMEOUT_CONF.key diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetBlacklistSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetBlacklistSuite.scala index dc909b1d1761e..e651846d399ad 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetBlacklistSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetBlacklistSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.scheduler import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.internal.config -import org.apache.spark.util.ManualClock +import org.apache.spark.util.{ManualClock, SystemClock} class TaskSetBlacklistSuite extends SparkFunSuite { @@ -81,4 +81,71 @@ class TaskSetBlacklistSuite extends SparkFunSuite { } } + 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 b/c 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 taskSetBlacklist = new TaskSetBlacklist(conf, stageId = 0, new SystemClock()) + // fail a task twice on hostA, exec:1 + taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "1", index = 0) + taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "1", index = 0) + assert(taskSetBlacklist.isExecutorBlacklistedForTask("1", 0)) + assert(!taskSetBlacklist.isNodeBlacklistedForTask("hostA", 0)) + assert(!taskSetBlacklist.isExecutorBlacklistedForTaskSet("1")) + assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) + + // fail the same task once more on hostA, exec:2 + taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "2", index = 0) + assert(taskSetBlacklist.isNodeBlacklistedForTask("hostA", 0)) + assert(!taskSetBlacklist.isExecutorBlacklistedForTaskSet("2")) + assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) + + // fail another task on hostA, exec:1. Now that executor has failures on two different tasks, + // so its blacklisted + taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "1", index = 1) + assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("1")) + assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) + + // fail a third task on hostA, exec:2, so that exec is blacklisted for the whole task set + taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "2", index = 2) + assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("2")) + assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) + + // 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. + taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "3", index = 3) + taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "3", index = 4) + assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("3")) + assert(taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) + } + + 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 taskSetBlacklist = new TaskSetBlacklist(conf, stageId = 0, new SystemClock()) + taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "1", index = 0) + taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "1", index = 1) + assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("1")) + assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) + + taskSetBlacklist.updateBlacklistForFailedTask("hostB", exec = "2", index = 0) + taskSetBlacklist.updateBlacklistForFailedTask("hostB", exec = "2", index = 1) + assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("1")) + assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("2")) + assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) + assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostB")) + } + } From 27b4bde699645c13e38ac2a819929aa8a4f0125c Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 26 Sep 2016 15:03:30 -0500 Subject: [PATCH 43/57] fix logging --- .../scala/org/apache/spark/scheduler/TaskSetManager.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) 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 4ab62e375906b..507bff18ac103 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -471,7 +471,7 @@ private[spark] class TaskSetManager( // val timeTaken = clock.getTime() - startTime val taskName = s"task ${info.id} in stage ${taskSet.id}" logInfo(s"Starting $taskName (TID $taskId, $host, executor ${info.executorId}, " + - s"partition ${task.partitionId},$taskLocality, ${serializedTask.limit} bytes)") + s"partition ${task.partitionId}, $taskLocality, ${serializedTask.limit} bytes)") sched.dagScheduler.taskStarted(task, info) return Some(new TaskDescription(taskId = taskId, attemptNumber = attemptNum, execId, @@ -695,9 +695,9 @@ private[spark] class TaskSetManager( } if (!successful(index)) { tasksSuccessful += 1 - logInfo("Finished task %s in stage %s (TID %d) in %d ms on %s (executor %s) (%d/%d)".format( - info.id, taskSet.id, info.taskId, info.duration, info.host, info.executorId, - tasksSuccessful, numTasks)) + logInfo(s"Finished task ${info.id} in stage ${taskSet.id} (TID ${info.taskId}) in" + + s" ${info.duration} ms on ${info.host} (executor ${info.executorId})" + + s" ($tasksSuccessful/$numTasks)") // Mark successful and stop if all the tasks have succeeded. successful(index) = true if (tasksSuccessful == numTasks) { From 278fff343eaa1b917f17d7591e39b0543538d253 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 26 Sep 2016 15:25:40 -0500 Subject: [PATCH 44/57] undo some un-intentional changes --- .../scala/org/apache/spark/scheduler/TaskSetBlacklist.scala | 2 -- .../apache/spark/scheduler/SchedulerIntegrationSuite.scala | 1 - .../org/apache/spark/scheduler/TaskSetManagerSuite.scala | 4 +--- 3 files changed, 1 insertion(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetBlacklist.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetBlacklist.scala index a70e4ae52ca8e..e4845a7c7f61d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetBlacklist.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetBlacklist.scala @@ -139,6 +139,4 @@ private[scheduler] class TaskSetBlacklist(val conf: SparkConf, val stageId: Int, } } } - - } diff --git a/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala index 9f7184a75042e..0f90544e45e7a 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala @@ -648,7 +648,6 @@ class BasicSchedulerIntegrationSuite extends SchedulerIntegrationSuite[SingleCor assertDataStructuresEmpty(noFailure = false) } - def testNoBlacklist(name: String)(body: => Unit): Unit = { // in these simple tests, we only have one executor, so it doens't make sense to turn on the // blacklist. Just an artifact of this simple test-framework still kinda acting like local-mode 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 b8d28b407ce64..4d175f961bb1c 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -114,9 +114,7 @@ class FakeTaskScheduler(sc: SparkContext, liveExecutors: (String, String)* /* ex } } - override def taskSetFinished(manager: TaskSetManager): Unit = { - finishedManagers += manager - } + override def taskSetFinished(manager: TaskSetManager): Unit = finishedManagers += manager override def isExecutorAlive(execId: String): Boolean = executors.contains(execId) From 882b385c966112c0345fce7fe92e3a0aa31ed22d Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 26 Sep 2016 11:56:21 -0500 Subject: [PATCH 45/57] remove app level blacklisting (wip) --- .../spark/internal/config/package.scala | 10 - .../spark/scheduler/BlacklistTracker.scala | 248 +------------- .../spark/scheduler/TaskSchedulerImpl.scala | 61 +--- .../spark/scheduler/TaskSetBlacklist.scala | 14 +- .../spark/scheduler/TaskSetManager.scala | 17 +- .../cluster/CoarseGrainedClusterMessage.scala | 3 +- .../apache/spark/HeartbeatReceiverSuite.scala | 4 +- .../scheduler/BlacklistTrackerSuite.scala | 295 +---------------- .../org/apache/spark/scheduler/FakeTask.scala | 10 +- .../scheduler/TaskSchedulerImplSuite.scala | 302 +----------------- .../spark/scheduler/TaskSetManagerSuite.scala | 47 ++- docs/configuration.md | 22 -- .../spark/deploy/yarn/ApplicationMaster.scala | 6 +- .../spark/deploy/yarn/YarnAllocator.scala | 21 +- .../cluster/YarnSchedulerBackend.scala | 5 +- .../deploy/yarn/YarnAllocatorSuite.scala | 12 +- 16 files changed, 70 insertions(+), 1007 deletions(-) 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 eb4542ad75cfc..616a4998e5b6b 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 @@ -109,21 +109,11 @@ package object config { .intConf .createWithDefault(2) - private[spark] val MAX_FAILURES_PER_EXEC = - ConfigBuilder("spark.blacklist.application.maxFailedTasksPerExecutor") - .intConf - .createWithDefault(2) - private[spark] val MAX_FAILURES_PER_EXEC_STAGE = ConfigBuilder("spark.blacklist.stage.maxFailedTasksPerExecutor") .intConf .createWithDefault(2) - private[spark] val MAX_FAILED_EXEC_PER_NODE = - ConfigBuilder("spark.blacklist.application.maxFailedExecutorsPerNode") - .intConf - .createWithDefault(2) - private[spark] val MAX_FAILED_EXEC_PER_NODE_STAGE = ConfigBuilder("spark.blacklist.stage.maxFailedExecutorsPerNode") .intConf diff --git a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala index 0661c161616d5..362a032db27af 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala @@ -17,248 +17,10 @@ package org.apache.spark.scheduler -import java.util.concurrent.atomic.AtomicReference - -import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} - import org.apache.spark.SparkConf -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 ( - conf: SparkConf, - clock: Clock = new SystemClock()) extends Logging { - - 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) - - /** - * 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: HashMap[String, ExecutorFailureList] = new HashMap() - val executorIdToBlacklistStatus: HashMap[String, BlacklistedExecutor] = new HashMap() - val nodeIdToBlacklistExpiryTime: HashMap[String, Long] = new HashMap() - /** - * An immutable copy of the set of nodes that are currently blacklisted. Kept in an - * AtomicReference to make [[nodeBlacklist()]] thread-safe. - */ - private val _nodeBlacklist: AtomicReference[Set[String]] = new AtomicReference(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 periodically clean up the list of blacklisted executors. - */ - val nodeToFailedExecs: HashMap[String, HashSet[String]] = new HashMap() - - 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"has timed out") - execsToUnblacklist.foreach { exec => - val status = executorIdToBlacklistStatus.remove(exec).get - val failedExecsOnNode = nodeToFailedExecs(status.node) - failedExecsOnNode.remove(exec) - if (failedExecsOnNode.isEmpty) { - nodeToFailedExecs.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) } - _nodeBlacklist.set(nodeIdToBlacklistExpiryTime.keySet.toSet) - } - updateNextExpiryTime() - } - } - - private def updateNextExpiryTime(): Unit = { - if (executorIdToBlacklistStatus.nonEmpty) { - nextExpiryTime = executorIdToBlacklistStatus.map{_._2.expiryTime}.min - } else { - nextExpiryTime = Long.MaxValue - } - } - - - 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() - val expiryTime = now + BLACKLIST_TIMEOUT_MILLIS - failuresByExec.foreach { case (exec, failuresInTaskSet) => - val allFailuresOnOneExecutor = - executorIdToFailureList.getOrElseUpdate(exec, new ExecutorFailureList) - // 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. - allFailuresOnOneExecutor.dropFailuresWithTimeoutBefore(now) - allFailuresOnOneExecutor.addFailures(stageId, stageAttemptId, failuresInTaskSet) - val newTotal = allFailuresOnOneExecutor.numUniqueTaskFailures - - if (newTotal >= MAX_FAILURES_PER_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, expiryTime)) - executorIdToFailureList.remove(exec) - updateNextExpiryTime() - - // 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 = nodeToFailedExecs.getOrElseUpdate(node, HashSet[String]()) - blacklistedExecsOnNode += exec - if (blacklistedExecsOnNode.size >= MAX_FAILED_EXEC_PER_NODE) { - logInfo(s"Blacklisting node $node because it has ${blacklistedExecsOnNode.size} " + - s"executors blacklisted: ${blacklistedExecsOnNode}") - nodeIdToBlacklistExpiryTime.put(node, expiryTime) - _nodeBlacklist.set(nodeIdToBlacklistExpiryTime.keySet.toSet) - } - } - } - } - - 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 nodeToFailedExecs, - // 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). Designed to - * efficiently remove failures that are older than the timeout, and query for the number of unique - * failed tasks. - * 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, sorted by time ascending. - */ - private var failures = ArrayBuffer[(TaskId, Long)]() - - def addFailures( - stage: Int, - stageAttempt: Int, - failuresInTaskSet: ExecutorFailuresInTaskSet): Unit = { - // The new failures may interleave with the old ones, so rebuild the failures in sorted order. - // This shouldn't be expensive because if there were a lot of failures, the executor would - // have been blacklisted. - if (failuresInTaskSet.taskToFailureCountAndExpiryTime.nonEmpty) { - failuresInTaskSet.taskToFailureCountAndExpiryTime.foreach { case (taskIdx, (_, time)) => - failures += ((TaskId(stage, stageAttempt, taskIdx), time)) - } - // sort by failure time, so we can quickly determine if a failure has gone past the timeout - failures = failures.sortBy(_._2) - } - } - - /** - * The number of unique tasks that failed on this executor. Only counts failures within the - * timeout, and in successful tasksets. - */ - def numUniqueTaskFailures: Int = failures.size - - def isEmpty: Boolean = failures.isEmpty - - def dropFailuresWithTimeoutBefore(dropBefore: Long): Unit = { - val minExpiryTime = failures.headOption.map(_._2).getOrElse(Long.MaxValue) - if (minExpiryTime < dropBefore) { - val minIndexToKeep = failures.indexWhere(_._2 >= dropBefore) - if (minIndexToKeep == -1) { - failures.clear() - } else { - failures = failures.drop(minIndexToKeep) - } - } - } - - override def toString(): String = { - s"failures = $failures" - } - } - -} - +import org.apache.spark.internal.Logging +import org.apache.spark.util.Utils private[scheduler] object BlacklistTracker extends Logging { @@ -333,9 +95,7 @@ private[scheduler] object BlacklistTracker extends Logging { 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.MAX_FAILED_EXEC_PER_NODE_STAGE ).foreach { config => val v = conf.get(config) if (v <= 0) { @@ -368,5 +128,3 @@ private[scheduler] object BlacklistTracker extends Logging { } } } - -private final case class BlacklistedExecutor(node: String, expiryTime: Long) 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 3be687778fab4..1bc3bbb24e41c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -50,27 +50,13 @@ import org.apache.spark.util.{AccumulatorV2, SystemClock, ThreadUtils, Utils} * acquire a lock on us, so we need to make sure that we don't try to lock the backend while * we are holding a lock on ourselves. */ -private[spark] class TaskSchedulerImpl private[scheduler]( +private[spark] class TaskSchedulerImpl( val sc: SparkContext, val maxTaskFailures: Int, - private val blacklistTracker: Option[BlacklistTracker], isLocal: Boolean = false) extends TaskScheduler with Logging { - def this(sc: SparkContext) = { - this( - sc, - sc.conf.getInt("spark.task.maxFailures", 4), - TaskSchedulerImpl.createBlacklistTracker(sc.conf)) - } - - def this(sc: SparkContext, maxTaskFailures: Int, isLocal: Boolean) = { - this( - sc, - maxTaskFailures, - TaskSchedulerImpl.createBlacklistTracker(sc.conf), - isLocal = isLocal) - } + def this(sc: SparkContext) = this(sc, sc.conf.getInt("spark.task.maxFailures", 4)) val conf = sc.conf @@ -220,7 +206,7 @@ private[spark] class TaskSchedulerImpl private[scheduler]( private[scheduler] def createTaskSetManager( taskSet: TaskSet, maxTaskFailures: Int): TaskSetManager = { - new TaskSetManager(this, taskSet, maxTaskFailures, blacklistTracker, new SystemClock) + new TaskSetManager(this, taskSet, maxTaskFailures, new SystemClock) } override def cancelTasks(stageId: Int, interruptThread: Boolean): Unit = synchronized { @@ -267,12 +253,9 @@ private[spark] class TaskSchedulerImpl private[scheduler]( availableCpus: Array[Int], tasks: Seq[ArrayBuffer[TaskDescription]]) : Boolean = { var launchedTask = false - // nodes and executors that are blacklisted for the entire application have already been - // filtered out by this point for (i <- 0 until shuffledOffers.size) { - val offer = shuffledOffers(i) - val host = offer.host - val execId = offer.executorId + val execId = shuffledOffers(i).executorId + val host = shuffledOffers(i).host if (availableCpus(i) >= CPUS_PER_TASK) { try { for (task <- taskSet.resourceOffer(execId, host, maxLocality)) { @@ -322,24 +305,12 @@ private[spark] class TaskSchedulerImpl private[scheduler]( } } - // Before making any offers, remove any nodes from the blacklist whose blacklist 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. - blacklistTracker.foreach(_.applyBlacklistTimeout()) - - val sortedTaskSets = rootPool.getSortedTaskSetQueue - val filteredOffers = blacklistTracker.map { bl => - offers.filter { offer => - !bl.isNodeBlacklisted(offer.host) && - !bl.isExecutorBlacklisted(offer.executorId) - } - }.getOrElse(offers) - // Randomly shuffle offers to avoid always placing tasks on the same set of workers. - val shuffledOffers = Random.shuffle(filteredOffers) + val shuffledOffers = Random.shuffle(offers) // Build a list of tasks to assign to each worker. val tasks = shuffledOffers.map(o => new ArrayBuffer[TaskDescription](o.cores)) val availableCpus = shuffledOffers.map(o => o.cores).toArray + val sortedTaskSets = rootPool.getSortedTaskSetQueue for (taskSet <- sortedTaskSets) { logDebug("parentName: %s, name: %s, runningTasks: %s".format( taskSet.parent.name, taskSet.name, taskSet.runningTasks)) @@ -586,7 +557,6 @@ private[spark] class TaskSchedulerImpl private[scheduler]( executorIdToHost -= executorId rootPool.executorLost(executorId, host, reason) } - blacklistTracker.foreach(_.handleRemovedExecutor(executorId)) } def executorAdded(execId: String, host: String) { @@ -613,14 +583,6 @@ private[spark] class TaskSchedulerImpl private[scheduler]( executorIdToTaskCount.getOrElse(execId, -1) > 0 } - /** - * Get a snapshot of the currently blacklisted nodes for the entire application. This is - * thread-safe -- it can be called without a lock on the TaskScheduler. - */ - def nodeBlacklist(): scala.collection.immutable.Set[String] = { - blacklistTracker.map(_.nodeBlacklist()).getOrElse(scala.collection.immutable.Set()) - } - // By default, rack is unknown def getRackForHost(value: String): Option[String] = None @@ -699,13 +661,4 @@ private[spark] object TaskSchedulerImpl { retval.toList } - - private def createBlacklistTracker(conf: SparkConf): Option[BlacklistTracker] = { - if (BlacklistTracker.isBlacklistEnabled(conf)) { - Some(new BlacklistTracker(conf)) - } else { - None - } - } - } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetBlacklist.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetBlacklist.scala index e4845a7c7f61d..90584a933fd07 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetBlacklist.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetBlacklist.scala @@ -28,10 +28,6 @@ import org.apache.spark.util.Clock * (task, executor) / (task, nodes) pairs, and also completely blacklisting 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 - * about task failures until a taskset completes successfully. - * * THREADING: As a helper to [[TaskSetManager]], this class is designed to only be called from code * with a lock on the TaskScheduler (e.g. its event handlers). It should not be called from other * threads. @@ -46,9 +42,7 @@ private[scheduler] class TaskSetBlacklist(val conf: SparkConf, val stageId: Int, private val TIMEOUT_MILLIS = BlacklistTracker.getBlacklistTimeout(conf) /** - * 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. */ val execToFailures: HashMap[String, ExecutorFailuresInTaskSet] = new HashMap() @@ -63,8 +57,8 @@ private[scheduler] class TaskSetBlacklist(val conf: SparkConf, val stageId: Int, /** * 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 - * altogether. That is to keep this method as fast as possible in the inner-loop of the + * need to return true if the executor is blacklisted for the entire stage. + * 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( @@ -88,7 +82,7 @@ private[scheduler] class TaskSetBlacklist(val conf: SparkConf, val stageId: Int, /** * Return true if this executor is blacklisted for the given stage. Completely ignores whether - * the executor is blacklisted overall (or anything to do with the node the executor is on). That + * 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. */ 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 507bff18ac103..50cac3ba259fd 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -51,7 +51,6 @@ private[spark] class TaskSetManager( val sched: TaskSchedulerImpl, val taskSet: TaskSet, val maxTaskFailures: Int, - val blacklistTracker: Option[BlacklistTracker] = None, val clock: Clock = new SystemClock()) extends Schedulable with Logging { private val conf = sched.sc.conf @@ -86,7 +85,11 @@ private[spark] class TaskSetManager( var calculatedTasks = 0 val taskSetBlacklistOpt: Option[TaskSetBlacklist] = { - blacklistTracker.map { _ => new TaskSetBlacklist(conf, stageId, clock) } + if (BlacklistTracker.isBlacklistEnabled(conf)) { + Some(new TaskSetBlacklist(conf, stageId, clock)) + } else { + None + } } val runningTasksSet = new HashSet[Long] @@ -485,10 +488,6 @@ private[spark] class TaskSetManager( private def maybeFinishTaskSet() { if (isZombie && runningTasks == 0) { sched.taskSetFinished(this) - if (tasksSuccessful == numTasks) { - blacklistTracker.foreach(_.updateBlacklistForSuccessfulTaskSet(taskSet.stageId, - taskSet.stageAttemptId, taskSetBlacklistOpt.get.execToFailures)) - } } } @@ -590,8 +589,7 @@ private[spark] class TaskSetManager( */ private[scheduler] def abortIfCompletelyBlacklisted( hostToExecutors: HashMap[String, HashSet[String]]): Unit = { - blacklistTracker.foreach { appBlacklist => - val taskSetBlacklist = taskSetBlacklistOpt.get + taskSetBlacklistOpt.foreach { taskSetBlacklist => // If no executors have registered yet, don't abort the stage, just wait. We probably // got here because a task set was added before the executors registered. if (hostToExecutors.nonEmpty) { @@ -618,7 +616,7 @@ private[spark] class TaskSetManager( // when that unschedulable task is the last task remaining. val blacklistedEverywhere = hostToExecutors.forall { case (host, execs) => // Check if the task can run on the node - val nodeBlacklisted = appBlacklist.isNodeBlacklisted(host) || + val nodeBlacklisted = taskSetBlacklist.isNodeBlacklistedForTaskSet(host) || taskSetBlacklist.isNodeBlacklistedForTask(host, indexInTaskSet) if (nodeBlacklisted) { @@ -626,7 +624,6 @@ private[spark] class TaskSetManager( } else { // Check if the task can run on any of the executors execs.forall { exec => - appBlacklist.isExecutorBlacklisted(exec) || taskSetBlacklist.isExecutorBlacklistedForTaskSet(exec) || taskSetBlacklist.isExecutorBlacklistedForTask(exec, indexInTaskSet) } 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 684bf75c03d80..edc8aac5d1515 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 @@ -94,8 +94,7 @@ private[spark] object CoarseGrainedClusterMessages { case class RequestExecutors( requestedTotal: Int, localityAwareTasks: Int, - hostToLocalTaskCount: Map[String, Int], - nodeBlacklist: Set[String]) + hostToLocalTaskCount: Map[String, Int]) extends CoarseGrainedClusterMessage // Check if an executor was force-killed but for a reason unrelated to the running tasks. diff --git a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala index 7b6a2313f9e2a..915d7a1b8b164 100644 --- a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala +++ b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala @@ -272,7 +272,7 @@ private class FakeSchedulerBackend( protected override def doRequestTotalExecutors(requestedTotal: Int): Future[Boolean] = { clusterManagerEndpoint.ask[Boolean]( - RequestExecutors(requestedTotal, localityAwareTasks, hostToLocalTaskCount, Set.empty[String])) + RequestExecutors(requestedTotal, localityAwareTasks, hostToLocalTaskCount)) } protected override def doKillExecutors(executorIds: Seq[String]): Future[Boolean] = { @@ -291,7 +291,7 @@ private class FakeClusterManager(override val rpcEnv: RpcEnv) extends RpcEndpoin def getExecutorIdsToKill: Set[String] = executorIdsToKill.toSet override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { - case RequestExecutors(requestedTotal, _, _, _) => + case RequestExecutors(requestedTotal, _, _) => targetNumExecutors = requestedTotal context.reply(true) case KillExecutors(executorIds) => diff --git a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala index 745bffa1b252a..91c9135844601 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala @@ -17,299 +17,10 @@ package org.apache.spark.scheduler -import org.mockito.Mockito.when -import org.scalatest.BeforeAndAfterEach -import org.scalatest.mock.MockitoSugar - -import org.apache.spark._ +import org.apache.spark.{SparkConf, SparkFunSuite} 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 scheduler: TaskSchedulerImpl = _ - private var conf: SparkConf = _ - - override def afterEach(): Unit = { - if (blacklist != null) { - blacklist = null - } - if (scheduler != null) { - scheduler.stop() - scheduler = null - } - super.afterEach() - } - - val allExecutorAndHostIds = (('A' to 'Z').map("host" + _) ++ (1 to 100).map{_.toString}).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 (for thread-safety), 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 { opt => - val actual = f(opt) - val exp = expected.contains(opt) - assert(actual === exp, raw"""for string "$opt" """) - } - } - - 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) - scheduler - } - def createTaskSetBlacklist(stageId: Int = 0): TaskSetBlacklist = { - new TaskSetBlacklist(conf, stageId, clock) - } - - def configureBlacklistAndScheduler(confs: (String, String)*): Unit = { - conf = new SparkConf().setAppName("test").setMaster("local") - .set(config.BLACKLIST_ENABLED.key, "true") - confs.foreach { case (k, v) => conf.set(k, v) } - scheduler = mockTaskSchedWithConf(conf) - - clock.setTime(0) - blacklist = new BlacklistTracker(conf, clock) - } - - test("executors can be blacklisted with only a few failures per stage") { - configureBlacklistAndScheduler() - // for 4 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 do not unblacklist the - // executor. - val failuresTillBlacklisted = conf.get(config.MAX_FAILURES_PER_EXEC) - var failuresSoFar = 0 - (0 until failuresTillBlacklisted * 10).foreach { stage => - val taskSetBlacklist = createTaskSetBlacklist(stage) - if (stage % 2 == 0) { - // fail every other task - taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "1", index = 0) - failuresSoFar += 1 - } - blacklist.updateBlacklistForSuccessfulTaskSet(stage, 0, taskSetBlacklist.execToFailures) - assert(failuresSoFar == stage / 2 + 1) - if (failuresSoFar < failuresTillBlacklisted) { - assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set()) - } else { - assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1")) - } - } - } - - // if an executor has many task failures, but the task set ends up failing, don't count it - // against the executor - test("executors aren't blacklisted if task sets fail") { - configureBlacklistAndScheduler() - // for 4 different stages, executor 1 fails a task, and then the taskSet fails. - (0 until 4).foreach { stage => - val taskSetBlacklist = createTaskSetBlacklist(stage) - taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "1", index = 0) - } - assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set()) - } - - Seq(true, false).foreach { succeedTaskSet => - test(s"stage blacklist updates correctly on stage completion ($succeedTaskSet)") { - // within one taskset, an executor fails a few times, so its blacklisted for the taskset. - // but if the taskset fails, we don't blacklist the executor after the stage. - configureBlacklistAndScheduler() - val stageId = 1 + (if (succeedTaskSet) 1 else 0) - val taskSetBlacklist = createTaskSetBlacklist(stageId) - (0 until 4).foreach { index => - taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "1", index = index) - } - assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("1")) - assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set()) - if (succeedTaskSet) { - // the task set succeeded elsewhere, so we count those failures against our executor, - // and blacklist it across stages - blacklist.updateBlacklistForSuccessfulTaskSet(stageId, 0, taskSetBlacklist.execToFailures) - assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1")) - } 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") { - configureBlacklistAndScheduler() - val taskSetBlacklist0 = createTaskSetBlacklist(stageId = 0) - (0 until 4).foreach { partition => - taskSetBlacklist0.updateBlacklistForFailedTask("hostA", exec = "1", index = partition) - } - blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, taskSetBlacklist0.execToFailures) - assert(blacklist.nodeBlacklist() === Set()) - assertEquivalentToSet(blacklist.isNodeBlacklisted(_), Set()) - assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1")) - - val taskSetBlacklist1 = createTaskSetBlacklist(stageId = 1) - (0 until 4).foreach { partition => - taskSetBlacklist1.updateBlacklistForFailedTask("hostA", exec = "2", index = partition) - } - blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, taskSetBlacklist1.execToFailures) - assert(blacklist.nodeBlacklist() === Set("hostA")) - assertEquivalentToSet(blacklist.isNodeBlacklisted(_), Set("hostA")) - assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1", "2")) - - clock.advance(blacklist.BLACKLIST_TIMEOUT_MILLIS + 1) - blacklist.applyBlacklistTimeout() - assert(blacklist.nodeBlacklist() === Set()) - assertEquivalentToSet(blacklist.isNodeBlacklisted(_), Set()) - assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set()) - - // fail one more task, but executor isn't put back into blacklist since count reset to 0 - val taskSetBlacklist2 = createTaskSetBlacklist(stageId = 2) - taskSetBlacklist2.updateBlacklistForFailedTask("hostA", exec = "1", index = 0) - 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. - configureBlacklistAndScheduler() - val taskSetBlacklist0 = createTaskSetBlacklist(stageId = 0) - // Lets say that executor 1 dies completely. We get a task failure for the last task, but - // the taskset then finishes successfully (elsewhere). - (0 until 4).foreach { partition => - taskSetBlacklist0.updateBlacklistForFailedTask("hostA", exec = "1", index = partition) - } - blacklist.handleRemovedExecutor("1") - blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, taskSetBlacklist0.execToFailures) - assert(blacklist.isExecutorBlacklisted("1")) - clock.advance(blacklist.BLACKLIST_TIMEOUT_MILLIS / 2) - - // 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) - } - blacklist.handleRemovedExecutor("2") - blacklist.updateBlacklistForSuccessfulTaskSet(1, 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")) - assert(blacklist.isNodeBlacklisted("hostA")) - - clock.advance(blacklist.BLACKLIST_TIMEOUT_MILLIS / 2 + 1) - blacklist.applyBlacklistTimeout() - // executor 1 is no longer explicitly blacklisted, since we've gone past its recovery time, - // but everything else is still blacklisted. - assert(!blacklist.isExecutorBlacklisted("1")) - assert(blacklist.isExecutorBlacklisted("2")) - assert(blacklist.isNodeBlacklisted("hostA")) - // make sure we don't leak memory - assert(!blacklist.executorIdToBlacklistStatus.contains("1")) - assert(!blacklist.nodeToFailedExecs("hostA").contains("1")) - clock.advance(blacklist.BLACKLIST_TIMEOUT_MILLIS) - blacklist.applyBlacklistTimeout() - assert(!blacklist.nodeIdToBlacklistExpiryTime.contains("hostA")) - } - - test("task failures expire with time") { - configureBlacklistAndScheduler() - var stageId = 0 - def failOneTaskInTaskSet(exec: String): Unit = { - val taskSetBlacklist = createTaskSetBlacklist(stageId = stageId) - taskSetBlacklist.updateBlacklistForFailedTask("host-" + exec, exec, 0) - 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")) - assert(blacklist.nextExpiryTime === t1 + blacklist.BLACKLIST_TIMEOUT_MILLIS) - - // Fail a second executor, and go over its expiry as well. - 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")) - assert(blacklist.nextExpiryTime === t1 + blacklist.BLACKLIST_TIMEOUT_MILLIS) - - clock.setTime(t1 + blacklist.BLACKLIST_TIMEOUT_MILLIS + 1) - blacklist.applyBlacklistTimeout() - assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("3")) - 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()) - // 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("only blacklist nodes for the application when all the blacklisted executors are all on " + - "same host") { - // we blacklist executors on two different hosts -- make sure that doesn't lead to any - // node blacklisting - configureBlacklistAndScheduler() - val taskSetBlacklist0 = createTaskSetBlacklist(stageId = 0) - taskSetBlacklist0.updateBlacklistForFailedTask("hostA", exec = "1", index = 0) - taskSetBlacklist0.updateBlacklistForFailedTask("hostA", exec = "1", index = 1) - blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, taskSetBlacklist0.execToFailures) - assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1")) - assertEquivalentToSet(blacklist.isNodeBlacklisted(_), Set()) - - val taskSetBlacklist1 = createTaskSetBlacklist(stageId = 1) - taskSetBlacklist1.updateBlacklistForFailedTask("hostB", exec = "2", index = 0) - taskSetBlacklist1.updateBlacklistForFailedTask("hostB", exec = "2", index = 1) - blacklist.updateBlacklistForSuccessfulTaskSet(1, 0, taskSetBlacklist1.execToFailures) - assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1", "2")) - assertEquivalentToSet(blacklist.isNodeBlacklisted(_), Set()) - } +class BlacklistTrackerSuite extends SparkFunSuite { test("blacklist still respects legacy configs") { val legacyKey = config.BLACKLIST_LEGACY_TIMEOUT_CONF.key @@ -364,8 +75,6 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M 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") diff --git a/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala b/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala index 37ca6f3d53793..87600fe504b98 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala @@ -33,20 +33,16 @@ object FakeTask { * locations for each task (given as varargs) if this sequence is not empty. */ def createTaskSet(numTasks: Int, prefLocs: Seq[TaskLocation]*): TaskSet = { - createTaskSet(numTasks, 0, 0, prefLocs: _*) + createTaskSet(numTasks, 0, prefLocs: _*) } - def createTaskSet( - numTasks: Int, - stageId: Int, - stageAttemptId: Int, - prefLocs: Seq[TaskLocation]*): TaskSet = { + def createTaskSet(numTasks: Int, stageAttemptId: Int, prefLocs: Seq[TaskLocation]*): TaskSet = { if (prefLocs.size != 0 && prefLocs.size != numTasks) { throw new IllegalArgumentException("Wrong number of task locations") } val tasks = Array.tabulate[Task[_]](numTasks) { i => new FakeTask(0, i, if (prefLocs.size != 0) prefLocs(i) else Nil) } - new TaskSet(tasks, stageId, stageAttemptId, 0, null) + new TaskSet(tasks, 0, stageAttemptId, 0, null) } } 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 62ce6d1e22841..d73c747da6a0e 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -17,17 +17,11 @@ package org.apache.spark.scheduler -import scala.collection.mutable.HashMap - -import org.mockito.Matchers._ -import org.mockito.Mockito.{atLeast, never, spy, times, verify, when} import org.scalatest.BeforeAndAfterEach -import org.scalatest.mock.MockitoSugar import org.apache.spark._ import org.apache.spark.internal.config import org.apache.spark.internal.Logging -import org.apache.spark.storage.BlockManagerId class FakeSchedulerBackend extends SchedulerBackend { def start() {} @@ -37,7 +31,7 @@ class FakeSchedulerBackend extends SchedulerBackend { } class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with BeforeAndAfterEach - with Logging with MockitoSugar { + with Logging { var failedTaskSetException: Option[Throwable] = None var failedTaskSetReason: String = null @@ -46,16 +40,11 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B var taskScheduler: TaskSchedulerImpl = null var dagScheduler: DAGScheduler = null - val stageToMockTaskSetBlacklist = new HashMap[Int, TaskSetBlacklist]() - val stageToMockTaskSetManager = new HashMap[Int, TaskSetManager]() - override def beforeEach(): Unit = { super.beforeEach() failedTaskSet = false failedTaskSetException = None failedTaskSetReason = null - stageToMockTaskSetBlacklist.clear() - stageToMockTaskSetManager.clear() } override def afterEach(): Unit = { @@ -77,41 +66,6 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B } sc = new SparkContext(conf) taskScheduler = new TaskSchedulerImpl(sc) - setupHelper() - } - - def setupScheduler(blacklist: BlacklistTracker, confs: (String, String)*): TaskSchedulerImpl = { - sc = new SparkContext("local", "TaskSchedulerImplSuite") - confs.foreach { case (k, v) => - sc.conf.set(k, v) - } - taskScheduler = - new TaskSchedulerImpl(sc, sc.conf.getInt("spark.task.maxFailures", 4), Some(blacklist)) - setupHelper() - } - - def setupSchedulerWithMockTsm(blacklist: BlacklistTracker): TaskSchedulerImpl = { - sc = new SparkContext("local", "TaskSchedulerImplSuite") - taskScheduler = - new TaskSchedulerImpl(sc, sc.conf.getInt("spark.task.maxFailures", 4), Some(blacklist)) { - override def createTaskSetManager(taskSet: TaskSet, maxFailures: Int): TaskSetManager = { - val tsm = super.createTaskSetManager(taskSet, maxFailures) - val tsmSpy = spy(tsm) - val taskSetBlacklist = mock[TaskSetBlacklist] - when(tsmSpy.taskSetBlacklistOpt).thenReturn(Some(taskSetBlacklist)) - stageToMockTaskSetManager(taskSet.stageId) = tsmSpy - stageToMockTaskSetBlacklist(taskSet.stageId) = taskSetBlacklist - // intentionally bogus, just lets us easily verify - val execToFailures = new HashMap[String, ExecutorFailuresInTaskSet]() - execToFailures(taskSet.stageId.toString) = new ExecutorFailuresInTaskSet("dummy") - when(taskSetBlacklist.execToFailures).thenReturn(execToFailures) - tsmSpy - } - } - setupHelper() - } - - def setupHelper(): TaskSchedulerImpl = { taskScheduler.initialize(new FakeSchedulerBackend) // Need to initialize a DAGScheduler for the taskScheduler to use for callbacks. dagScheduler = new DAGScheduler(sc, taskScheduler) { @@ -210,8 +164,8 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B test("refuse to schedule concurrent attempts for the same stage (SPARK-8103)") { val taskScheduler = setupScheduler() - val attempt1 = FakeTask.createTaskSet(1, 0, 0) - val attempt2 = FakeTask.createTaskSet(1, 0, 1) + val attempt1 = FakeTask.createTaskSet(1, 0) + val attempt2 = FakeTask.createTaskSet(1, 1) taskScheduler.submitTasks(attempt1) intercept[IllegalStateException] { taskScheduler.submitTasks(attempt2) } @@ -219,7 +173,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B taskScheduler.taskSetManagerForAttempt(attempt1.stageId, attempt1.stageAttemptId) .get.isZombie = true taskScheduler.submitTasks(attempt2) - val attempt3 = FakeTask.createTaskSet(1, 0, 2) + val attempt3 = FakeTask.createTaskSet(1, 2) intercept[IllegalStateException] { taskScheduler.submitTasks(attempt3) } taskScheduler.taskSetManagerForAttempt(attempt2.stageId, attempt2.stageAttemptId) .get.isZombie = true @@ -248,7 +202,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B assert(0 === taskDescriptions2.length) // if we schedule another attempt for the same stage, it should get scheduled - val attempt2 = FakeTask.createTaskSet(10, 0, 1) + val attempt2 = FakeTask.createTaskSet(10, 1) // submit attempt 2, offer some resources, some tasks get scheduled taskScheduler.submitTasks(attempt2) @@ -280,7 +234,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B assert(0 === taskDescriptions2.length) // submit attempt 2 - val attempt2 = FakeTask.createTaskSet(10, 0, 1) + val attempt2 = FakeTask.createTaskSet(10, 1) taskScheduler.submitTasks(attempt2) // attempt 1 finished (this can happen even if it was marked zombie earlier -- all tasks were @@ -328,211 +282,6 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B assert(!failedTaskSet) } - test("scheduled tasks obey task and stage blacklists") { - val blacklist = mock[BlacklistTracker] - taskScheduler = setupSchedulerWithMockTsm(blacklist) - (0 to 2).foreach { stageId => - val taskSet = FakeTask.createTaskSet(numTasks = 2, stageId = stageId, stageAttemptId = 0) - taskScheduler.submitTasks(taskSet) - } - - val offers = Seq( - new WorkerOffer("executor0", "host0", 1), - new WorkerOffer("executor1", "host1", 1), - new WorkerOffer("executor2", "host1", 1), - new WorkerOffer("executor3", "host2", 10) - ) - - // setup our mock blacklist: - // stage 0 is blacklisted on node "host1" - // stage 1 is blacklisted on executor "executor3" - // stage 0, part 0 is blacklisted on executor 0 - // (later stubs take precedence over earlier ones) - when(blacklist.isNodeBlacklisted(anyString())).thenReturn(false) - when(blacklist.isExecutorBlacklisted(anyString())).thenReturn(false) - // setup some defaults, then override them with particulars - stageToMockTaskSetBlacklist.values.foreach { taskSetBlacklist => - when(taskSetBlacklist.isNodeBlacklistedForTaskSet(anyString())).thenReturn(false) - when(taskSetBlacklist.isExecutorBlacklistedForTaskSet(anyString())).thenReturn(false) - when(taskSetBlacklist.isExecutorBlacklistedForTask(anyString(), anyInt())).thenReturn(false) - when(taskSetBlacklist.isNodeBlacklistedForTask(anyString(), anyInt())).thenReturn(false) - } - when(stageToMockTaskSetBlacklist(0).isNodeBlacklistedForTaskSet("host1")).thenReturn(true) - when(stageToMockTaskSetBlacklist(1).isExecutorBlacklistedForTaskSet("executor3")) - .thenReturn(true) - when(stageToMockTaskSetBlacklist(0).isExecutorBlacklistedForTask("executor0", 0)) - .thenReturn(true) - - val firstTaskAttempts = taskScheduler.resourceOffers(offers).flatten - // these verifications are tricky b/c (a) we reference them multiple times -- also invoked when - // we check if we need to abort any stages from unschedulability and (b) resources - // are offered to the taskSets until they have no more free cores, so we don't necessarily - // even make an offer for each resource to each taskSet. - ('0' until '2').foreach { hostNum => - verify(blacklist, atLeast(1)).isNodeBlacklisted(s"host$hostNum") - } - (0 to 2).foreach { stageId => - verify(stageToMockTaskSetBlacklist(stageId), atLeast(1)) - .isNodeBlacklistedForTaskSet(anyString()) - } - for { - exec <- Seq("executor1", "executor2") - part <- 0 to 1 - } { - // the node blacklist should ensure we never check the task blacklist. This is important - // for performance, otherwise we end up changing an O(1) operation into a - // O(numPendingTasks) one - verify(stageToMockTaskSetBlacklist(0), never).isExecutorBlacklistedForTask(exec, part) - } - - // similarly, the executor blacklist for an entire stage should prevent us from ever checking - // the blacklist for specific parts in a stage. - (0 to 1).foreach { part => - verify(stageToMockTaskSetBlacklist(1), never).isExecutorBlacklistedForTask("executor3", part) - } - - // we should schedule all tasks. - assert(firstTaskAttempts.size === 6) - def tasksForStage(stageId: Int): Seq[TaskDescription] = { - firstTaskAttempts.filter{_.name.contains(s"stage $stageId")} - } - tasksForStage(0).foreach { task => - // exec 1 & 2 blacklisted for node - // exec 0 blacklisted just for part 0 - if (task.index == 0) { - assert(task.executorId === "executor3") - } else { - assert(Set("executor0", "executor3").contains(task.executorId)) - } - } - tasksForStage(1).foreach { task => - // exec 3 blacklisted - assert("executor3" != task.executorId) - } - // no restrictions on stage 2 - - // have all tasksets finish (stages 0 & 1 successfully, 2 unsuccessfully) - (0 to 2).foreach { stageId => - val tasks = tasksForStage(stageId) - val tsm = taskScheduler.taskSetManagerForAttempt(stageId, 0).get - val valueSer = SparkEnv.get.serializer.newInstance() - if (stageId == 2) { - // just need to make one task fail 4 times - var task = tasks(0) - val taskIndex = task.index - (0 until 4).foreach { attempt => - assert(task.attemptNumber === attempt) - tsm.handleFailedTask(task.taskId, TaskState.FAILED, TaskResultLost) - val nextAttempts = - taskScheduler.resourceOffers(Seq(WorkerOffer("executor4", "host4", 1))).flatten - if (attempt < 3) { - assert(nextAttempts.size === 1) - task = nextAttempts(0) - assert(task.index === taskIndex) - } else { - assert(nextAttempts.size === 0) - } - } - // end the other task of the taskset, doesn't matter whether it succeeds or fails - val otherTask = tasks(1) - val result = new DirectTaskResult[Int](valueSer.serialize(otherTask.taskId), Seq()) - tsm.handleSuccessfulTask(otherTask.taskId, result) - } else { - tasks.foreach { task => - val result = new DirectTaskResult[Int](valueSer.serialize(task.taskId), Seq()) - tsm.handleSuccessfulTask(task.taskId, result) - } - } - } - - // the tasksSets complete, so the tracker should be notified - verify(blacklist, times(1)).updateBlacklistForSuccessfulTaskSet( - 0, 0, stageToMockTaskSetBlacklist(0).execToFailures) - verify(blacklist, times(1)).updateBlacklistForSuccessfulTaskSet( - 1, 0, stageToMockTaskSetBlacklist(1).execToFailures) - } - - test("scheduled tasks obey node and executor blacklists") { - val blacklist = mock[BlacklistTracker] - taskScheduler = setupSchedulerWithMockTsm(blacklist) - (0 to 2).foreach { stageId => - val taskSet = FakeTask.createTaskSet(numTasks = 2, stageId = stageId, stageAttemptId = 0) - taskScheduler.submitTasks(taskSet) - } - - val offers = Seq( - new WorkerOffer("executor0", "host0", 1), - new WorkerOffer("executor1", "host1", 1), - new WorkerOffer("executor2", "host1", 1), - new WorkerOffer("executor3", "host2", 10) - ) - - // setup our mock blacklist: - // host1, executor0 & executor3 are completely blacklisted (which covers all the executors) - when(blacklist.isNodeBlacklisted(anyString())).thenReturn(false) - when(blacklist.isNodeBlacklisted("host1")).thenReturn(true) - when(blacklist.isExecutorBlacklisted(anyString())).thenReturn(false) - when(blacklist.isExecutorBlacklisted("executor0")).thenReturn(true) - when(blacklist.isExecutorBlacklisted("executor3")).thenReturn(true) - - val stageToTsm = (0 to 2).map { stageId => - val tsm = taskScheduler.taskSetManagerForAttempt(stageId, 0).get - stageId -> tsm - }.toMap - - val firstTaskAttempts = taskScheduler.resourceOffers(offers).flatten - firstTaskAttempts.foreach { task => logInfo(s"scheduled $task on ${task.executorId}") } - assert(firstTaskAttempts.isEmpty) - ('0' until '2').foreach { hostNum => - verify(blacklist, atLeast(1)).isNodeBlacklisted("host" + hostNum) - } - - // we should have aborted the existing stages, since they aren't schedulable - (0 to 2).foreach { stageId => - assert(stageToTsm(stageId).isZombie) - } - } - - test("abort stage when all executors are blacklisted") { - val blacklist = mock[BlacklistTracker] - taskScheduler = setupSchedulerWithMockTsm(blacklist) - val taskSet = FakeTask.createTaskSet(numTasks = 10, stageId = 0, stageAttemptId = 0) - taskScheduler.submitTasks(taskSet) - val tsm = stageToMockTaskSetManager(0) - - // first just submit some offers so the scheduler knows about all the executors - taskScheduler.resourceOffers(Seq( - WorkerOffer("executor0", "host0", 2), - WorkerOffer("executor1", "host0", 2), - WorkerOffer("executor2", "host0", 2), - WorkerOffer("executor3", "host1", 2) - )) - - // now say our blacklist updates to blacklist a bunch of resources, but *not* everything - when(blacklist.isNodeBlacklisted(anyString())).thenReturn(false) - when(blacklist.isNodeBlacklisted("host1")).thenReturn(true) - when(blacklist.isExecutorBlacklisted(anyString())).thenReturn(false) - when(blacklist.isExecutorBlacklisted("executor0")).thenReturn(true) - - // make an offer on the blacklisted resources. We won't schedule anything, but also won't - // abort yet, since we know of other resources that work - assert(taskScheduler.resourceOffers(Seq( - WorkerOffer("executor0", "host0", 2), - WorkerOffer("executor3", "host1", 2) - )).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) - assert(taskScheduler.resourceOffers(Seq( - WorkerOffer("executor0", "host0", 2), - WorkerOffer("executor3", "host1", 2) - )).flatten.size === 0) - assert(tsm.isZombie) - verify(tsm).abort(anyString(), anyObject()) - } - 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 // test explores a particular corner case -- you may have one task fail, but still be @@ -625,8 +374,8 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B test("SPARK-16106 locality levels updated if executor added to existing host") { val taskScheduler = setupScheduler() - taskScheduler.submitTasks(FakeTask.createTaskSet(2, 0, 0, - (0 until 2).map { _ => Seq(TaskLocation("host0", "executor2"))}: _* + taskScheduler.submitTasks(FakeTask.createTaskSet(2, 0, + (0 until 2).map { _ => Seq(TaskLocation("host0", "executor2")) }: _* )) val taskDescs = taskScheduler.resourceOffers(Seq( @@ -659,39 +408,4 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B assert(thirdTaskDescs.size === 0) assert(taskScheduler.getExecutorsAliveOnHost("host1") === Some(Set("executor1", "executor3"))) } - - test("check for executors that can be expired from blacklist") { - val blacklist = mock[BlacklistTracker] - taskScheduler = setupScheduler(blacklist) - - taskScheduler.submitTasks(FakeTask.createTaskSet(1, 0, 0)) - taskScheduler.resourceOffers(Seq( - new WorkerOffer("executor0", "host0", 1) - )).flatten - - verify(blacklist).applyBlacklistTimeout() - } - - test("don't update blacklist for shuffle-fetch failures, preemption, denied commits, " + - "or killed tasks") { - val blacklist = mock[BlacklistTracker] - taskScheduler = setupSchedulerWithMockTsm(blacklist) - val stage0 = FakeTask.createTaskSet(numTasks = 4, stageId = 0, stageAttemptId = 0) - taskScheduler.submitTasks(stage0) - val taskDescs = taskScheduler.resourceOffers( - Seq(new WorkerOffer("executor0", "host0", 10))).flatten - assert(taskDescs.size === 4) - - val tsm = stageToMockTaskSetManager(0) - taskScheduler.handleFailedTask(tsm, taskDescs(0).taskId, TaskState.FAILED, - FetchFailed(BlockManagerId("executor1", "host1", 12345), 0, 0, 0, "ignored")) - taskScheduler.handleFailedTask(tsm, taskDescs(1).taskId, TaskState.FAILED, - ExecutorLostFailure("executor0", exitCausedByApp = false, reason = None)) - taskScheduler.handleFailedTask(tsm, taskDescs(2).taskId, TaskState.FAILED, - TaskCommitDenied(0, 2, 0)) - taskScheduler.handleFailedTask(tsm, taskDescs(3).taskId, TaskState.KILLED, - TaskKilled) - verify(stageToMockTaskSetBlacklist(0), never()) - .updateBlacklistForFailedTask(anyString(), anyString(), anyInt()) - } } 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 4d175f961bb1c..5cc4774a65e1c 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -22,7 +22,7 @@ import java.util.Random import scala.collection.mutable import scala.collection.mutable.ArrayBuffer -import org.mockito.Mockito.{mock, verify, when} +import org.mockito.Mockito.{mock, verify} import org.apache.spark._ import org.apache.spark.internal.config @@ -181,7 +181,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg sched = new FakeTaskScheduler(sc, ("exec1", "host1")) val taskSet = FakeTask.createTaskSet(1) val clock = new ManualClock - val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) val accumUpdates = taskSet.tasks.head.metrics.internalAccums // Offer a host with NO_PREF as the constraint, @@ -234,7 +234,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg sched = new FakeTaskScheduler(sc, ("execA", "host1"), ("execC", "host2")) val taskSet = FakeTask.createTaskSet(1, Seq(TaskLocation("host1", "execB"))) val clock = new ManualClock - val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) // An executor that is not NODE_LOCAL should be rejected. assert(manager.resourceOffer("execC", "host2", ANY) === None) @@ -255,7 +255,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg Seq() // Last task has no locality prefs ) val clock = new ManualClock - val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) // First offer host1, exec1: first task should be chosen assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0) assert(manager.resourceOffer("exec1", "host1", PROCESS_LOCAL) == None) @@ -284,7 +284,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg Seq() // Last task has no locality prefs ) val clock = new ManualClock - val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) // First offer host1, exec1: first task should be chosen assert(manager.resourceOffer("exec1", "host1", PROCESS_LOCAL).get.index === 0) assert(manager.resourceOffer("exec3", "host2", PROCESS_LOCAL).get.index === 1) @@ -304,7 +304,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg Seq(TaskLocation("host2")) ) val clock = new ManualClock - val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) // First offer host1: first task should be chosen assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0) @@ -342,7 +342,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg Seq(TaskLocation("host3")) ) val clock = new ManualClock - val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) // First offer host1: first task should be chosen assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0) @@ -374,7 +374,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg sched = new FakeTaskScheduler(sc, ("exec1", "host1")) val taskSet = FakeTask.createTaskSet(1) val clock = new ManualClock - val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0) @@ -391,7 +391,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg sched = new FakeTaskScheduler(sc, ("exec1", "host1")) val taskSet = FakeTask.createTaskSet(1) val clock = new ManualClock - val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) // Fail the task MAX_TASK_FAILURES times, and check that the task set is aborted // after the last failure. @@ -412,8 +412,8 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg test("executors should be blacklisted after task failure, in spite of locality preferences") { val rescheduleDelay = 300L val conf = new SparkConf(). - set(config.BLACKLIST_ENABLED.key, "true"). - set(config.BLACKLIST_TIMEOUT_CONF.key, rescheduleDelay.toString). + set(config.BLACKLIST_ENABLED, true). + set(config.BLACKLIST_TIMEOUT_CONF, rescheduleDelay). // don't wait to jump locality levels in this test set("spark.locality.wait", "0") @@ -424,9 +424,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg // affinity to exec1 on host1 - which we will fail. val taskSet = FakeTask.createTaskSet(1, Seq(TaskLocation("host1", "exec1"))) val clock = new ManualClock - - val blacklist = new BlacklistTracker(conf, clock) - val manager = new TaskSetManager(sched, taskSet, 4, Some(blacklist), clock = clock) + val manager = new TaskSetManager(sched, taskSet, 4, clock) { val offerResult = manager.resourceOffer("exec1", "host1", PROCESS_LOCAL) @@ -482,7 +480,6 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg // Despite advancing beyond the time for expiring executors from within the blacklist, // we *never* expire from *within* the stage blacklist clock.advance(rescheduleDelay) - blacklist.applyBlacklistTimeout() { val offerResult = manager.resourceOffer("exec1", "host1", PROCESS_LOCAL) @@ -516,7 +513,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg Seq(TaskLocation("host2", "execC")), Seq()) val clock = new ManualClock - val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) // Only ANY is valid assert(manager.myLocalityLevels.sameElements(Array(NO_PREF, ANY))) // Add a new executor @@ -547,7 +544,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg Seq(TaskLocation("host1", "execB")), Seq(TaskLocation("host2", "execC")), Seq()) - val manager = new TaskSetManager(sched, taskSet, 1, clock = new ManualClock) + val manager = new TaskSetManager(sched, taskSet, 1, new ManualClock) sched.addExecutor("execA", "host1") manager.executorAdded() sched.addExecutor("execC", "host2") @@ -580,7 +577,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg Seq(TaskLocation("host1", "execA")), Seq(TaskLocation("host1", "execA"))) val clock = new ManualClock - val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) assert(manager.myLocalityLevels.sameElements(Array(PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL, ANY))) // Set allowed locality to ANY @@ -671,7 +668,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg Seq(), Seq(TaskLocation("host3", "execC"))) val clock = new ManualClock - val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) assert(manager.resourceOffer("execA", "host1", PROCESS_LOCAL).get.index === 0) assert(manager.resourceOffer("execA", "host1", NODE_LOCAL) == None) @@ -699,7 +696,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg Seq(), Seq(TaskLocation("host3"))) val clock = new ManualClock - val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) // node-local tasks are scheduled without delay assert(manager.resourceOffer("execA", "host1", NODE_LOCAL).get.index === 0) @@ -721,7 +718,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg Seq(ExecutorCacheTaskLocation("host1", "execA")), Seq(ExecutorCacheTaskLocation("host2", "execB"))) val clock = new ManualClock - val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) // process-local tasks are scheduled first assert(manager.resourceOffer("execA", "host1", NODE_LOCAL).get.index === 2) @@ -741,7 +738,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg Seq(ExecutorCacheTaskLocation("host1", "execA")), Seq(ExecutorCacheTaskLocation("host2", "execB"))) val clock = new ManualClock - val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) // process-local tasks are scheduled first assert(manager.resourceOffer("execA", "host1", PROCESS_LOCAL).get.index === 1) @@ -761,7 +758,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg Seq(TaskLocation("host1", "execA")), Seq(TaskLocation("host2", "execB.1"))) val clock = new ManualClock - val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) // Only ANY is valid assert(manager.myLocalityLevels.sameElements(Array(ANY))) // Add a new executor @@ -795,7 +792,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg Seq(TaskLocation("host2")), Seq(TaskLocation("hdfs_cache_host3"))) val clock = new ManualClock - val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) assert(manager.myLocalityLevels.sameElements(Array(PROCESS_LOCAL, NODE_LOCAL, ANY))) sched.removeExecutor("execA") manager.executorAdded() @@ -824,7 +821,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg sc.conf.set("spark.speculation.multiplier", "0.0") sc.conf.set("spark.speculation.quantile", "0.6") val clock = new ManualClock() - val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) val accumUpdatesByTask: Array[Seq[AccumulatorV2[_, _]]] = taskSet.tasks.map { task => task.metrics.internalAccums } diff --git a/docs/configuration.md b/docs/configuration.md index f120524bc1acb..a4675d8565288 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1297,28 +1297,6 @@ Apart from these, the following properties are also available, and may be useful the entire node is marked as failed for the stage. - - spark.blacklist.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 - 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 - may get marked as idle and be reclaimed by the cluster manager. - - - - spark.blacklist.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 - 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 - may get marked as idle and be reclaimed by the cluster manager. - - spark.speculation false diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index a1405495a0549..ad50ea789a913 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -684,11 +684,11 @@ private[spark] class ApplicationMaster( } override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { - case r: RequestExecutors => + case RequestExecutors(requestedTotal, localityAwareTasks, hostToLocalTaskCount) => Option(allocator) match { case Some(a) => - if (a.requestTotalExecutorsWithPreferredLocalities(r.requestedTotal, - r.localityAwareTasks, r.hostToLocalTaskCount, r.nodeBlacklist)) { + if (a.requestTotalExecutorsWithPreferredLocalities(requestedTotal, + localityAwareTasks, hostToLocalTaskCount)) { resetAllocatorInterval() } context.reply(true) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala index 004e30912a863..0b66d1cf08eac 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -114,8 +114,6 @@ private[yarn] class YarnAllocator( @volatile private var targetNumExecutors = YarnSparkHadoopUtil.getInitialTargetExecutorNumber(sparkConf) - private var currentNodeBlacklist = Set.empty[String] - // Executor loss reason requests that are pending - maps from executor ID for inquiry to a // list of requesters that should be responded to once we find out why the given executor // was lost. @@ -219,35 +217,18 @@ private[yarn] class YarnAllocator( * @param localityAwareTasks number of locality aware tasks to be used as container placement hint * @param hostToLocalTaskCount a map of preferred hostname to possible task counts to be used as * container placement hint. - * @param nodeBlacklist a set of blacklisted nodes, which is passed in to avoid allocating new - * containers on them. It will be used to update the application master's - * blacklist. * @return Whether the new requested total is different than the old value. */ def requestTotalExecutorsWithPreferredLocalities( requestedTotal: Int, localityAwareTasks: Int, - hostToLocalTaskCount: Map[String, Int], - nodeBlacklist: Set[String]): Boolean = synchronized { + hostToLocalTaskCount: Map[String, Int]): Boolean = synchronized { this.numLocalityAwareTasks = localityAwareTasks this.hostToLocalTaskCounts = hostToLocalTaskCount if (requestedTotal != targetNumExecutors) { logInfo(s"Driver requested a total number of $requestedTotal executor(s).") targetNumExecutors = requestedTotal - - // Update blacklist infomation to YARN ResouceManager for this application, - // in order to avoid allocating new Containers on the problematic nodes. - val blacklistAdditions = nodeBlacklist -- currentNodeBlacklist - val blacklistRemovals = currentNodeBlacklist -- nodeBlacklist - if (blacklistAdditions.nonEmpty) { - logInfo(s"adding nodes to blacklist: $blacklistAdditions") - } - if (blacklistRemovals.nonEmpty) { - logInfo(s"removing nodes from blacklist: $blacklistRemovals") - } - amClient.updateBlacklist(blacklistAdditions.toList.asJava, blacklistRemovals.toList.asJava) - currentNodeBlacklist = nodeBlacklist true } else { false diff --git a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala index af30686cfa205..2f9ea1911fd61 100644 --- a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala +++ b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala @@ -126,11 +126,8 @@ private[spark] abstract class YarnSchedulerBackend( * This includes executors already pending or running. */ override def doRequestTotalExecutors(requestedTotal: Int): Future[Boolean] = { - val nodeBlacklist: Set[String] = scheduler.nodeBlacklist() - val filteredHostToLocalTaskCount = hostToLocalTaskCount.filterKeys(!nodeBlacklist.contains(_)) yarnSchedulerEndpointRef.ask[Boolean]( - RequestExecutors(requestedTotal, localityAwareTasks, filteredHostToLocalTaskCount, - nodeBlacklist)) + RequestExecutors(requestedTotal, localityAwareTasks, hostToLocalTaskCount)) } /** diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala index 33f54493c3aa8..994dc75d34c30 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala @@ -202,7 +202,7 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter handler.getNumExecutorsRunning should be (0) handler.getPendingAllocate.size should be (4) - handler.requestTotalExecutorsWithPreferredLocalities(3, 0, Map.empty, Set.empty) + handler.requestTotalExecutorsWithPreferredLocalities(3, 0, Map.empty) handler.updateResourceRequests() handler.getPendingAllocate.size should be (3) @@ -213,7 +213,7 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter handler.allocatedContainerToHostMap.get(container.getId).get should be ("host1") handler.allocatedHostToContainersMap.get("host1").get should contain (container.getId) - handler.requestTotalExecutorsWithPreferredLocalities(2, 0, Map.empty, Set.empty) + handler.requestTotalExecutorsWithPreferredLocalities(2, 0, Map.empty) handler.updateResourceRequests() handler.getPendingAllocate.size should be (1) } @@ -224,7 +224,7 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter handler.getNumExecutorsRunning should be (0) handler.getPendingAllocate.size should be (4) - handler.requestTotalExecutorsWithPreferredLocalities(3, 0, Map.empty, Set.empty) + handler.requestTotalExecutorsWithPreferredLocalities(3, 0, Map.empty) handler.updateResourceRequests() handler.getPendingAllocate.size should be (3) @@ -234,7 +234,7 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter handler.getNumExecutorsRunning should be (2) - handler.requestTotalExecutorsWithPreferredLocalities(1, 0, Map.empty, Set.empty) + handler.requestTotalExecutorsWithPreferredLocalities(1, 0, Map.empty) handler.updateResourceRequests() handler.getPendingAllocate.size should be (0) handler.getNumExecutorsRunning should be (2) @@ -250,7 +250,7 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter val container2 = createContainer("host2") handler.handleAllocatedContainers(Array(container1, container2)) - handler.requestTotalExecutorsWithPreferredLocalities(1, 0, Map.empty, Set.empty) + handler.requestTotalExecutorsWithPreferredLocalities(1, 0, Map.empty) handler.executorIdToContainer.keys.foreach { id => handler.killExecutor(id ) } val statuses = Seq(container1, container2).map { c => @@ -272,7 +272,7 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter val container2 = createContainer("host2") handler.handleAllocatedContainers(Array(container1, container2)) - handler.requestTotalExecutorsWithPreferredLocalities(2, 0, Map(), Set.empty) + handler.requestTotalExecutorsWithPreferredLocalities(2, 0, Map()) val statuses = Seq(container1, container2).map { c => ContainerStatus.newInstance(c.getId(), ContainerState.COMPLETE, "Failed", -1) From 21e678995c6e06aa7892b23edc9d04b6f7e731e3 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 27 Sep 2016 08:32:26 -0500 Subject: [PATCH 46/57] typos, formatting --- .../org/apache/spark/scheduler/TaskSetBlacklist.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetBlacklist.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetBlacklist.scala index 90584a933fd07..a5bb278f725a7 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetBlacklist.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetBlacklist.scala @@ -24,7 +24,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.util.Clock /** - * Handles balcklisting executors and nodes within a taskset. This includes blacklisting specific + * Handles blacklisting executors and nodes within a taskset. This includes blacklisting specific * (task, executor) / (task, nodes) pairs, and also completely blacklisting executors and nodes * for the entire taskset. * @@ -95,9 +95,9 @@ private[scheduler] class TaskSetBlacklist(val conf: SparkConf, val stageId: Int, } private[scheduler] def updateBlacklistForFailedTask( - host: String, - exec: String, - index: Int): Unit = { + host: String, + exec: String, + index: Int): Unit = { val execFailures = execToFailures.getOrElseUpdate(exec, new ExecutorFailuresInTaskSet(host)) execFailures.updateWithFailure(index, clock.getTimeMillis() + TIMEOUT_MILLIS) From 9b953eafbed48dfe507997ff0ee4b07489f63b7b Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Fri, 30 Sep 2016 09:54:27 -0500 Subject: [PATCH 47/57] review feedback --- .../spark/scheduler/BlacklistTracker.scala | 6 ++---- .../spark/scheduler/TaskSetManager.scala | 20 ++++++++++--------- 2 files changed, 13 insertions(+), 13 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala index 362a032db27af..fc1abc3a2220b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala @@ -46,14 +46,12 @@ private[scheduler] object BlacklistTracker extends Logging { conf.get(config.BLACKLIST_LEGACY_TIMEOUT_CONF) match { case Some(legacyTimeout) => if (legacyTimeout == 0) { - logWarning(s"Turning off blacklisting due to legacy configuaration:" + - s" $legacyKey == 0") + logWarning(s"Turning off blacklisting due to legacy configuration: $legacyKey == 0") false } else { // mostly this is necessary just for tests, since real users that want the blacklist // will get it anyway by default - logWarning(s"Turning on blacklisting due to legacy configuration:" + - s" $legacyKey > 0") + logWarning(s"Turning on blacklisting due to legacy configuration: $legacyKey > 0") true } case None => 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 50cac3ba259fd..f8b27e6eb6dd6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -48,10 +48,10 @@ import org.apache.spark.util.{AccumulatorV2, Clock, SystemClock, Utils} * task set will be aborted */ private[spark] class TaskSetManager( - val sched: TaskSchedulerImpl, + sched: TaskSchedulerImpl, val taskSet: TaskSet, val maxTaskFailures: Int, - val clock: Clock = new SystemClock()) extends Schedulable with Logging { + clock: Clock = new SystemClock()) extends Schedulable with Logging { private val conf = sched.sc.conf @@ -264,10 +264,10 @@ private[spark] class TaskSetManager( } private def isTaskBlacklistedOnExecOrNode(index: Int, execId: String, host: String): Boolean = { - taskSetBlacklistOpt.map { blacklist => + taskSetBlacklistOpt.exists { blacklist => blacklist.isNodeBlacklistedForTask(host, index) || blacklist.isExecutorBlacklistedForTask(execId, index) - }.getOrElse(false) + } } /** @@ -412,10 +412,10 @@ private[spark] class TaskSetManager( maxLocality: TaskLocality.TaskLocality) : Option[TaskDescription] = { - val offerBlacklisted = taskSetBlacklistOpt.map { blacklist => + val offerBlacklisted = taskSetBlacklistOpt.exists { blacklist => blacklist.isNodeBlacklistedForTaskSet(host) || blacklist.isExecutorBlacklistedForTaskSet(execId) - }.getOrElse(false) + } if (!isZombie && !offerBlacklisted) { val curTime = clock.getTimeMillis() @@ -477,12 +477,14 @@ private[spark] class TaskSetManager( s"partition ${task.partitionId}, $taskLocality, ${serializedTask.limit} bytes)") sched.dagScheduler.taskStarted(task, info) - return Some(new TaskDescription(taskId = taskId, attemptNumber = attemptNum, execId, + Some(new TaskDescription(taskId = taskId, attemptNumber = attemptNum, execId, taskName, index, serializedTask)) case _ => + None } + } else { + None } - None } private def maybeFinishTaskSet() { @@ -631,7 +633,7 @@ private[spark] class TaskSetManager( } if (blacklistedEverywhere) { val partition = tasks(indexInTaskSet).partitionId - abort(s"Aborting ${taskSet} because task $indexInTaskSet (partition $partition) " + + abort(s"Aborting $taskSet because task $indexInTaskSet (partition $partition) " + s"cannot run anywhere due to node and executor blacklist. Blacklisting behavior " + s"can be configured via spark.blacklist.*.") } From 5568973d12b4027ca15d3cc4b27118e00c1c829b Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Fri, 30 Sep 2016 10:01:23 -0500 Subject: [PATCH 48/57] make spark.task.maxFailures a proper ConfigEntry --- .../org/apache/spark/internal/config/package.scala | 5 +++++ .../org/apache/spark/scheduler/BlacklistTracker.scala | 8 ++++---- .../org/apache/spark/scheduler/TaskSchedulerImpl.scala | 3 ++- .../spark/scheduler/BlacklistIntegrationSuite.scala | 4 ++-- .../apache/spark/scheduler/BlacklistTrackerSuite.scala | 10 +++++----- .../serializer/KryoSerializerDistributedSuite.scala | 2 +- .../spark/sql/execution/ui/SQLListenerSuite.scala | 3 ++- 7 files changed, 21 insertions(+), 14 deletions(-) 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 616a4998e5b6b..2a5fc1891f3af 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 @@ -93,6 +93,11 @@ package object config { .toSequence .createWithDefault(Nil) + private[spark] val MAX_TASK_FAILURES = + ConfigBuilder("spark.task.maxFailures") + .intConf + .createWithDefault(4) + // Blacklist confs private[spark] val BLACKLIST_ENABLED = ConfigBuilder("spark.blacklist.enabled") diff --git a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala index fc1abc3a2220b..dcd4d7db70247 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala @@ -112,16 +112,16 @@ private[scheduler] object BlacklistTracker extends Logging { } } - val maxTaskFailures = conf.getInt("spark.task.maxFailures", 4) + val maxTaskFailures = conf.get(config.MAX_TASK_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 >= spark.task.maxFailures " + + s"( = ${maxNodeAttempts}) was >= ${config.MAX_TASK_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 spark.task.maxFailures, or " + - s"disable blacklisting with ${config.BLACKLIST_ENABLED.key}") + s"${config.MAX_TASK_ATTEMPTS_PER_NODE.key}, increase ${config.MAX_TASK_FAILURES.key}, " + + s"or disable blacklisting with ${config.BLACKLIST_ENABLED.key}") } } } 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 1bc3bbb24e41c..49047cdef224d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -28,6 +28,7 @@ import scala.util.Random import org.apache.spark._ import org.apache.spark.TaskState.TaskState +import org.apache.spark.internal.config import org.apache.spark.internal.Logging import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.scheduler.TaskLocality.TaskLocality @@ -56,7 +57,7 @@ private[spark] class TaskSchedulerImpl( isLocal: Boolean = false) extends TaskScheduler with Logging { - def this(sc: SparkContext) = this(sc, sc.conf.getInt("spark.task.maxFailures", 4)) + def this(sc: SparkContext) = this(sc, sc.conf.get(config.MAX_TASK_FAILURES)) val conf = sc.conf diff --git a/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala index 0573200de172c..4509e628d743a 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala @@ -62,7 +62,7 @@ class BlacklistIntegrationSuite extends SchedulerIntegrationSuite[MultiExecutorM extraConfs = Seq( config.BLACKLIST_ENABLED.key -> "true", config.MAX_TASK_ATTEMPTS_PER_NODE.key -> "5", - "spark.task.maxFailures" -> "4", + config.MAX_TASK_FAILURES.key -> "4", "spark.testing.nHosts" -> "2", "spark.testing.nExecutorsPerHost" -> "5", "spark.testing.nCoresPerExecutor" -> "10", @@ -88,7 +88,7 @@ class BlacklistIntegrationSuite extends SchedulerIntegrationSuite[MultiExecutorM "With default settings, job can succeed despite multiple bad executors on node", extraConfs = Seq( config.BLACKLIST_ENABLED.key -> "true", - "spark.task.maxFailures" -> "4", + config.MAX_TASK_FAILURES.key -> "4", "spark.testing.nHosts" -> "2", "spark.testing.nExecutorsPerHost" -> "5", "spark.testing.nCoresPerExecutor" -> "10" diff --git a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala index 91c9135844601..4c555a072aac1 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala @@ -54,20 +54,20 @@ class BlacklistTrackerSuite extends SparkFunSuite { (2, 3), (3, 3) ).foreach { case (maxTaskFailures, maxNodeAttempts) => - conf.set("spark.task.maxFailures", maxTaskFailures.toString) + conf.set(config.MAX_TASK_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 >= spark.task.maxFailures " + + s"( = ${maxNodeAttempts}) was >= ${config.MAX_TASK_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 spark.task.maxFailures, or disable " + - s"blacklisting with ${config.BLACKLIST_ENABLED.key}") + s"${config.MAX_TASK_ATTEMPTS_PER_NODE.key}, increase ${config.MAX_TASK_FAILURES.key}, " + + s"or disable blacklisting with ${config.BLACKLIST_ENABLED.key}") } - conf.remove("spark.task.maxFailures") + conf.remove(config.MAX_TASK_FAILURES) conf.remove(config.MAX_TASK_ATTEMPTS_PER_NODE) Seq( 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 01e4135c398af..46aa9c37986cc 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala @@ -30,7 +30,7 @@ class KryoSerializerDistributedSuite extends SparkFunSuite with LocalSparkContex val conf = new SparkConf(false) .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") .set("spark.kryo.registrator", classOf[AppJarRegistrator].getName) - .set("spark.task.maxFailures", "1") + .set(config.MAX_TASK_FAILURES, 1) .set(config.BLACKLIST_ENABLED, false) val jar = TestUtils.createJarWithClasses(List(AppJarRegistrator.customClassName)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala index 6e60b0e4fad15..19b6d2603129c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala @@ -23,6 +23,7 @@ import org.mockito.Mockito.mock import org.apache.spark._ import org.apache.spark.executor.TaskMetrics +import org.apache.spark.internal.config import org.apache.spark.rdd.RDD import org.apache.spark.scheduler._ import org.apache.spark.sql.{DataFrame, SparkSession} @@ -446,7 +447,7 @@ class SQLListenerMemoryLeakSuite extends SparkFunSuite { val conf = new SparkConf() .setMaster("local") .setAppName("test") - .set("spark.task.maxFailures", "1") // Don't retry the tasks to run this test quickly + .set(config.MAX_TASK_FAILURES, 1) // Don't retry the tasks to run this test quickly .set("spark.sql.ui.retainedExecutions", "50") // Set it to 50 to run this test quickly val sc = new SparkContext(conf) try { From ab2ad380c7e5457f0178462fc70a196e2759a058 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 4 Oct 2016 21:05:23 -0500 Subject: [PATCH 49/57] review feedback --- .../spark/scheduler/BlacklistTracker.scala | 63 ++++----- .../scheduler/ExecutorFailuresInTaskSet.scala | 11 +- .../spark/scheduler/TaskSchedulerImpl.scala | 2 +- .../spark/scheduler/TaskSetBlacklist.scala | 14 +- .../spark/scheduler/TaskSetManager.scala | 128 ++++++++---------- .../scheduler/BlacklistIntegrationSuite.scala | 37 +---- .../scheduler/BlacklistTrackerSuite.scala | 39 +++--- .../scheduler/SchedulerIntegrationSuite.scala | 20 +-- .../scheduler/TaskSetBlacklistSuite.scala | 18 ++- .../spark/scheduler/TaskSetManagerSuite.scala | 19 ++- 10 files changed, 151 insertions(+), 200 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala index dcd4d7db70247..cd6618f23a14d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala @@ -57,7 +57,7 @@ private[scheduler] object BlacklistTracker extends Logging { case None => // local-cluster is *not* considered local for these purposes, we still want the // blacklist enabled by default - !Utils.isLocalMaster(conf) + false } } } @@ -85,44 +85,39 @@ private[scheduler] object BlacklistTracker extends Logging { throw new IllegalArgumentException(s"$k was $v, but must be > 0.") } - // undocumented escape hatch for validation -- just for tests that want to run in an "unsafe" - // configuration. - if (!conf.get("spark.blacklist.testing.skipValidation", "false").toBoolean) { - - 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 - ).foreach { config => - val v = conf.get(config) - if (v <= 0) { - mustBePos(config.key, v.toString) - } + 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 + ).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 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.MAX_TASK_FAILURES) - val maxNodeAttempts = conf.get(config.MAX_TASK_ATTEMPTS_PER_NODE) + val maxTaskFailures = conf.get(config.MAX_TASK_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.MAX_TASK_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.MAX_TASK_FAILURES.key}, " + - s"or disable blacklisting with ${config.BLACKLIST_ENABLED.key}") - } + if (maxNodeAttempts >= maxTaskFailures) { + throw new IllegalArgumentException(s"${config.MAX_TASK_ATTEMPTS_PER_NODE.key} " + + s"( = ${maxNodeAttempts}) was >= ${config.MAX_TASK_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.MAX_TASK_FAILURES.key}, " + + s"or disable blacklisting with ${config.BLACKLIST_ENABLED.key}") } } } 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 3659d028cc3bf..c7a9ab2c56440 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ExecutorFailuresInTaskSet.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ExecutorFailuresInTaskSet.scala @@ -22,10 +22,10 @@ import scala.collection.mutable.HashMap * Small helper for tracking failed tasks for blacklisting purposes. Info on all failures for one * task set, within one task set. */ -class ExecutorFailuresInTaskSet(val node: String) { +private[scheduler] class ExecutorFailuresInTaskSet(val node: String) { /** * Mapping from index of the tasks in the taskset, to the number of times it has failed on this - * executor and the last time it failed. + * executor and the expiry time. */ val taskToFailureCountAndExpiryTime = HashMap[Int, (Int, Long)]() @@ -38,6 +38,13 @@ class ExecutorFailuresInTaskSet(val node: String) { def numUniqueTasksWithFailures: Int = taskToFailureCountAndExpiryTime.size + /** + * Return the number of times this executor has failed on the given task index. + */ + def getNumTaskFailures(index: Int): Int = { + taskToFailureCountAndExpiryTime.get(index).map(_._1).getOrElse(0) + } + override def toString(): String = { s"numUniqueTasksWithFailures = $numUniqueTasksWithFailures; " + s"tasksToFailureCount = $taskToFailureCountAndExpiryTime" 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 59518a931ddfb..d0555cff48b2e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -207,7 +207,7 @@ private[spark] class TaskSchedulerImpl( private[scheduler] def createTaskSetManager( taskSet: TaskSet, maxTaskFailures: Int): TaskSetManager = { - new TaskSetManager(this, taskSet, maxTaskFailures, new SystemClock) + new TaskSetManager(this, taskSet, maxTaskFailures) } override def cancelTasks(stageId: Int, interruptThread: Boolean): Unit = synchronized { diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetBlacklist.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetBlacklist.scala index a5bb278f725a7..0c9a871a646ad 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetBlacklist.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetBlacklist.scala @@ -28,9 +28,9 @@ import org.apache.spark.util.Clock * (task, executor) / (task, nodes) pairs, and also completely blacklisting executors and nodes * for the entire taskset. * - * THREADING: As a helper to [[TaskSetManager]], this class is designed to only be called from code - * with a lock on the TaskScheduler (e.g. its event handlers). It should not be called from other - * threads. + * 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(val conf: SparkConf, val stageId: Int, val clock: Clock) extends Logging { @@ -66,8 +66,7 @@ private[scheduler] class TaskSetBlacklist(val conf: SparkConf, val stageId: Int, index: Int): Boolean = { execToFailures.get(executorId) .map { execFailures => - val count = execFailures.taskToFailureCountAndExpiryTime.get(index).map(_._1).getOrElse(0) - count >= MAX_TASK_ATTEMPTS_PER_EXECUTOR + execFailures.getNumTaskFailures(index) >= MAX_TASK_ATTEMPTS_PER_EXECUTOR } .getOrElse(false) } @@ -81,7 +80,7 @@ private[scheduler] class TaskSetBlacklist(val conf: SparkConf, val stageId: Int, } /** - * Return true if this executor is blacklisted for the given stage. Completely ignores whether + * Return true if this executor is blacklisted for the given stage. Completely ignores * 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. @@ -111,13 +110,14 @@ private[scheduler] class TaskSetBlacklist(val conf: SparkConf, val stageId: Int, // because jobs are aborted based on the number task attempts; if we counted unique // executors, it would be hard to config to ensure that you try another // node before hitting the max number of task failures. - failures.taskToFailureCountAndExpiryTime.getOrElse(index, (0, 0))._1 + failures.getNumTaskFailures(index) } }.sum if (failuresOnHost >= MAX_TASK_ATTEMPTS_PER_NODE) { nodeToBlacklistedTasks.getOrElseUpdate(host, new HashSet()) += index } + // Check if enough tasks have failed on the executor to blacklist it for the entire stage. if (execFailures.numUniqueTasksWithFailures >= MAX_FAILURES_PER_EXEC_STAGE) { if (blacklistedExecs.add(exec)) { logInfo(s"Blacklisting executor ${exec} for stage $stageId") 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 f8b27e6eb6dd6..f3283f5298e52 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -84,7 +84,7 @@ private[spark] class TaskSetManager( var totalResultSize = 0L var calculatedTasks = 0 - val taskSetBlacklistOpt: Option[TaskSetBlacklist] = { + private val taskSetBlacklistOpt: Option[TaskSetBlacklist] = { if (BlacklistTracker.isBlacklistEnabled(conf)) { Some(new TaskSetBlacklist(conf, stageId, clock)) } else { @@ -429,58 +429,55 @@ private[spark] class TaskSetManager( } } - dequeueTask(execId, host, allowedLocality) match { - case Some((index, taskLocality, speculative)) => - // Found a task; do some bookkeeping and return a task description - val task = tasks(index) - val taskId = sched.newTaskId() - // Do various bookkeeping - copiesRunning(index) += 1 - val attemptNum = taskAttempts(index).size - val info = new TaskInfo(taskId, index, attemptNum, curTime, - execId, host, taskLocality, speculative) - taskInfos(taskId) = info - taskAttempts(index) = info :: taskAttempts(index) - // Update our locality level for delay scheduling - // NO_PREF will not affect the variables related to delay scheduling - if (maxLocality != TaskLocality.NO_PREF) { - currentLocalityIndex = getLocalityIndex(taskLocality) - lastLaunchTime = curTime - } - // Serialize and return the task - val startTime = clock.getTimeMillis() - val serializedTask: ByteBuffer = try { - Task.serializeWithDependencies(task, sched.sc.addedFiles, sched.sc.addedJars, ser) - } catch { - // If the task cannot be serialized, then there's no point to re-attempt the task, - // as it will always fail. So just abort the whole task-set. - case NonFatal(e) => - val msg = s"Failed to serialize task $taskId, not attempting to retry it." - logError(msg, e) - abort(s"$msg Exception during serialization: $e") - throw new TaskNotSerializableException(e) - } - if (serializedTask.limit > TaskSetManager.TASK_SIZE_TO_WARN_KB * 1024 && - !emittedTaskSizeWarning) { - emittedTaskSizeWarning = true - logWarning(s"Stage ${task.stageId} contains a task of very large size " + - s"(${serializedTask.limit / 1024} KB). The maximum recommended task size is " + - s"${TaskSetManager.TASK_SIZE_TO_WARN_KB} KB.") - } - addRunningTask(taskId) - - // We used to log the time it takes to serialize the task, but task size is already - // a good proxy to task serialization time. - // val timeTaken = clock.getTime() - startTime - val taskName = s"task ${info.id} in stage ${taskSet.id}" - logInfo(s"Starting $taskName (TID $taskId, $host, executor ${info.executorId}, " + - s"partition ${task.partitionId}, $taskLocality, ${serializedTask.limit} bytes)") - - sched.dagScheduler.taskStarted(task, info) - Some(new TaskDescription(taskId = taskId, attemptNumber = attemptNum, execId, - taskName, index, serializedTask)) - case _ => - None + dequeueTask(execId, host, allowedLocality).map { case ((index, taskLocality, speculative)) => + // Found a task; do some bookkeeping and return a task description + val task = tasks(index) + val taskId = sched.newTaskId() + // Do various bookkeeping + copiesRunning(index) += 1 + val attemptNum = taskAttempts(index).size + val info = new TaskInfo(taskId, index, attemptNum, curTime, + execId, host, taskLocality, speculative) + taskInfos(taskId) = info + taskAttempts(index) = info :: taskAttempts(index) + // Update our locality level for delay scheduling + // NO_PREF will not affect the variables related to delay scheduling + if (maxLocality != TaskLocality.NO_PREF) { + currentLocalityIndex = getLocalityIndex(taskLocality) + lastLaunchTime = curTime + } + // Serialize and return the task + val startTime = clock.getTimeMillis() + val serializedTask: ByteBuffer = try { + Task.serializeWithDependencies(task, sched.sc.addedFiles, sched.sc.addedJars, ser) + } catch { + // If the task cannot be serialized, then there's no point to re-attempt the task, + // as it will always fail. So just abort the whole task-set. + case NonFatal(e) => + val msg = s"Failed to serialize task $taskId, not attempting to retry it." + logError(msg, e) + abort(s"$msg Exception during serialization: $e") + throw new TaskNotSerializableException(e) + } + if (serializedTask.limit > TaskSetManager.TASK_SIZE_TO_WARN_KB * 1024 && + !emittedTaskSizeWarning) { + emittedTaskSizeWarning = true + logWarning(s"Stage ${task.stageId} contains a task of very large size " + + s"(${serializedTask.limit / 1024} KB). The maximum recommended task size is " + + s"${TaskSetManager.TASK_SIZE_TO_WARN_KB} KB.") + } + addRunningTask(taskId) + + // We used to log the time it takes to serialize the task, but task size is already + // a good proxy to task serialization time. + // val timeTaken = clock.getTime() - startTime + val taskName = s"task ${info.id} in stage ${taskSet.id}" + logInfo(s"Starting $taskName (TID $taskId, $host, executor ${info.executorId}, " + + s"partition ${task.partitionId}, $taskLocality, ${serializedTask.limit} bytes)") + + sched.dagScheduler.taskStarted(task, info) + new TaskDescription(taskId = taskId, attemptNumber = attemptNum, execId, + taskName, index, serializedTask) } } else { None @@ -575,28 +572,22 @@ private[spark] class TaskSetManager( /** * Check whether the given task set has been blacklisted 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 fail the task set, otherwise the job - * will hang. - * * 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 * making sure at least one of the unscheduled tasks is schedulable. This means we may not detect * the hang as quickly as we could have, but we'll always detect the hang eventually, and the * 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 on unscheduled task, and then iterates through each - * executor until it finds one that the task hasn't failed on already). + * 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). */ private[scheduler] def abortIfCompletelyBlacklisted( hostToExecutors: HashMap[String, HashSet[String]]): Unit = { taskSetBlacklistOpt.foreach { taskSetBlacklist => - // If no executors have registered yet, don't abort the stage, just wait. We probably - // got here because a task set was added before the executors registered. + // 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) { - // take any task that needs to be scheduled, and see if we can find some executor it *could* - // run on + // find any task that needs to be scheduled val pendingTask: Option[Int] = { // usually this will just take the last pending task, but because of the lazy removal // from each list, we may need to go deeper in the list. We poll from the end because @@ -616,7 +607,7 @@ private[spark] class TaskSetManager( // try to find some executor this task can run on. Its possible that some *other* // task isn't schedulable anywhere, but we will discover that in some later call, // when that unschedulable task is the last task remaining. - val blacklistedEverywhere = hostToExecutors.forall { case (host, execs) => + val blacklistedEverywhere = hostToExecutors.forall { case (host, execsOnHost) => // Check if the task can run on the node val nodeBlacklisted = taskSetBlacklist.isNodeBlacklistedForTaskSet(host) || @@ -625,7 +616,7 @@ private[spark] class TaskSetManager( true } else { // Check if the task can run on any of the executors - execs.forall { exec => + execsOnHost.forall { exec => taskSetBlacklist.isExecutorBlacklistedForTaskSet(exec) || taskSetBlacklist.isExecutorBlacklistedForTask(exec, indexInTaskSet) } @@ -783,10 +774,6 @@ private[spark] class TaskSetManager( None } - if (reason.countTowardsTaskFailures) { - taskSetBlacklistOpt.foreach(_.updateBlacklistForFailedTask(info.host, info.executorId, index)) - } - sched.dagScheduler.taskEnded(tasks(index), reason, null, accumUpdates, info) if (successful(index)) { @@ -799,6 +786,7 @@ private[spark] class TaskSetManager( } if (!isZombie && reason.countTowardsTaskFailures) { + taskSetBlacklistOpt.foreach(_.updateBlacklistForFailedTask(info.host, info.executorId, index)) assert (null != failureReason) numFailures(index) += 1 if (numFailures(index) >= maxTaskFailures) { diff --git a/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala index 4509e628d743a..f6015cd51c2bd 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala @@ -54,36 +54,6 @@ class BlacklistIntegrationSuite extends SchedulerIntegrationSuite[MultiExecutorM assertDataStructuresEmpty(noFailure = false) } - // even with the blacklist turned on, bad configs can lead to job failure. To survive one - // bad node, you need to make sure that - // maxTaskFailures > min(spark.blacklist.task.maxTaskAttemptsPerNode, nExecutorsPerHost) - testScheduler( - "With blacklist on, job will still fail if there are too many bad executors on bad host", - extraConfs = Seq( - config.BLACKLIST_ENABLED.key -> "true", - config.MAX_TASK_ATTEMPTS_PER_NODE.key -> "5", - config.MAX_TASK_FAILURES.key -> "4", - "spark.testing.nHosts" -> "2", - "spark.testing.nExecutorsPerHost" -> "5", - "spark.testing.nCoresPerExecutor" -> "10", - // Blacklisting will normally immediately complain that this config is invalid -- the point - // of this test is to expose that the configuration is unsafe, so skip the validation. - "spark.blacklist.testing.skipValidation" -> "true" - ) - ) { - // to reliably reproduce the failure, 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 -- we never try scheduling tasks on the good host-1, since 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) - awaitJobTermination(jobFuture, duration) - } - assertDataStructuresEmpty(noFailure = false) - } - testScheduler( "With default settings, job can succeed despite multiple bad executors on node", extraConfs = Seq( @@ -94,9 +64,10 @@ class BlacklistIntegrationSuite extends SchedulerIntegrationSuite[MultiExecutorM "spark.testing.nCoresPerExecutor" -> "10" ) ) { - // to reliably reproduce the failure, 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. + // To reliably reproduce the failure that would occur without blacklisting, 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. val rdd = new MockRDDWithLocalityPrefs(sc, 1, Nil, badHost) diff --git a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala index 4c555a072aac1..b2e7ec5df015c 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala @@ -23,36 +23,29 @@ import org.apache.spark.internal.config class BlacklistTrackerSuite extends SparkFunSuite { test("blacklist still respects legacy configs") { - val legacyKey = config.BLACKLIST_LEGACY_TIMEOUT_CONF.key + 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)) - { - val localConf = new SparkConf().setMaster("local") - assert(!BlacklistTracker.isBlacklistEnabled(localConf)) - localConf.set(legacyKey, "5000") - assert(BlacklistTracker.isBlacklistEnabled(localConf)) - assert(5000 === BlacklistTracker.getBlacklistTimeout(localConf)) - - localConf.set(legacyKey, "0") - assert(!BlacklistTracker.isBlacklistEnabled(localConf)) - } - - { - val distConf = new SparkConf().setMaster("yarn-cluster") - assert(BlacklistTracker.isBlacklistEnabled(distConf)) - assert(60 * 60 * 1000L === BlacklistTracker.getBlacklistTimeout(distConf)) - distConf.set(legacyKey, "5000") - assert(5000 === BlacklistTracker.getBlacklistTimeout(distConf)) - distConf.set(config.BLACKLIST_TIMEOUT_CONF.key, "10h") - assert(10 * 60 * 60 * 1000L == BlacklistTracker.getBlacklistTimeout(distConf)) - } + // 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 precendence + 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-cluster") Seq( (2, 2), - (2, 3), - (3, 3) + (2, 3) ).foreach { case (maxTaskFailures, maxNodeAttempts) => conf.set(config.MAX_TASK_FAILURES, maxTaskFailures) conf.set(config.MAX_TASK_ATTEMPTS_PER_NODE.key, maxNodeAttempts.toString) diff --git a/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala index 99db24c8161c6..c1ec8fce6a531 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala @@ -326,11 +326,7 @@ private[spark] abstract class MockBackend( executorIdToExecutor(task.executorId).freeCores += taskScheduler.CPUS_PER_TASK freeCores += taskScheduler.CPUS_PER_TASK } - // optimization (which is used by the actual backends too) -- don't revive offers on *all* - // executors when a task completes, just on the one which completed - val exec = executorIdToExecutor(task.executorId) - reviveWithOffers(IndexedSeq(WorkerOffer(executorId = exec.executorId, host = exec.host, - cores = exec.freeCores))) + reviveOffers() } } @@ -386,11 +382,7 @@ private[spark] abstract class MockBackend( * scheduling. */ override def reviveOffers(): Unit = { - reviveWithOffers(generateOffers()) - } - - def reviveWithOffers(offers: IndexedSeq[WorkerOffer]): Unit = { - val newTaskDescriptions = taskScheduler.resourceOffers(offers).flatten + val newTaskDescriptions = taskScheduler.resourceOffers(generateOffers()).flatten // get the task now, since that requires a lock on TaskSchedulerImpl, to prevent individual // tests from introducing a race if they need it val newTasks = taskScheduler.synchronized { @@ -598,7 +590,7 @@ class BasicSchedulerIntegrationSuite extends SchedulerIntegrationSuite[SingleCor * (a) map output is available whenever we run stage 1 * (b) we get a second attempt for stage 0 & stage 1 */ - testNoBlacklist("job with fetch failure") { + testScheduler("job with fetch failure") { val input = new MockRDD(sc, 2, Nil) val shuffledRdd = shuffle(10, input) val shuffleId = shuffledRdd.shuffleDeps.head.shuffleId @@ -634,7 +626,7 @@ class BasicSchedulerIntegrationSuite extends SchedulerIntegrationSuite[SingleCor assert(stageToAttempts === Map(0 -> Set(0, 1), 1 -> Set(0, 1))) } - testNoBlacklist("job failure after 4 attempts") { + testScheduler("job failure after 4 attempts") { def runBackend(): Unit = { val (taskDescription, _) = backend.beginTask() backend.taskFailed(taskDescription, new RuntimeException("test task failure")) @@ -648,9 +640,9 @@ class BasicSchedulerIntegrationSuite extends SchedulerIntegrationSuite[SingleCor assertDataStructuresEmpty(noFailure = false) } - def testNoBlacklist(name: String)(body: => Unit): Unit = { + override def testScheduler(name: String)(body: => Unit): Unit = { // in these simple tests, we only have one executor, so it doens't make sense to turn on the - // blacklist. Just an artifact of this simple test-framework still kinda acting like local-mode + // blacklist. testScheduler(name, extraConfs = Seq(BLACKLIST_ENABLED.key -> "false"))(body) } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetBlacklistSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetBlacklistSuite.scala index e651846d399ad..779c1b5ea3668 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetBlacklistSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetBlacklistSuite.scala @@ -29,27 +29,33 @@ class TaskSetBlacklistSuite extends SparkFunSuite { val taskSetBlacklist = new TaskSetBlacklist(conf, stageId = 0, clock = clock) clock.setTime(0) + // task 0 should be blacklisted on exec1, and nowhere else taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "exec1", index = 0) for { executor <- (1 to 4).map(_.toString) index <- 0 until 10 } { - val exp = (executor == "exec1" && index == 0) - assert(taskSetBlacklist.isExecutorBlacklistedForTask(executor, index) === exp) + val shouldBeBlacklisted = (executor == "exec1" && index == 0) + assert(taskSetBlacklist.isExecutorBlacklistedForTask(executor, index) === shouldBeBlacklisted) } assert(!taskSetBlacklist.isExecutorBlacklistedForTaskSet("exec1")) assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) - // Task 1 & 2 failed on both executor 1 & 2, so we should blacklist all executors on that host, - // for all tasks for the stage. Note the api expects multiple checks for each type of - // blacklist -- this actually fits naturally with its use in the scheduler. + // 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) taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "exec1", index = 1) assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("exec1")) assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) + // Mark one task as failed on exec2 -- not enough for any further blacklisting yet. taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "exec2", index = 0) assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("exec1")) assert(!taskSetBlacklist.isExecutorBlacklistedForTaskSet("exec2")) assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) + // 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) assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("exec1")) assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("exec2")) @@ -85,7 +91,7 @@ class TaskSetBlacklistSuite extends SparkFunSuite { // 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 b/c of spark.task.maxFailures. + // 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. 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 5cc4774a65e1c..4fa8398c77c6f 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -825,7 +825,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val accumUpdatesByTask: Array[Seq[AccumulatorV2[_, _]]] = taskSet.tasks.map { task => task.metrics.internalAccums } - // Offer resources for 4 tasks to start + // Offer resources for 5 tasks to start val tasks = new ArrayBuffer[TaskDescription]() for ((k, v) <- List( "exec1" -> "host1", @@ -840,20 +840,19 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg tasks += task } assert(sched.startedTasks.toSet === (0 until 5).toSet) - // Complete 3 tasks and leave 2 task in running + // Complete 3 tasks and leave 2 tasks in running for (id <- Set(0, 1, 2)) { manager.handleSuccessfulTask(id, createTaskResult(id, accumUpdatesByTask(id))) assert(sched.endedTasks(id) === Success) } def runningTaskForIndex(index: Int): TaskDescription = { - val t = tasks.find { task => task.index == index && !sched.endedTasks.contains(task.taskId) } - t match { - case Some(x) => x - case None => - throw new RuntimeException(s"couldn't find index $index in " + - s"tasks: ${tasks.map{t => t.index -> t.taskId}} with endedTasks:" + - s" ${sched.endedTasks.keys}") + tasks.find { task => + task.index == index && !sched.endedTasks.contains(task.taskId) + }.getOrElse { + throw new RuntimeException(s"couldn't find index $index in " + + s"tasks: ${tasks.map{t => t.index -> t.taskId}} with endedTasks:" + + s" ${sched.endedTasks.keys}") } } @@ -872,7 +871,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg } } - // we can't be sure which one of our running tasks will get anothe speculative copy + // we can't be sure which one of our running tasks will get another speculative copy val originalTasks = Seq(3, 4).map { index => index -> runningTaskForIndex(index) }.toMap // checkSpeculatableTasks checks that the task runtime is greater than the threshold for From 89d3c5eb44939c38b0be14a6fc10c2139d0126ab Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 4 Oct 2016 21:07:05 -0500 Subject: [PATCH 50/57] revert changes to the speculative execution test; instead add a seperate test for killing speculative tasks. --- .../spark/scheduler/TaskSetManagerSuite.scala | 65 +++++++++++++++++-- 1 file changed, 59 insertions(+), 6 deletions(-) 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 4fa8398c77c6f..69edcf3347243 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -816,19 +816,16 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg test("Kill other task attempts when one attempt belonging to the same task succeeds") { sc = new SparkContext("local", "test") sched = new FakeTaskScheduler(sc, ("exec1", "host1"), ("exec2", "host2")) - val taskSet = FakeTask.createTaskSet(5) + val taskSet = FakeTask.createTaskSet(4) // Set the speculation multiplier to be 0 so speculative tasks are launched immediately sc.conf.set("spark.speculation.multiplier", "0.0") - sc.conf.set("spark.speculation.quantile", "0.6") val clock = new ManualClock() val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) val accumUpdatesByTask: Array[Seq[AccumulatorV2[_, _]]] = taskSet.tasks.map { task => task.metrics.internalAccums } - // Offer resources for 5 tasks to start - val tasks = new ArrayBuffer[TaskDescription]() + // Offer resources for 4 tasks to start for ((k, v) <- List( - "exec1" -> "host1", "exec1" -> "host1", "exec1" -> "host1", "exec2" -> "host2", @@ -837,6 +834,62 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg assert(taskOption.isDefined) val task = taskOption.get assert(task.executorId === k) + } + assert(sched.startedTasks.toSet === Set(0, 1, 2, 3)) + // Complete the 3 tasks and leave 1 task in running + for (id <- Set(0, 1, 2)) { + manager.handleSuccessfulTask(id, createTaskResult(id, accumUpdatesByTask(id))) + assert(sched.endedTasks(id) === Success) + } + + // checkSpeculatableTasks checks that the task runtime is greater than the threshold for + // speculating. Since we use a threshold of 0 for speculation, tasks need to be running for + // > 0ms, so advance the clock by 1ms here. + clock.advance(1) + assert(manager.checkSpeculatableTasks(0)) + // Offer resource to start the speculative attempt for the running task + val taskOption5 = manager.resourceOffer("exec1", "host1", NO_PREF) + assert(taskOption5.isDefined) + val task5 = taskOption5.get + assert(task5.index === 3) + assert(task5.taskId === 4) + assert(task5.executorId === "exec1") + assert(task5.attemptNumber === 1) + sched.backend = mock(classOf[SchedulerBackend]) + // Complete the speculative attempt for the running task + manager.handleSuccessfulTask(4, createTaskResult(3, accumUpdatesByTask(3))) + // Verify that it kills other running attempt + verify(sched.backend).killTask(3, "exec2", true) + // Because the SchedulerBackend was a mock, the 2nd copy of the task won't actually be + // killed, so the FakeTaskScheduler is only told about the successful completion + // of the speculated task. + assert(sched.endedTasks(3) === Success) + } + + test("Killing speculative tasks does not count towards aborting the taskset") { + sc = new SparkContext("local", "test") + sched = new FakeTaskScheduler(sc, ("exec1", "host1"), ("exec2", "host2")) + val taskSet = FakeTask.createTaskSet(5) + // Set the speculation multiplier to be 0 so speculative tasks are launched immediately + sc.conf.set("spark.speculation.multiplier", "0.0") + sc.conf.set("spark.speculation.quantile", "0.6") + val clock = new ManualClock() + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) + val accumUpdatesByTask: Array[Seq[AccumulatorV2[_, _]]] = taskSet.tasks.map { task => + task.metrics.internalAccums + } + // Offer resources for 5 tasks to start + val tasks = new ArrayBuffer[TaskDescription]() + for ((k, v) <- List( + "exec1" -> "host1", + "exec1" -> "host1", + "exec1" -> "host1", + "exec2" -> "host2", + "exec2" -> "host2")) { + val taskOption = manager.resourceOffer(k, v, NO_PREF) + assert(taskOption.isDefined) + val task = taskOption.get + assert(task.executorId === k) tasks += task } assert(sched.startedTasks.toSet === (0 until 5).toSet) @@ -857,7 +910,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg } // have each of the running tasks fail 3 times (not enough to abort the stage) - (3 until 6).foreach { attempt => + (0 until 3).foreach { attempt => Seq(3, 4).foreach { index => val task = runningTaskForIndex(index) logInfo(s"failing task $task") From a6c863f2462986b66a93f0beac3bb1f163afa50d Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 4 Oct 2016 23:25:42 -0500 Subject: [PATCH 51/57] review feedback --- .../org/apache/spark/scheduler/BlacklistTracker.scala | 4 ++-- .../spark/scheduler/ExecutorFailuresInTaskSet.scala | 5 +++-- .../org/apache/spark/scheduler/TaskSetManager.scala | 11 ++++++----- 3 files changed, 11 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala index cd6618f23a14d..5c7ffe810d872 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala @@ -37,8 +37,8 @@ private[scheduler] object BlacklistTracker extends Logging { */ def isBlacklistEnabled(conf: SparkConf): Boolean = { conf.get(config.BLACKLIST_ENABLED) match { - case Some(isEnabled) => - isEnabled + 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 based on the cluster-mode (off for local-mode, on otherwise). 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 c7a9ab2c56440..58864df382e35 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ExecutorFailuresInTaskSet.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ExecutorFailuresInTaskSet.scala @@ -32,8 +32,9 @@ private[scheduler] class ExecutorFailuresInTaskSet(val node: String) { def updateWithFailure(taskIndex: Int, failureExpiryTime: Long): Unit = { val (prevFailureCount, prevFailureExpiryTime) = taskToFailureCountAndExpiryTime.getOrElse(taskIndex, (0, -1L)) - assert(failureExpiryTime >= prevFailureExpiryTime) - taskToFailureCountAndExpiryTime(taskIndex) = (prevFailureCount + 1, failureExpiryTime) + // just in case we encounter non-monotonicity in the clock, take the max time + val newExpiryTime = math.max(prevFailureExpiryTime, failureExpiryTime) + taskToFailureCountAndExpiryTime(taskIndex) = (prevFailureCount + 1, newExpiryTime) } def numUniqueTasksWithFailures: Int = taskToFailureCountAndExpiryTime.size 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 f3283f5298e52..0e2257321035c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -84,7 +84,7 @@ private[spark] class TaskSetManager( var totalResultSize = 0L var calculatedTasks = 0 - private val taskSetBlacklistOpt: Option[TaskSetBlacklist] = { + private val taskSetBlacklistHelperOpt: Option[TaskSetBlacklist] = { if (BlacklistTracker.isBlacklistEnabled(conf)) { Some(new TaskSetBlacklist(conf, stageId, clock)) } else { @@ -264,7 +264,7 @@ private[spark] class TaskSetManager( } private def isTaskBlacklistedOnExecOrNode(index: Int, execId: String, host: String): Boolean = { - taskSetBlacklistOpt.exists { blacklist => + taskSetBlacklistHelperOpt.exists { blacklist => blacklist.isNodeBlacklistedForTask(host, index) || blacklist.isExecutorBlacklistedForTask(execId, index) } @@ -412,7 +412,7 @@ private[spark] class TaskSetManager( maxLocality: TaskLocality.TaskLocality) : Option[TaskDescription] = { - val offerBlacklisted = taskSetBlacklistOpt.exists { blacklist => + val offerBlacklisted = taskSetBlacklistHelperOpt.exists { blacklist => blacklist.isNodeBlacklistedForTaskSet(host) || blacklist.isExecutorBlacklistedForTaskSet(execId) } @@ -583,7 +583,7 @@ private[spark] class TaskSetManager( */ private[scheduler] def abortIfCompletelyBlacklisted( hostToExecutors: HashMap[String, HashSet[String]]): Unit = { - taskSetBlacklistOpt.foreach { taskSetBlacklist => + taskSetBlacklistHelperOpt.foreach { taskSetBlacklist => // 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) { @@ -786,7 +786,8 @@ private[spark] class TaskSetManager( } if (!isZombie && reason.countTowardsTaskFailures) { - taskSetBlacklistOpt.foreach(_.updateBlacklistForFailedTask(info.host, info.executorId, index)) + taskSetBlacklistHelperOpt.foreach(_.updateBlacklistForFailedTask( + info.host, info.executorId, index)) assert (null != failureReason) numFailures(index) += 1 if (numFailures(index) >= maxTaskFailures) { From 9086106fa0dfdce8358f50ea81c0e6f14ee3a85a Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 5 Oct 2016 10:57:08 -0500 Subject: [PATCH 52/57] blacklisting off by default; small style changes --- .../scheduler/ExecutorFailuresInTaskSet.scala | 4 +-- .../spark/scheduler/TaskSchedulerImpl.scala | 2 +- .../spark/scheduler/TaskSetBlacklist.scala | 21 ++++++------- .../spark/scheduler/TaskSetManager.scala | 5 +++ .../scheduler/TaskSetBlacklistSuite.scala | 31 ++++++++++++------- docs/configuration.md | 3 +- 6 files changed, 38 insertions(+), 28 deletions(-) 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 58864df382e35..2bdb369eee3e2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ExecutorFailuresInTaskSet.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ExecutorFailuresInTaskSet.scala @@ -19,8 +19,8 @@ package org.apache.spark.scheduler import scala.collection.mutable.HashMap /** - * Small helper for tracking failed tasks for blacklisting purposes. Info on all failures for one - * task set, within one task set. + * Small helper for tracking failed tasks for blacklisting 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/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index d0555cff48b2e..3c6af4d4019d3 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -34,7 +34,7 @@ import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.scheduler.TaskLocality.TaskLocality import org.apache.spark.scheduler.local.LocalSchedulerBackend import org.apache.spark.storage.BlockManagerId -import org.apache.spark.util.{AccumulatorV2, SystemClock, ThreadUtils, Utils} +import org.apache.spark.util.{AccumulatorV2, ThreadUtils, Utils} /** * Schedules tasks for multiple types of clusters by acting through a SchedulerBackend. diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetBlacklist.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetBlacklist.scala index 0c9a871a646ad..653456a51e1f7 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetBlacklist.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetBlacklist.scala @@ -48,7 +48,8 @@ private[scheduler] class TaskSetBlacklist(val conf: SparkConf, val stageId: Int, /** * Map from node to all executors on it with failures. Needed because we want to know about - * executors on a node even after they have died. + * executors on a node even after they have died. (We don't want to bother tracking the + * node -> execs mapping in the usual case when there aren't any failures). */ private val nodeToExecsWithFailures: HashMap[String, HashSet[String]] = new HashMap() private val nodeToBlacklistedTasks: HashMap[String, HashSet[Int]] = new HashMap() @@ -64,19 +65,15 @@ private[scheduler] class TaskSetBlacklist(val conf: SparkConf, val stageId: Int, def isExecutorBlacklistedForTask( executorId: String, index: Int): Boolean = { - execToFailures.get(executorId) - .map { execFailures => - execFailures.getNumTaskFailures(index) >= MAX_TASK_ATTEMPTS_PER_EXECUTOR - } - .getOrElse(false) + execToFailures.get(executorId).exists { execFailures => + execFailures.getNumTaskFailures(index) >= MAX_TASK_ATTEMPTS_PER_EXECUTOR + } } def isNodeBlacklistedForTask( node: String, index: Int): Boolean = { - nodeToBlacklistedTasks.get(node) - .map(_.contains(index)) - .getOrElse(false) + nodeToBlacklistedTasks.get(node).exists(_.contains(index)) } /** @@ -98,10 +95,12 @@ private[scheduler] class TaskSetBlacklist(val conf: SparkConf, val stageId: Int, exec: String, index: Int): Unit = { val execFailures = execToFailures.getOrElseUpdate(exec, new ExecutorFailuresInTaskSet(host)) - execFailures.updateWithFailure(index, clock.getTimeMillis() + TIMEOUT_MILLIS) + execFailures.updateWithFailure( + taskIndex = index, + failureExpiryTime = clock.getTimeMillis() + TIMEOUT_MILLIS) // check if this task has also failed on other executors on the same host -- if its gone - // over the limit, blacklist it from the entire host + // over the limit, blacklist this task from the entire host. val execsWithFailuresOnNode = nodeToExecsWithFailures.getOrElseUpdate(host, new HashSet()) execsWithFailuresOnNode += exec val failuresOnHost = execsWithFailuresOnNode.toIterator.flatMap { exec => 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 0e2257321035c..9491bc7a0497e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -572,6 +572,11 @@ private[spark] class TaskSetManager( /** * Check whether the given task set has been blacklisted 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 fail the task set, otherwise the job + * will hang. + * * 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 * making sure at least one of the unscheduled tasks is schedulable. This means we may not detect diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetBlacklistSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetBlacklistSuite.scala index 779c1b5ea3668..f366831db5419 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetBlacklistSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetBlacklistSuite.scala @@ -22,13 +22,20 @@ import org.apache.spark.util.{ManualClock, SystemClock} class TaskSetBlacklistSuite extends SparkFunSuite { - test("Blacklisting individual tasks") { + 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 taskSetBlacklist = new TaskSetBlacklist(conf, stageId = 0, 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) for { @@ -41,11 +48,7 @@ class TaskSetBlacklistSuite extends SparkFunSuite { assert(!taskSetBlacklist.isExecutorBlacklistedForTaskSet("exec1")) assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) - // 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) + // Mark task 1 failed on exec1 -- this pushes the executor into the blacklist taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "exec1", index = 1) assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("exec1")) assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) @@ -60,6 +63,8 @@ class TaskSetBlacklistSuite extends SparkFunSuite { assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("exec1")) assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("exec2")) assert(taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) + // 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 @@ -68,6 +73,8 @@ class TaskSetBlacklistSuite extends SparkFunSuite { 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) } @@ -88,7 +95,7 @@ class TaskSetBlacklistSuite extends SparkFunSuite { } test("multiple attempts for the same task count once") { - // make sure that for blacklisting tasks, the node counts task attempts, not executors. But for + // 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. @@ -101,7 +108,7 @@ class TaskSetBlacklistSuite extends SparkFunSuite { .set(config.MAX_FAILURES_PER_EXEC_STAGE, 2) .set(config.MAX_FAILED_EXEC_PER_NODE_STAGE, 3) val taskSetBlacklist = new TaskSetBlacklist(conf, stageId = 0, new SystemClock()) - // fail a task twice on hostA, exec:1 + // Fail a task twice on hostA, exec:1 taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "1", index = 0) taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "1", index = 0) assert(taskSetBlacklist.isExecutorBlacklistedForTask("1", 0)) @@ -109,24 +116,24 @@ class TaskSetBlacklistSuite extends SparkFunSuite { assert(!taskSetBlacklist.isExecutorBlacklistedForTaskSet("1")) assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) - // fail the same task once more on hostA, exec:2 + // Fail the same task once more on hostA, exec:2 taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "2", index = 0) assert(taskSetBlacklist.isNodeBlacklistedForTask("hostA", 0)) assert(!taskSetBlacklist.isExecutorBlacklistedForTaskSet("2")) assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) - // fail another task on hostA, exec:1. Now that executor has failures on two different tasks, + // Fail another task on hostA, exec:1. Now that executor has failures on two different tasks, // so its blacklisted taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "1", index = 1) assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("1")) assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) - // fail a third task on hostA, exec:2, so that exec is blacklisted for the whole task set + // Fail a third task on hostA, exec:2, so that exec is blacklisted for the whole task set taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "2", index = 2) assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("2")) assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) - // fail a fourth & fifth task on hostA, exec:3. Now we've got three executors that are + // 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. taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "3", index = 3) taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "3", index = 4) diff --git a/docs/configuration.md b/docs/configuration.md index a4675d8565288..6eb3069145108 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1248,8 +1248,7 @@ Apart from these, the following properties are also available, and may be useful spark.blacklist.enabled - true in cluster mode;
- false in local mode + false If set to "true", prevent Spark from scheduling tasks on executors that have been blacklisted From bb654bb6adff313861dfe8c31aeb83c22518a9fc Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 5 Oct 2016 22:38:08 -0500 Subject: [PATCH 53/57] minor cleanup --- .../spark/scheduler/BlacklistTracker.scala | 29 +++++++------------ 1 file changed, 10 insertions(+), 19 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala index 5c7ffe810d872..fca4c6d37e446 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala @@ -18,8 +18,8 @@ package org.apache.spark.scheduler import org.apache.spark.SparkConf -import org.apache.spark.internal.config import org.apache.spark.internal.Logging +import org.apache.spark.internal.config import org.apache.spark.util.Utils private[scheduler] object BlacklistTracker extends Logging { @@ -31,9 +31,7 @@ private[scheduler] object BlacklistTracker extends Logging { * order: * 1. Is it specifically enabled or disabled? * 2. Is it enabled via the legacy timeout conf? - * 3. Use the default for the spark-master: - * - off for local mode - * - on for distributed modes (including local-cluster) + * 3. Default is off */ def isBlacklistEnabled(conf: SparkConf): Boolean = { conf.get(config.BLACKLIST_ENABLED) match { @@ -41,23 +39,16 @@ private[scheduler] object BlacklistTracker extends Logging { enabled case None => // if they've got a non-zero setting for the legacy conf, always enable the blacklist, - // otherwise, use the default based on the cluster-mode (off for local-mode, on otherwise). + // otherwise, use the default. val legacyKey = config.BLACKLIST_LEGACY_TIMEOUT_CONF.key - conf.get(config.BLACKLIST_LEGACY_TIMEOUT_CONF) match { - case Some(legacyTimeout) => - if (legacyTimeout == 0) { - logWarning(s"Turning off blacklisting due to legacy configuration: $legacyKey == 0") - false - } else { - // mostly this is necessary just for tests, since real users that want the blacklist - // will get it anyway by default - logWarning(s"Turning on blacklisting due to legacy configuration: $legacyKey > 0") - true - } - case None => - // local-cluster is *not* considered local for these purposes, we still want the - // blacklist enabled by default + 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 + } } } } From 354f36bd36c7615883c08542eea333704e421164 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 5 Oct 2016 22:39:28 -0500 Subject: [PATCH 54/57] remove timeout from this pr, as its only relevant at the app-level --- .../scheduler/ExecutorFailuresInTaskSet.scala | 19 ++++++++----------- .../spark/scheduler/TaskSetBlacklist.scala | 5 +---- .../scheduler/TaskSetBlacklistSuite.scala | 7 +++---- 3 files changed, 12 insertions(+), 19 deletions(-) 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 2bdb369eee3e2..20ab27d127aba 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ExecutorFailuresInTaskSet.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ExecutorFailuresInTaskSet.scala @@ -25,29 +25,26 @@ import scala.collection.mutable.HashMap private[scheduler] class ExecutorFailuresInTaskSet(val node: String) { /** * Mapping from index of the tasks in the taskset, to the number of times it has failed on this - * executor and the expiry time. + * executor. */ - val taskToFailureCountAndExpiryTime = HashMap[Int, (Int, Long)]() + val taskToFailureCount = HashMap[Int, Int]() - def updateWithFailure(taskIndex: Int, failureExpiryTime: Long): Unit = { - val (prevFailureCount, prevFailureExpiryTime) = - taskToFailureCountAndExpiryTime.getOrElse(taskIndex, (0, -1L)) - // just in case we encounter non-monotonicity in the clock, take the max time - val newExpiryTime = math.max(prevFailureExpiryTime, failureExpiryTime) - taskToFailureCountAndExpiryTime(taskIndex) = (prevFailureCount + 1, newExpiryTime) + def updateWithFailure(taskIndex: Int): Unit = { + val prevFailureCount = taskToFailureCount.getOrElse(taskIndex, 0) + taskToFailureCount(taskIndex) = prevFailureCount + 1 } - def numUniqueTasksWithFailures: Int = taskToFailureCountAndExpiryTime.size + def numUniqueTasksWithFailures: Int = taskToFailureCount.size /** * Return the number of times this executor has failed on the given task index. */ def getNumTaskFailures(index: Int): Int = { - taskToFailureCountAndExpiryTime.get(index).map(_._1).getOrElse(0) + taskToFailureCount.getOrElse(index, 0) } override def toString(): String = { s"numUniqueTasksWithFailures = $numUniqueTasksWithFailures; " + - s"tasksToFailureCount = $taskToFailureCountAndExpiryTime" + s"tasksToFailureCount = $taskToFailureCount" } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetBlacklist.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetBlacklist.scala index 653456a51e1f7..9af50e693e762 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetBlacklist.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetBlacklist.scala @@ -39,7 +39,6 @@ private[scheduler] class TaskSetBlacklist(val conf: SparkConf, val stageId: Int, private val MAX_TASK_ATTEMPTS_PER_NODE = conf.get(config.MAX_TASK_ATTEMPTS_PER_NODE) private val MAX_FAILURES_PER_EXEC_STAGE = conf.get(config.MAX_FAILURES_PER_EXEC_STAGE) private val MAX_FAILED_EXEC_PER_NODE_STAGE = conf.get(config.MAX_FAILED_EXEC_PER_NODE_STAGE) - private val TIMEOUT_MILLIS = BlacklistTracker.getBlacklistTimeout(conf) /** * A map from each executor to the task failures on that executor. @@ -95,9 +94,7 @@ private[scheduler] class TaskSetBlacklist(val conf: SparkConf, val stageId: Int, exec: String, index: Int): Unit = { val execFailures = execToFailures.getOrElseUpdate(exec, new ExecutorFailuresInTaskSet(host)) - execFailures.updateWithFailure( - taskIndex = index, - failureExpiryTime = clock.getTimeMillis() + TIMEOUT_MILLIS) + execFailures.updateWithFailure(index) // 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. diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetBlacklistSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetBlacklistSuite.scala index f366831db5419..8c902af5685ff 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetBlacklistSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetBlacklistSuite.scala @@ -83,12 +83,11 @@ class TaskSetBlacklistSuite extends SparkFunSuite { val execToFailures = taskSetBlacklist.execToFailures assert(execToFailures.keySet === Set("exec1", "exec2")) - val expectedExpiryTime = BlacklistTracker.getBlacklistTimeout(conf) Seq("exec1", "exec2").foreach { exec => assert( - execToFailures(exec).taskToFailureCountAndExpiryTime === Map( - 0 -> (1, expectedExpiryTime), - 1 -> (1, expectedExpiryTime) + execToFailures(exec).taskToFailureCount === Map( + 0 -> 1, + 1 -> 1 ) ) } From 34eff27bf25d80d4b6d8a31e7cbbadd2794d2e9c Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 6 Oct 2016 15:09:35 -0500 Subject: [PATCH 55/57] review feedback --- .../spark/scheduler/TaskSchedulerImpl.scala | 2 +- .../spark/scheduler/TaskSetBlacklist.scala | 24 ++++++++----------- 2 files changed, 11 insertions(+), 15 deletions(-) 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 3c6af4d4019d3..3e3f1ad031e66 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -28,8 +28,8 @@ import scala.util.Random import org.apache.spark._ import org.apache.spark.TaskState.TaskState -import org.apache.spark.internal.config import org.apache.spark.internal.Logging +import org.apache.spark.internal.config import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.scheduler.TaskLocality.TaskLocality import org.apache.spark.scheduler.local.LocalSchedulerBackend diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetBlacklist.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetBlacklist.scala index 9af50e693e762..f4b0f55b7686a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetBlacklist.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetBlacklist.scala @@ -19,8 +19,8 @@ package org.apache.spark.scheduler import scala.collection.mutable.{HashMap, HashSet} import org.apache.spark.SparkConf -import org.apache.spark.internal.config import org.apache.spark.internal.Logging +import org.apache.spark.internal.config import org.apache.spark.util.Clock /** @@ -43,17 +43,17 @@ private[scheduler] class TaskSetBlacklist(val conf: SparkConf, val stageId: Int, /** * A map from each executor to the task failures on that executor. */ - val execToFailures: HashMap[String, ExecutorFailuresInTaskSet] = new HashMap() + val execToFailures = new HashMap[String, ExecutorFailuresInTaskSet]() /** * Map from node to all executors on it with failures. Needed because we want to know about * executors on a node even after they have died. (We don't want to bother tracking the * node -> execs mapping in the usual case when there aren't any failures). */ - private val nodeToExecsWithFailures: HashMap[String, HashSet[String]] = new HashMap() - private val nodeToBlacklistedTasks: HashMap[String, HashSet[Int]] = new HashMap() - private val blacklistedExecs: HashSet[String] = new HashSet() - private val blacklistedNodes: HashSet[String] = new HashSet() + 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]() /** * Return true if this executor is blacklisted for the given task. This does *not* @@ -61,18 +61,14 @@ private[scheduler] class TaskSetBlacklist(val conf: SparkConf, val stageId: Int, * 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 isExecutorBlacklistedForTask(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 = { - nodeToBlacklistedTasks.get(node).exists(_.contains(index)) + def isNodeBlacklistedForTask(node: String, index: Int): Boolean = { + nodeToBlacklistedTaskIndexes.get(node).exists(_.contains(index)) } /** @@ -110,7 +106,7 @@ private[scheduler] class TaskSetBlacklist(val conf: SparkConf, val stageId: Int, } }.sum if (failuresOnHost >= MAX_TASK_ATTEMPTS_PER_NODE) { - nodeToBlacklistedTasks.getOrElseUpdate(host, new HashSet()) += index + nodeToBlacklistedTaskIndexes.getOrElseUpdate(host, new HashSet()) += index } // Check if enough tasks have failed on the executor to blacklist it for the entire stage. From c805a0ba5b2d90062b04d043ffdaa2dda559e136 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 10 Oct 2016 11:17:32 -0500 Subject: [PATCH 56/57] remove some unnecessary changes --- .../org/apache/spark/DistributedSuite.scala | 17 ++++------------- .../scheduler/SchedulerIntegrationSuite.scala | 7 ------- docs/configuration.md | 8 -------- 3 files changed, 4 insertions(+), 28 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/DistributedSuite.scala b/core/src/test/scala/org/apache/spark/DistributedSuite.scala index 62f5a6e616d4a..4e36adc8baf3f 100644 --- a/core/src/test/scala/org/apache/spark/DistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/DistributedSuite.scala @@ -21,7 +21,6 @@ import org.scalatest.concurrent.Timeouts._ import org.scalatest.Matchers import org.scalatest.time.{Millis, Span} -import org.apache.spark.internal.config.BLACKLIST_ENABLED import org.apache.spark.storage.{RDDBlockId, StorageLevel} import org.apache.spark.util.io.ChunkedByteBuffer @@ -109,9 +108,7 @@ class DistributedSuite extends SparkFunSuite with Matchers with LocalSparkContex } test("repeatedly failing task") { - val conf = new SparkConf().setAppName("test").setMaster(clusterUrl) - .set(BLACKLIST_ENABLED, false) - sc = new SparkContext(conf) + sc = new SparkContext(clusterUrl, "test") val thrown = intercept[SparkException] { // scalastyle:off println sc.parallelize(1 to 10, 10).foreach(x => println(x / 0)) @@ -126,9 +123,7 @@ class DistributedSuite extends SparkFunSuite with Matchers with LocalSparkContex // than hanging due to retrying the failed task infinitely many times (eventually the // standalone scheduler will remove the application, causing the job to hang waiting to // reconnect to the master). - val conf = new SparkConf().setAppName("test").setMaster(clusterUrl) - .set(BLACKLIST_ENABLED, false) - sc = new SparkContext(conf) + sc = new SparkContext(clusterUrl, "test") failAfter(Span(100000, Millis)) { val thrown = intercept[SparkException] { // One of the tasks always fails. @@ -142,9 +137,7 @@ class DistributedSuite extends SparkFunSuite with Matchers with LocalSparkContex test("repeatedly failing task that crashes JVM with a zero exit code (SPARK-16925)") { // Ensures that if a task which causes the JVM to exit with a zero exit code will cause the // Spark job to eventually fail. - val conf = new SparkConf().setAppName("test").setMaster(clusterUrl) - .set(BLACKLIST_ENABLED, false) - sc = new SparkContext(conf) + sc = new SparkContext(clusterUrl, "test") failAfter(Span(100000, Millis)) { val thrown = intercept[SparkException] { sc.parallelize(1 to 1, 1).foreachPartition { _ => System.exit(0) } @@ -262,9 +255,7 @@ class DistributedSuite extends SparkFunSuite with Matchers with LocalSparkContex test("recover from repeated node failures during shuffle-reduce") { import DistributedSuite.{markNodeIfIdentity, failOnMarkedIdentity} DistributedSuite.amMaster = true - val conf = new SparkConf().setAppName("test").setMaster(clusterUrl) - .set(BLACKLIST_ENABLED, false) - sc = new SparkContext(conf) + sc = new SparkContext(clusterUrl, "test") for (i <- 1 to 3) { val data = sc.parallelize(Seq(true, true), 2) assert(data.count === 2) diff --git a/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala index c1ec8fce6a531..c28aa06623a60 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala @@ -31,7 +31,6 @@ import org.scalatest.Assertions.AssertionsHelper import org.apache.spark._ import org.apache.spark.TaskState._ -import org.apache.spark.internal.config.BLACKLIST_ENABLED import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.util.{CallSite, ThreadUtils, Utils} @@ -639,10 +638,4 @@ class BasicSchedulerIntegrationSuite extends SchedulerIntegrationSuite[SingleCor } assertDataStructuresEmpty(noFailure = false) } - - override def testScheduler(name: String)(body: => Unit): Unit = { - // in these simple tests, we only have one executor, so it doens't make sense to turn on the - // blacklist. - testScheduler(name, extraConfs = Seq(BLACKLIST_ENABLED.key -> "false"))(body) - } } diff --git a/docs/configuration.md b/docs/configuration.md index 6eb3069145108..373e22d71a872 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1256,14 +1256,6 @@ Apart from these, the following properties are also available, and may be useful other "spark.blacklist" configuration options. - - spark.blacklist.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. - - spark.blacklist.task.maxTaskAttemptsPerExecutor 1 From 4501e6c089f99f2cc62443cca668f77fea2745aa Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 12 Oct 2016 09:44:16 -0500 Subject: [PATCH 57/57] fix merge --- .../scala/org/apache/spark/internal/config/package.scala | 7 ------- 1 file changed, 7 deletions(-) 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 2a5fc1891f3af..47368eb7aaf97 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 @@ -136,13 +136,6 @@ package object config { .createOptional // End blacklist confs - // Note: This is a SQL config but needs to be in core because the REPL depends on it - private[spark] val CATALOG_IMPLEMENTATION = ConfigBuilder("spark.sql.catalogImplementation") - .internal() - .stringConf - .checkValues(Set("hive", "in-memory")) - .createWithDefault("in-memory") - private[spark] val LISTENER_BUS_EVENT_QUEUE_SIZE = ConfigBuilder("spark.scheduler.listenerbus.eventqueue.size") .intConf