Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r70007113
  
    --- Diff: 
core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala ---
    @@ -0,0 +1,320 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.scheduler
    +
    +import java.util.concurrent.atomic.AtomicReference
    +
    +import scala.collection.mutable.{HashMap, HashSet}
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.util.Clock
    +import org.apache.spark.util.SystemClock
    +import org.apache.spark.util.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).
    + *
    + * 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 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)
    +  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())
    +  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
    +    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)
    +      }
    +    }
    + }
    +
    +  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 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)
    +  }
    +
    +  /**
    +   * Return true if this executor is blacklisted for the given stage.  
Completely ignores whether
    --- End diff --
    
    Could you elaborate in the comments why this behavior is desired (i.e. 
ignoring whether the executor itself is blacklisted)?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org

Reply via email to