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

    https://github.com/apache/spark/pull/17113#discussion_r106246809
  
    --- Diff: 
core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala ---
    @@ -145,6 +146,63 @@ 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, 
numFailedTasks: Int): Unit = {
    +    if (BLACKLIST_FETCH_FAILURE_ENABLED) {
    +      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))
    +        listenerBus.post(SparkListenerExecutorBlacklisted(now, exec, 
numFailedTasks))
    +        updateNextExpiryTime()
    +        killBlacklistedExecutor(exec)
    +
    +        val blacklistedExecsOnNode = 
nodeToBlacklistedExecs.getOrElseUpdate(exec, HashSet[String]())
    +        blacklistedExecsOnNode += exec
    +
    +        if (SparkEnv.get.blockManager.externalShuffleServiceEnabled &&
    +          !nodeIdToBlacklistExpiryTime.contains(host)) {
    --- End diff --
    
    nit: double-indent second line of `if`.
    
    also -- this looks extremely drastic, so I think its worth including a 
comment here about why we do this, something like:  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.


---
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 [email protected] or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to