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

    https://github.com/apache/spark/pull/14079#discussion_r72502080
  
    --- Diff: 
core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala ---
    @@ -0,0 +1,214 @@
    +/*
    + * 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.internal.config
    +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).
    + *
    + * 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_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)
    +
    +  // a count of failed tasks for each executor.  Only counts failures 
after tasksets complete
    +  // successfully
    +  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 = 
executorIdToBlacklistStatus.filter(_._2.expiryTime < now).keys
    +      if (execsToClear.nonEmpty) {
    +        logInfo(s"Removing executors $execsToClear from blacklist during 
periodic recovery")
    --- End diff --
    
    Removing nodes $nodesToClear from blacklist because the blacklist has 
expired (or "timed out")?


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