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

    https://github.com/apache/spark/pull/17113#discussion_r118261422
  
    --- Diff: 
core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala ---
    @@ -145,6 +146,75 @@ private[scheduler] class BlacklistTracker (
         nextExpiryTime = math.min(execMinExpiry, nodeMinExpiry)
       }
     
    +  private def killBlacklistedExecutor(exec: String): Unit = {
    +    if (conf.get(config.BLACKLIST_KILL_ENABLED)) {
    +      allocationClient match {
    +        case Some(a) =>
    +          logInfo(s"Killing blacklisted executor id $exec " +
    +            s"since spark.blacklist.killBlacklistedExecutors is set.")
    +          a.killExecutors(Seq(exec), true, true)
    +        case None =>
    +          logWarning(s"Not attempting to kill blacklisted executor id 
$exec " +
    +            s"since allocation client is not defined.")
    +      }
    +    }
    +  }
    +
    +  private def killExecutorsOnBlacklistedNode(node: String): Unit = {
    +    if (conf.get(config.BLACKLIST_KILL_ENABLED)) {
    +      allocationClient match {
    +        case Some(a) =>
    +          logInfo(s"Killing all executors on blacklisted host $node " +
    +            s"since spark.blacklist.killBlacklistedExecutors is set.")
    +          if (a.killExecutorsOnHost(node) == false) {
    +            logError(s"Killing executors on node $node failed.")
    +          }
    +        case None =>
    +          logWarning(s"Not attempting to kill executors on blacklisted 
host $node " +
    +            s"since allocation client is not defined.")
    +      }
    +    }
    +  }
    +
    +  def updateBlacklistForFetchFailure(host: String, exec: String): Unit = {
    +    if (BLACKLIST_FETCH_FAILURE_ENABLED) {
    +      logWarning(
    +        s"""
    +           |${config.BLACKLIST_FETCH_FAILURE_ENABLED.key} is enabled. If 
we blacklist
    +           |on fetch failures, we are implicitly saying that we believe 
the failure is
    +           |non-transient, and can't be recovered from (even if this is 
the first fetch failure).
    +           |If the external shuffle-service is on, then every other 
executor on this node would
    +           |be suffering from the same issue, so we should blacklist (and 
potentially kill) all
    +           |of them immediately.
    +         """.stripMargin)
    +
    +      val now = clock.getTimeMillis()
    +      val expiryTimeForNewBlacklists = now + BLACKLIST_TIMEOUT_MILLIS
    +      if (!executorIdToBlacklistStatus.contains(exec)) {
    +        logInfo(s"Blacklisting executor $exec due to fetch failure")
    +
    +        executorIdToBlacklistStatus.put(exec, BlacklistedExecutor(host, 
expiryTimeForNewBlacklists))
    +        // We hardcoded number of failure tasks to 1 for fetch failure, 
because there's no
    +        // reattempt for such failure.
    +        listenerBus.post(SparkListenerExecutorBlacklisted(now, exec, 1))
    +        updateNextExpiryTime()
    +        killBlacklistedExecutor(exec)
    +
    +        val blacklistedExecsOnNode = 
nodeToBlacklistedExecs.getOrElseUpdate(exec, HashSet[String]())
    +        blacklistedExecsOnNode += exec
    +
    +        if (conf.getBoolean("spark.shuffle.service.enabled", false) &&
    +            !nodeIdToBlacklistExpiryTime.contains(host)) {
    +          logInfo(s"blacklisting node $host due to fetch failure of 
external shuffle service")
    +
    +          nodeIdToBlacklistExpiryTime.put(host, expiryTimeForNewBlacklists)
    +          listenerBus.post(SparkListenerNodeBlacklisted(now, exec, 
blacklistedExecsOnNode.size))
    --- End diff --
    
    the parameter to SparkListenerNodeBlacklisted  needs to be the node, not 
the exec.
    
    we should test on the UI to make sure this shows up correctly, I think 
there might be a race condition between this and when the fetchfailure code in 
dag scheduler would remove the executor so it doesn't always show up on UI


---
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